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 2022/11/02 08:49:14 UTC

[plc4x] branch feature/cdutz/go-ads-ng updated: feat(ads): ADS Auto-Discovery

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

cdutz pushed a commit to branch feature/cdutz/go-ads-ng
in repository https://gitbox.apache.org/repos/asf/plc4x.git


The following commit(s) were added to refs/heads/feature/cdutz/go-ads-ng by this push:
     new c3d00f5bb feat(ads): ADS Auto-Discovery
c3d00f5bb is described below

commit c3d00f5bba64808442edc8f892901bdf48fc2505
Author: Christofer Dutz <ch...@c-ware.de>
AuthorDate: Wed Nov 2 09:49:04 2022 +0100

    feat(ads): ADS Auto-Discovery
    
    First working version that correctly sends out the requests to all valid IPs and then correctly processes the responses
---
 .../plugins/codegenerator/language/mspec/MSpec.g4  |   6 +-
 plc4go/internal/ads/Discoverer.go                  | 249 +++++++++++
 plc4go/internal/ads/Discoverer_test.go             |  17 +
 plc4go/internal/ads/DiscoveryMessageCodec.go       | 100 +++++
 plc4go/internal/ads/Driver.go                      |  22 +-
 plc4go/internal/ads/Field.go                       | 163 +++-----
 plc4go/internal/ads/FieldHandler.go                | 238 +++++++----
 plc4go/internal/ads/FieldHandler_test.go           | 374 +++++++++++++++++
 plc4go/internal/ads/Reader.go                      | 365 ++++++++--------
 plc4go/internal/ads/Writer.go                      | 220 +++++-----
 plc4go/internal/ads/fieldtype_string.go            |  54 ---
 plc4go/internal/cbus/fieldtype_string.go           |  17 -
 plc4go/internal/knxnetip/ConnectionHelper.go       |   2 +-
 plc4go/internal/knxnetip/Discoverer.go             |  17 +-
 plc4go/internal/modbus/fieldtype_string.go         |  17 -
 plc4go/internal/s7/fieldtype_string.go             |  17 -
 plc4go/pkg/api/model/plc_field.go                  |   5 +-
 plc4go/pom.xml                                     |  13 +
 .../ads/discovery/readwrite/ParserHelper.go}       |  31 +-
 .../ads/discovery/readwrite/XmlParserHelper.go     |  57 +++
 .../ads/discovery/readwrite/model/AdsDiscovery.go  | 372 +++++++++++++++++
 .../discovery/readwrite/model/AdsDiscoveryBlock.go | 217 ++++++++++
 .../readwrite/model/AdsDiscoveryBlockAmsNetId.go   | 238 +++++++++++
 .../model/AdsDiscoveryBlockFingerprint.go          | 209 ++++++++++
 .../readwrite/model/AdsDiscoveryBlockHostName.go   | 203 +++++++++
 .../readwrite/model/AdsDiscoveryBlockOsData.go     | 209 ++++++++++
 .../readwrite/model/AdsDiscoveryBlockPassword.go   | 203 +++++++++
 .../readwrite/model/AdsDiscoveryBlockRouteName.go  | 203 +++++++++
 .../readwrite/model/AdsDiscoveryBlockStatus.go     | 238 +++++++++++
 .../readwrite/model/AdsDiscoveryBlockType.go       | 184 ++++++++
 .../readwrite/model/AdsDiscoveryBlockUserName.go   | 203 +++++++++
 .../readwrite/model/AdsDiscoveryBlockVersion.go    | 209 ++++++++++
 .../readwrite/model/AdsDiscoveryConstants.go       | 159 +++++++
 .../discovery/readwrite/model/AdsPortNumbers.go    | 216 ++++++++++
 .../ads/discovery/readwrite/model/AmsNetId.go      | 284 +++++++++++++
 .../ads/discovery/readwrite/model/AmsString.go     | 215 ++++++++++
 .../ads/discovery/readwrite/model/Operation.go     | 176 ++++++++
 .../ads/discovery/readwrite/model/Status.go        | 136 ++++++
 .../ads/discovery/readwrite/model/plc4x_common.go  |  27 ++
 .../bacnetip/readwrite/model/BACnetVendorId.go     | 464 +++++++++++++++++++++
 .../knxnetip/readwrite/model/KnxManufacturer.go    |  58 ++-
 plc4go/spi/model/DefaultPlcWriteRequest.go         |   5 +-
 plc4go/spi/testutils/steptype_string.go            |  17 -
 plc4go/spi/values/IEC61131ValueHandler.go          |  59 +--
 .../spi/transaction/RequestTransactionManager.java |  22 +-
 plc4j/tools/pom.xml                                |   2 +-
 .../knxnetip/readwrite/model/KnxManufacturer.cs    |  40 +-
 47 files changed, 5891 insertions(+), 661 deletions(-)

diff --git a/code-generation/protocol-base-mspec/src/main/antlr4/org/apache/plc4x/plugins/codegenerator/language/mspec/MSpec.g4 b/code-generation/protocol-base-mspec/src/main/antlr4/org/apache/plc4x/plugins/codegenerator/language/mspec/MSpec.g4
index f2f825369..972c84311 100644
--- a/code-generation/protocol-base-mspec/src/main/antlr4/org/apache/plc4x/plugins/codegenerator/language/mspec/MSpec.g4
+++ b/code-generation/protocol-base-mspec/src/main/antlr4/org/apache/plc4x/plugins/codegenerator/language/mspec/MSpec.g4
@@ -343,13 +343,13 @@ STRING_CHARACTER
 // Stuff we just want to ignore
 
 LINE_COMMENT
- : '//' ~[\r\n]* -> skip
+ : '//' ~[\r\n]* -> channel(HIDDEN)
  ;
 
 BLOCK_COMMENT
- : '/*' .*? '*/' -> skip
+ : '/*' .*? '*/' -> channel(HIDDEN)
  ;
 
 WS
- : [ \t\r\n\u000C]+ -> skip
+ : [ \t\r\n\u000C]+ -> channel(HIDDEN)
  ;
diff --git a/plc4go/internal/ads/Discoverer.go b/plc4go/internal/ads/Discoverer.go
new file mode 100644
index 000000000..7179af8ab
--- /dev/null
+++ b/plc4go/internal/ads/Discoverer.go
@@ -0,0 +1,249 @@
+/*
+ * 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 ads
+
+import (
+	"context"
+	"encoding/binary"
+	"fmt"
+	"net"
+	"net/url"
+	"strconv"
+	"time"
+
+	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/pkg/api/values"
+	"github.com/apache/plc4x/plc4go/protocols/ads/discovery/readwrite/model"
+	driverModel "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
+	"github.com/apache/plc4x/plc4go/spi"
+	internalModel "github.com/apache/plc4x/plc4go/spi/model"
+	"github.com/apache/plc4x/plc4go/spi/options"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	values2 "github.com/apache/plc4x/plc4go/spi/values"
+	"github.com/rs/zerolog/log"
+)
+
+type Discoverer struct {
+	messageCodec spi.MessageCodec
+}
+
+func NewDiscoverer() *Discoverer {
+	return &Discoverer{}
+}
+
+func (d *Discoverer) Discover(ctx context.Context, callback func(event apiModel.PlcDiscoveryItem), discoveryOptions ...options.WithDiscoveryOption) error {
+
+	////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+	// Set up a listening socket on all devices for processing the responses to any search requests
+
+	// Open a listening udp socket for the incoming responses
+	responseAddr, err := net.ResolveUDPAddr("udp4", fmt.Sprintf(":%d", model.AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT))
+	if err != nil {
+		panic(err)
+	}
+	socket, err := net.ListenUDP("udp4", responseAddr)
+	if err != nil {
+		panic(err)
+	}
+	defer socket.Close()
+
+	// Start a worker to receive responses
+	go func() {
+		buf := make([]byte, 1024)
+		for {
+			length, fromAddr, err := socket.ReadFromUDP(buf)
+			if length == 0 {
+				continue
+			}
+			readBuffer := utils.NewLittleEndianReadBufferByteBased(buf[0:length])
+			discoveryResponse, err := model.AdsDiscoveryParse(readBuffer)
+			if err != nil {
+				log.Error().Err(err).Str("src-ip", fromAddr.String()).Msg("error decoding response")
+				continue
+			}
+
+			if (discoveryResponse.GetRequestId() == 0) &&
+				(discoveryResponse.GetPortNumber() == model.AdsPortNumbers_SYSTEM_SERVICE) &&
+				(discoveryResponse.GetOperation() == model.Operation_DISCOVERY_RESPONSE) {
+				remoteAmsNetId := discoveryResponse.GetAmsNetId()
+				var hostNameBlock model.AdsDiscoveryBlockHostName
+				//var osDataBlock model.AdsDiscoveryBlockOsData
+				var versionBlock model.AdsDiscoveryBlockVersion
+				var fingerprintBlock model.AdsDiscoveryBlockFingerprint
+				for _, block := range discoveryResponse.GetBlocks() {
+					switch block.GetBlockType() {
+					case model.AdsDiscoveryBlockType_HOST_NAME:
+						hostNameBlock = block.(model.AdsDiscoveryBlockHostName)
+						/*									case model.AdsDiscoveryBlockType_OS_DATA:
+															osDataBlock = block.(model.AdsDiscoveryBlockOsData)*/
+					case model.AdsDiscoveryBlockType_VERSION:
+						versionBlock = block.(model.AdsDiscoveryBlockVersion)
+					case model.AdsDiscoveryBlockType_FINGERPRINT:
+						fingerprintBlock = block.(model.AdsDiscoveryBlockFingerprint)
+					}
+				}
+
+				if hostNameBlock != nil {
+					opts := make(map[string][]string)
+					//					opts["sourceAmsNetId"] = []string{localIpV4Address.String() + ".1.1"}
+					opts["sourceAmsPort"] = []string{"65534"}
+					opts["targetAmsNetId"] = []string{strconv.Itoa(int(remoteAmsNetId.GetOctet1())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet2())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet3())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet4())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet5())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet6()))}
+					// TODO: Check if this is legit, or if we can get the information from somewhere.
+					opts["targetAmsPort"] = []string{"851"}
+
+					attributes := make(map[string]values.PlcValue)
+					attributes["hostName"] = values2.NewPlcSTRING(hostNameBlock.GetHostName().GetText())
+					if versionBlock != nil {
+						versionData := versionBlock.GetVersionData()
+						patchVersion := (int(versionData[3])&0xFF)<<8 | (int(versionData[2]) & 0xFF)
+						attributes["twinCatVersion"] = values2.NewPlcSTRING(fmt.Sprintf("%d.%d.%d", int(versionData[0])&0xFF, int(versionData[1])&0xFF, patchVersion))
+					}
+					if fingerprintBlock != nil {
+						attributes["fingerprint"] = values2.NewPlcSTRING(string(fingerprintBlock.GetData()))
+					}
+					// TODO: Find out how to handle the OS Data
+
+					// Add an entry to the results.
+					remoteAddress, err2 := url.Parse("udp://" + strconv.Itoa(int(remoteAmsNetId.GetOctet1())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet2())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet3())) + "." +
+						strconv.Itoa(int(remoteAmsNetId.GetOctet4())) + ":" +
+						strconv.Itoa(int(driverModel.AdsConstants_ADSTCPDEFAULTPORT)))
+					if err2 == nil {
+						plcDiscoveryItem := &internalModel.DefaultPlcDiscoveryItem{
+							ProtocolCode:  "ads",
+							TransportCode: "tcp",
+							TransportUrl:  *remoteAddress,
+							Options:       opts,
+							Name:          hostNameBlock.GetHostName().GetText(),
+							Attributes:    attributes,
+						}
+
+						// Pass the event back to the callback
+						callback(plcDiscoveryItem)
+					}
+				}
+			}
+		}
+	}()
+
+	////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+	// Find out which interfaces to use for sending out search requests
+
+	allInterfaces, err := net.Interfaces()
+	if err != nil {
+		return err
+	}
+
+	// If no device is explicitly selected via option, simply use all of them
+	// However if a discovery option is present to select a device by name, only
+	// add those devices matching any of the given names.
+	var interfaces []net.Interface
+	deviceNames := options.FilterDiscoveryOptionsDeviceName(discoveryOptions)
+	if len(deviceNames) > 0 {
+		for _, curInterface := range allInterfaces {
+			for _, deviceNameOption := range deviceNames {
+				if curInterface.Name == deviceNameOption.GetDeviceName() {
+					interfaces = append(interfaces, curInterface)
+					break
+				}
+			}
+		}
+	} else {
+		interfaces = allInterfaces
+	}
+
+	////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
+	// Send out search requests on all selected interfaces
+
+	// Iterate over all network devices of this system.
+	for _, interf := range interfaces {
+		addrs, err := interf.Addrs()
+		if err != nil {
+			return err
+		}
+		// Iterate over all addresses the current interface has configured
+		// For ADS we're only interested in IPv4 addresses, as it doesn't
+		// seem to work with IPv6.
+		for _, addr := range addrs {
+			var ipv4Addr net.IP
+			switch addr.(type) {
+			// If the device is configured to communicate with a subnet
+			case *net.IPNet:
+				ipv4Addr = addr.(*net.IPNet).IP.To4()
+
+			// If the device is configured for a point-to-point connection
+			case *net.IPAddr:
+				ipv4Addr = addr.(*net.IPAddr).IP.To4()
+			}
+
+			// If we found an IPv4 address and this is not a loopback address,
+			// add it to the list of devices we will open ports and send discovery
+			// messages from.
+			if ipv4Addr != nil && !ipv4Addr.IsLoopback() {
+				// Calculate the broadcast address for this interface
+				broadcastAddress := make(net.IP, len(ipv4Addr))
+				binary.BigEndian.PutUint32(broadcastAddress, binary.BigEndian.Uint32(ipv4Addr)|^binary.BigEndian.Uint32(net.IP(addr.(*net.IPNet).Mask).To4()))
+
+				// Prepare the discovery packet data
+				// Create the discovery request message for this device.
+				amsNetId := model.NewAmsNetId(ipv4Addr[0], ipv4Addr[1], ipv4Addr[2], ipv4Addr[3], uint8(1), uint8(1))
+				discoveryRequestMessage := model.NewAdsDiscovery(0, model.Operation_DISCOVERY_REQUEST, amsNetId, model.AdsPortNumbers_SYSTEM_SERVICE, []model.AdsDiscoveryBlock{})
+
+				// Serialize the message
+				writeBuffer := utils.NewLittleEndianWriteBufferByteBased()
+				discoveryRequestMessage.Serialize(writeBuffer)
+
+				// Create a not-connected UDP connection to the broadcast address
+				requestAddr, err := net.ResolveUDPAddr("udp4", fmt.Sprintf("%s:%d", broadcastAddress.String(), model.AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT))
+				if err != nil {
+					log.Error().Err(err).Str("broadcast-ip", broadcastAddress.String()).Msg("Error resolving target socket for broadcast search")
+					continue
+				}
+				/*localAddr, err := net.ResolveUDPAddr("udp4", fmt.Sprintf("%s:%d", ipv4Addr.String(), model.AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT))
+				if err != nil {
+					log.Error().Err(err).Str("local-ip", ipv4Addr.String()).Msg("Error resolving local address for broadcast search")
+					continue
+				}
+				udp, err := net.DialUDP("udp4", localAddr, requestAddr)
+				if err != nil {
+					log.Error().Err(err).Str("local-ip", ipv4Addr.String()).Str("broadcast-ip", broadcastAddress.String()).
+						Msg("Error creating sending udp socket for broadcast search")
+					continue
+				}*/
+
+				// Send out the message.
+				_, err = socket.WriteTo(writeBuffer.GetBytes(), requestAddr)
+				if err != nil {
+					log.Error().Err(err).Str("broadcast-ip", broadcastAddress.String()).Msg("Error sending request for broadcast search")
+					continue
+				}
+			}
+		}
+	}
+
+	time.Sleep(time.Second * 10)
+	return nil
+}
diff --git a/plc4go/internal/ads/Discoverer_test.go b/plc4go/internal/ads/Discoverer_test.go
new file mode 100644
index 000000000..ec368aca2
--- /dev/null
+++ b/plc4go/internal/ads/Discoverer_test.go
@@ -0,0 +1,17 @@
+package ads
+
+import (
+	"context"
+	"testing"
+	"time"
+
+	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+)
+
+func TestDiscoverer(t *testing.T) {
+	discoverer := NewDiscoverer()
+	discoverer.Discover(context.Background(), func(event apiModel.PlcDiscoveryItem) {
+		print(event)
+	})
+	time.Sleep(time.Second * 30)
+}
diff --git a/plc4go/internal/ads/DiscoveryMessageCodec.go b/plc4go/internal/ads/DiscoveryMessageCodec.go
new file mode 100644
index 000000000..3aead461f
--- /dev/null
+++ b/plc4go/internal/ads/DiscoveryMessageCodec.go
@@ -0,0 +1,100 @@
+/*
+ * 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 ads
+
+import (
+	"github.com/apache/plc4x/plc4go/protocols/ads/discovery/readwrite/model"
+	"github.com/apache/plc4x/plc4go/spi"
+	"github.com/apache/plc4x/plc4go/spi/default"
+	"github.com/apache/plc4x/plc4go/spi/transports"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+	"github.com/rs/zerolog/log"
+)
+
+type DiscoveryMessageCodec struct {
+	_default.DefaultCodec
+}
+
+func NewDiscoveryMessageCodec(transportInstance transports.TransportInstance) *DiscoveryMessageCodec {
+	codec := &DiscoveryMessageCodec{}
+	codec.DefaultCodec = _default.NewDefaultCodec(codec, transportInstance)
+	return codec
+}
+
+func (m *DiscoveryMessageCodec) GetCodec() spi.MessageCodec {
+	return m
+}
+
+func (m *DiscoveryMessageCodec) Send(message spi.Message) error {
+	log.Trace().Msg("Sending message")
+	// Cast the message to the correct type of struct
+	tcpPaket := message.(model.AdsDiscovery)
+	// Serialize the request
+	wb := utils.NewLittleEndianWriteBufferByteBased()
+	err := tcpPaket.Serialize(wb)
+	if err != nil {
+		return errors.Wrap(err, "error serializing request")
+	}
+
+	// Send it to the PLC
+	err = m.GetTransportInstance().Write(wb.GetBytes())
+	if err != nil {
+		return errors.Wrap(err, "error sending request")
+	}
+	return nil
+}
+
+func (m *DiscoveryMessageCodec) Receive() (spi.Message, error) {
+	// We need at least 6 bytes in order to know how big the packet is in total
+	if num, err := m.GetTransportInstance().GetNumBytesAvailableInBuffer(); (err == nil) && (num >= 6) {
+		log.Debug().Msgf("we got %d readable bytes", num)
+		data, err := m.GetTransportInstance().PeekReadableBytes(6)
+		if err != nil {
+			log.Warn().Err(err).Msg("error peeking")
+			// TODO: Possibly clean up ...
+			return nil, nil
+		}
+		// Get the size of the entire packet little endian plus size of header
+		packetSize := (uint32(data[5]) << 24) + (uint32(data[4]) << 16) + (uint32(data[3]) << 8) + (uint32(data[2])) + 6
+		if num < packetSize {
+			log.Debug().Msgf("Not enough bytes. Got: %d Need: %d\n", num, packetSize)
+			return nil, nil
+		}
+		data, err = m.GetTransportInstance().Read(packetSize)
+		if err != nil {
+			// TODO: Possibly clean up ...
+			return nil, nil
+		}
+		rb := utils.NewLittleEndianReadBufferByteBased(data)
+		tcpPacket, err := model.AdsDiscoveryParse(rb)
+		if err != nil {
+			log.Warn().Err(err).Msg("error parsing")
+			// TODO: Possibly clean up ...
+			return nil, nil
+		}
+		return tcpPacket, nil
+	} else if err != nil {
+		log.Warn().Err(err).Msg("Got error reading")
+		return nil, nil
+	}
+	// TODO: maybe we return here a not enough error error
+	return nil, nil
+}
diff --git a/plc4go/internal/ads/Driver.go b/plc4go/internal/ads/Driver.go
index 61f33df4d..8f61fc636 100644
--- a/plc4go/internal/ads/Driver.go
+++ b/plc4go/internal/ads/Driver.go
@@ -20,12 +20,18 @@
 package ads
 
 import (
+	"context"
+	"net/url"
+	"strconv"
+
 	"github.com/apache/plc4x/plc4go/pkg/api"
+	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	adsModel "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
 	_default "github.com/apache/plc4x/plc4go/spi/default"
+	"github.com/apache/plc4x/plc4go/spi/options"
 	"github.com/apache/plc4x/plc4go/spi/transports"
 	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
-	"net/url"
 )
 
 type Driver struct {
@@ -51,7 +57,7 @@ func (m *Driver) GetConnection(transportUrl url.URL, transports map[string]trans
 		return ch
 	}
 	// Provide a default-port to the transport, which is used, if the user doesn't provide on in the connection string.
-	options["defaultTcpPort"] = []string{"48898"}
+	options["defaultTcpPort"] = []string{strconv.Itoa(int(adsModel.AdsConstants_ADSTCPDEFAULTPORT))}
 	// Have the transport create a new transport-instance.
 	transportInstance, err := transport.CreateTransportInstance(transportUrl, options)
 	if err != nil {
@@ -85,3 +91,15 @@ func (m *Driver) GetConnection(transportUrl url.URL, transports map[string]trans
 	log.Debug().Stringer("connection", connection).Msg("created connection, connecting now")
 	return connection.Connect()
 }
+
+func (m *Driver) SupportsDiscovery() bool {
+	return true
+}
+
+func (m *Driver) Discover(callback func(event apiModel.PlcDiscoveryItem), discoveryOptions ...options.WithDiscoveryOption) error {
+	return m.DiscoverWithContext(context.TODO(), callback, discoveryOptions...)
+}
+
+func (m *Driver) DiscoverWithContext(ctx context.Context, callback func(event apiModel.PlcDiscoveryItem), discoveryOptions ...options.WithDiscoveryOption) error {
+	return NewDiscoverer().Discover(ctx, callback, discoveryOptions...)
+}
diff --git a/plc4go/internal/ads/Field.go b/plc4go/internal/ads/Field.go
index 412a6015c..f82bcf5ba 100644
--- a/plc4go/internal/ads/Field.go
+++ b/plc4go/internal/ads/Field.go
@@ -21,81 +21,53 @@ package ads
 
 import (
 	"encoding/xml"
-	"fmt"
+
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
-	model2 "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
+	adsModel "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
 	"github.com/apache/plc4x/plc4go/spi/utils"
 	"github.com/pkg/errors"
 )
 
-type PlcField struct {
-	FieldType        FieldType
-	StringLength     int32
-	NumberOfElements uint32
-	Datatype         model2.AdsDataType
-}
-
-func (m PlcField) GetTypeName() string {
-	return m.FieldType.GetName()
-}
-
-func (m PlcField) GetQuantity() uint16 {
-	return uint16(m.NumberOfElements)
-}
-
-func (m PlcField) GetNumberOfElements() uint32 {
-	return m.NumberOfElements
-}
+const NONE = int32(-1)
 
-func (m PlcField) GetDatatype() model2.AdsDataType {
-	return m.Datatype
-}
-
-func (m PlcField) GetStringLength() int32 {
-	return m.StringLength
-}
-
-func (m PlcField) GetAddressString() string {
-	return fmt.Sprintf("%dx%05d%05d:%s", m.FieldType, m.StringLength, m.NumberOfElements, m.Datatype.String())
-}
-
-type AdsPlcField interface {
-	GetDatatype() model2.AdsDataType
-	GetStringLength() int32
-	GetNumberOfElements() uint32
+type PlcField struct {
 	model.PlcField
+
+	NumElements  int32
+	StartElement int32
+	EndElement   int32
 }
 
-func castToAdsFieldFromPlcField(plcField model.PlcField) (AdsPlcField, error) {
-	if adsField, ok := plcField.(AdsPlcField); ok {
-		return adsField, nil
+func needsResolving(plcField model.PlcField) bool {
+	switch plcField.(type) {
+	case SymbolicPlcField:
+		return true
+	case DirectPlcField:
+		return false
+	default:
+		return false
 	}
-	return nil, errors.Errorf("couldn't %T cast to AdsPlcField", plcField)
 }
 
 type DirectPlcField struct {
-	IndexGroup  uint32
-	IndexOffset uint32
 	PlcField
-}
 
-func (m DirectPlcField) GetAddressString() string {
-	return fmt.Sprintf("%dx%05d%05d%05d%05d:%s", m.FieldType, m.IndexGroup, m.IndexOffset, m.StringLength, m.NumberOfElements, m.Datatype.String())
+	IndexGroup   uint32
+	IndexOffset  uint32
+	AdsDatatype  adsModel.AdsDataType
+	StringLength int32
 }
 
-func newDirectAdsPlcField(indexGroup uint32, indexOffset uint32, adsDataType model2.AdsDataType, stringLength int32, numberOfElements uint32) (model.PlcField, error) {
-	fieldType := DirectAdsField
-	if stringLength > 0 {
-		fieldType = DirectAdsStringField
-	}
+func newDirectAdsPlcField(indexGroup uint32, indexOffset uint32, adsDatatype adsModel.AdsDataType, stringLength int32, numElements int32, startElement int32, endElement int32) (model.PlcField, error) {
 	return DirectPlcField{
-		IndexGroup:  indexGroup,
-		IndexOffset: indexOffset,
+		IndexGroup:   indexGroup,
+		IndexOffset:  indexOffset,
+		AdsDatatype:  adsDatatype,
+		StringLength: stringLength,
 		PlcField: PlcField{
-			FieldType:        fieldType,
-			StringLength:     stringLength,
-			NumberOfElements: numberOfElements,
-			Datatype:         adsDataType,
+			NumElements:  numElements,
+			StartElement: startElement,
+			EndElement:   endElement,
 		},
 	}, nil
 }
@@ -108,7 +80,7 @@ func castToDirectAdsFieldFromPlcField(plcField model.PlcField) (DirectPlcField,
 }
 
 func (m DirectPlcField) Serialize(writeBuffer utils.WriteBuffer) error {
-	if err := writeBuffer.PushContext(m.FieldType.GetName()); err != nil {
+	if err := writeBuffer.PushContext("DirectPlcField"); err != nil {
 		return err
 	}
 
@@ -118,22 +90,29 @@ func (m DirectPlcField) Serialize(writeBuffer utils.WriteBuffer) error {
 	if err := writeBuffer.WriteUint32("indexOffset", 32, m.IndexOffset); err != nil {
 		return err
 	}
-
-	if err := writeBuffer.WriteUint32("numberOfElements", 32, m.NumberOfElements); err != nil {
-		return err
-	}
-
-	if err := writeBuffer.WriteString("dataType", uint32(len([]rune(m.Datatype.String()))*8), "UTF-8", m.Datatype.String()); err != nil {
+	if err := writeBuffer.WriteString("adsDatatypeName", uint32(len([]rune(m.AdsDatatype.String()))*8), "UTF-8", m.AdsDatatype.String()); err != nil {
 		return err
 	}
-
-	if m.StringLength != 0 {
+	if (m.AdsDatatype == adsModel.AdsDataType_STRING || m.AdsDatatype == adsModel.AdsDataType_WSTRING) && (m.StringLength != NONE) {
 		if err := writeBuffer.WriteInt32("stringLength", 32, m.StringLength); err != nil {
 			return err
 		}
 	}
+	if m.NumElements != NONE {
+		if err := writeBuffer.WriteInt32("numElements", 32, m.NumElements); err != nil {
+			return err
+		}
+	}
+	if m.StartElement != NONE && m.EndElement != NONE {
+		if err := writeBuffer.WriteInt32("startElement", 32, m.StartElement); err != nil {
+			return err
+		}
+		if err := writeBuffer.WriteInt32("endElement", 32, m.EndElement); err != nil {
+			return err
+		}
+	}
 
-	if err := writeBuffer.PopContext(m.FieldType.GetName()); err != nil {
+	if err := writeBuffer.PopContext("DirectPlcField"); err != nil {
 		return err
 	}
 	return nil
@@ -144,41 +123,22 @@ func (m DirectPlcField) MarshalXMLAttr(name xml.Name) (xml.Attr, error) {
 }
 
 type SymbolicPlcField struct {
-	SymbolicAddress string
 	PlcField
-}
 
-func (m SymbolicPlcField) GetAddressString() string {
-	return fmt.Sprintf("%dx%s%05d%05d:%s", m.FieldType, m.SymbolicAddress, m.StringLength, m.NumberOfElements, m.Datatype.String())
+	SymbolicAddress string
 }
 
-func newAdsSymbolicPlcField(symbolicAddress string, adsDataType model2.AdsDataType, stringLength int32, numberOfElements uint32) (model.PlcField, error) {
-	fieldType := SymbolicAdsField
-	if stringLength > 0 {
-		fieldType = SymbolicAdsStringField
-	}
+func newAdsSymbolicPlcField(symbolicAddress string, numElements int32, startElement int32, endElement int32) (model.PlcField, error) {
 	return SymbolicPlcField{
 		SymbolicAddress: symbolicAddress,
 		PlcField: PlcField{
-			FieldType:        fieldType,
-			StringLength:     stringLength,
-			NumberOfElements: numberOfElements,
-			Datatype:         adsDataType,
+			NumElements:  numElements,
+			StartElement: startElement,
+			EndElement:   endElement,
 		},
 	}, nil
 }
 
-func needsResolving(plcField model.PlcField) bool {
-	switch plcField.(type) {
-	case SymbolicPlcField:
-		return true
-	case DirectPlcField:
-		return false
-	default:
-		return false
-	}
-}
-
 func castToSymbolicPlcFieldFromPlcField(plcField model.PlcField) (SymbolicPlcField, error) {
 	if adsField, ok := plcField.(SymbolicPlcField); ok {
 		return adsField, nil
@@ -187,29 +147,28 @@ func castToSymbolicPlcFieldFromPlcField(plcField model.PlcField) (SymbolicPlcFie
 }
 
 func (m SymbolicPlcField) Serialize(writeBuffer utils.WriteBuffer) error {
-	if err := writeBuffer.PushContext(m.FieldType.GetName()); err != nil {
+	if err := writeBuffer.PushContext("SymbolicPlcField"); err != nil {
 		return err
 	}
 
 	if err := writeBuffer.WriteString("symbolicAddress", uint32(len([]rune(m.SymbolicAddress))*8), "UTF-8", m.SymbolicAddress); err != nil {
 		return err
 	}
-
-	if err := writeBuffer.WriteUint32("numberOfElements", 32, m.NumberOfElements); err != nil {
-		return err
-	}
-
-	if err := writeBuffer.WriteString("dataType", uint32(len([]rune(m.Datatype.String()))*8), "UTF-8", m.Datatype.String()); err != nil {
-		return err
+	if m.NumElements != NONE {
+		if err := writeBuffer.WriteInt32("numElements", 32, m.NumElements); err != nil {
+			return err
+		}
 	}
-
-	if m.StringLength > 0 {
-		if err := writeBuffer.WriteInt32("stringLength", 32, m.StringLength); err != nil {
+	if m.StartElement != NONE && m.EndElement != NONE {
+		if err := writeBuffer.WriteInt32("startElement", 32, m.StartElement); err != nil {
+			return err
+		}
+		if err := writeBuffer.WriteInt32("endElement", 32, m.EndElement); err != nil {
 			return err
 		}
 	}
 
-	if err := writeBuffer.PopContext(m.FieldType.GetName()); err != nil {
+	if err := writeBuffer.PopContext("SymbolicPlcField"); err != nil {
 		return err
 	}
 	return nil
diff --git a/plc4go/internal/ads/FieldHandler.go b/plc4go/internal/ads/FieldHandler.go
index 6f2c31816..ab71fd87a 100644
--- a/plc4go/internal/ads/FieldHandler.go
+++ b/plc4go/internal/ads/FieldHandler.go
@@ -22,42 +22,27 @@ package ads
 import (
 	"encoding/binary"
 	"encoding/hex"
-	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
-	model2 "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
-	"github.com/apache/plc4x/plc4go/spi/utils"
-	"github.com/pkg/errors"
-	"github.com/rs/zerolog/log"
+	"fmt"
 	"regexp"
 	"strconv"
-)
-
-type FieldType uint8
 
-//go:generate stringer -type FieldType
-const (
-	DirectAdsStringField   FieldType = 0x00
-	DirectAdsField         FieldType = 0x01
-	SymbolicAdsStringField FieldType = 0x03
-	SymbolicAdsField       FieldType = 0x04
+	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
 )
 
-func (i FieldType) GetName() string {
-	return i.String()
-}
-
 type FieldHandler struct {
-	directAdsStringField   *regexp.Regexp
-	directAdsField         *regexp.Regexp
-	symbolicAdsStringField *regexp.Regexp
-	symbolicAdsField       *regexp.Regexp
+	directAdsStringField *regexp.Regexp
+	directAdsField       *regexp.Regexp
+	symbolicAdsField     *regexp.Regexp
 }
 
 func NewFieldHandler() FieldHandler {
 	return FieldHandler{
-		directAdsStringField:   regexp.MustCompile(`^((0[xX](?P<indexGroupHex>[0-9a-fA-F]+))|(?P<indexGroup>\d+))/((0[xX](?P<indexOffsetHex>[0-9a-fA-F]+))|(?P<indexOffset>\d+)):(?P<adsDataType>STRING|WSTRING)\((?P<stringLength>\d{1,3})\)(\[(?P<numberOfElements>\d+)])?`),
-		directAdsField:         regexp.MustCompile(`^((0[xX](?P<indexGroupHex>[0-9a-fA-F]+))|(?P<indexGroup>\d+))/((0[xX](?P<indexOffsetHex>[0-9a-fA-F]+))|(?P<indexOffset>\d+)):(?P<adsDataType>\w+)(\[(?P<numberOfElements>\d+)])?`),
-		symbolicAdsStringField: regexp.MustCompile(`^(?P<symbolicAddress>.+):(?P<adsDataType>'STRING'|'WSTRING')\((?P<stringLength>\d{1,3})\)(\[(?P<numberOfElements>\d+)])?`),
-		symbolicAdsField:       regexp.MustCompile(`^(?P<symbolicAddress>.+):(?P<adsDataType>\w+)(\[(?P<numberOfElements>\d+)])?`),
+		directAdsStringField: regexp.MustCompile(`^((0[xX](?P<indexGroupHex>[0-9a-fA-F]+))|(?P<indexGroup>\d+))/((0[xX](?P<indexOffsetHex>[0-9a-fA-F]+))|(?P<indexOffset>\d+)):(?P<adsDataType>STRING|WSTRING)\((?P<stringLength>\d{1,3})\)((\[(?P<numElements>\d+)])|(\[(?P<startElement>\d+)\.\.(?P<endElement>\d+)])|(\[(?P<startElement2>\d+):(?P<numElements2>\d+)]))?`),
+		directAdsField:       regexp.MustCompile(`^((0[xX](?P<indexGroupHex>[0-9a-fA-F]+))|(?P<indexGroup>\d+))/((0[xX](?P<indexOffsetHex>[0-9a-fA-F]+))|(?P<indexOffset>\d+)):(?P<adsDataType>\w+)((\[(?P<numElements>\d+)])|(\[(?P<startElement>\d+)\.\.(?P<endElement>\d+)])|(\[(?P<startElement2>\d+):(?P<numElements2>\d+)]))?`),
+		symbolicAdsField:     regexp.MustCompile(`^(?P<symbolicAddress>[^\[]+)((\[(?P<numElements>\d+)])|(\[(?P<startElement>\d+)\.\.(?P<endElement>\d+)])|(\[(?P<startElement2>\d+):(?P<numElements2>\d+)]))?`),
 	}
 }
 
@@ -65,10 +50,14 @@ func (m FieldHandler) ParseQuery(query string) (apiModel.PlcField, error) {
 	if match := utils.GetSubgroupMatches(m.directAdsStringField, query); match != nil {
 		var indexGroup uint32
 		if indexGroupHexString := match["indexGroupHex"]; indexGroupHexString != "" {
-			decodeString, err := hex.DecodeString(indexGroupHexString[2:])
+			decodeString, err := hex.DecodeString(indexGroupHexString)
 			if err != nil {
 				return nil, errors.Wrap(err, "Error decoding index group")
 			}
+			// Fill up the array with missing bytes to get an array of size 4 bytes.
+			for i := len(decodeString); i < 4; i++ {
+				decodeString = append([]byte{0}, decodeString...)
+			}
 			indexGroup = binary.BigEndian.Uint32(decodeString)
 		} else {
 			parsedIndexGroup, err := strconv.ParseUint(match["indexGroup"], 10, 32)
@@ -79,10 +68,14 @@ func (m FieldHandler) ParseQuery(query string) (apiModel.PlcField, error) {
 		}
 		var indexOffset uint32
 		if indexOffsetHexString := match["indexOffsetHex"]; indexOffsetHexString != "" {
-			decodeString, err := hex.DecodeString(indexOffsetHexString[2:])
+			decodeString, err := hex.DecodeString(indexOffsetHexString)
 			if err != nil {
 				return nil, errors.Wrap(err, "Error decoding index group")
 			}
+			// Fill up the array with missing bytes to get an array of size 4 bytes.
+			for i := len(decodeString); i < 4; i++ {
+				decodeString = append([]byte{0}, decodeString...)
+			}
 			indexOffset = binary.BigEndian.Uint32(decodeString)
 		} else {
 			parsedIndexOffset, err := strconv.ParseUint(match["indexOffset"], 10, 32)
@@ -91,28 +84,73 @@ func (m FieldHandler) ParseQuery(query string) (apiModel.PlcField, error) {
 			}
 			indexOffset = uint32(parsedIndexOffset)
 		}
-		stringLength, err := strconv.ParseInt(match["stringLength"], 10, 32)
-		if err != nil {
-			return nil, errors.Wrap(err, "Error decoding string length")
+		adsDataTypeName := match["adsDataType"]
+		if adsDataTypeName == "" {
+			return nil, errors.Errorf("Missing ads data type")
 		}
-		numberOfElements, err := strconv.ParseUint(match["numberOfElements"], 10, 32)
+		adsDataType, ok := model.AdsDataTypeByName(adsDataTypeName)
+		if !ok {
+			return nil, fmt.Errorf("invalid ads data type")
+		}
+
+		stringLength := NONE
+		numElements := NONE
+		startElement := NONE
+		endElement := NONE
+
+		tmpStringLength, err := strconv.ParseInt(match["stringLength"], 10, 32)
 		if err != nil {
-			log.Trace().Msg("Falling back to number of elements 1")
-			numberOfElements = 1
+			return nil, errors.Wrap(err, "Error decoding string length")
 		}
+		stringLength = int32(tmpStringLength)
 
-		typeByName, ok := model2.AdsDataTypeByName(match["adsDataType"])
-		if !ok {
-			return nil, errors.Errorf("Unknown type %s", match["adsDataType"])
+		if match["numElements"] != "" {
+			tmpNumElements, err := strconv.ParseUint(match["numElements"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements': %v", err)
+			} else {
+				numElements = int32(tmpNumElements)
+			}
+		} else if match["startElement"] != "" && match["endElement"] != "" {
+			tmpStartElement, err := strconv.ParseUint(match["startElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement': %v", err)
+			} else {
+				startElement = int32(tmpStartElement)
+			}
+			tmpEndElement, err := strconv.ParseUint(match["endElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'endElement': %v", err)
+			} else {
+				endElement = int32(tmpEndElement)
+			}
+		} else if match["startElement2"] != "" && match["numElements2"] != "" {
+			tmpStartElement2, err := strconv.ParseUint(match["startElement2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement2': %v", err)
+			} else {
+				startElement = int32(tmpStartElement2)
+			}
+			tmpNumElements2, err := strconv.ParseUint(match["numElements2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements2': %v", err)
+			} else {
+				numElements = int32(tmpNumElements2)
+			}
 		}
-		return newDirectAdsPlcField(indexGroup, indexOffset, typeByName, int32(stringLength), uint32(numberOfElements))
+
+		return newDirectAdsPlcField(indexGroup, indexOffset, adsDataType, stringLength, numElements, startElement, endElement)
 	} else if match := utils.GetSubgroupMatches(m.directAdsField, query); match != nil {
 		var indexGroup uint32
 		if indexGroupHexString := match["indexGroupHex"]; indexGroupHexString != "" {
-			decodeString, err := hex.DecodeString(indexGroupHexString[2:])
+			decodeString, err := hex.DecodeString(indexGroupHexString)
 			if err != nil {
 				return nil, errors.Wrap(err, "Error decoding index group")
 			}
+			// Fill up the array with missing bytes to get an array of size 4 bytes.
+			for i := len(decodeString); i < 4; i++ {
+				decodeString = append([]byte{0}, decodeString...)
+			}
 			indexGroup = binary.BigEndian.Uint32(decodeString)
 		} else {
 			parsedIndexGroup, err := strconv.ParseUint(match["indexGroup"], 10, 32)
@@ -123,10 +161,14 @@ func (m FieldHandler) ParseQuery(query string) (apiModel.PlcField, error) {
 		}
 		var indexOffset uint32
 		if indexOffsetHexString := match["indexOffsetHex"]; indexOffsetHexString != "" {
-			decodeString, err := hex.DecodeString(indexOffsetHexString[2:])
+			decodeString, err := hex.DecodeString(indexOffsetHexString)
 			if err != nil {
 				return nil, errors.Wrap(err, "Error decoding index group")
 			}
+			// Fill up the array with missing bytes to get an array of size 4 bytes.
+			for i := len(decodeString); i < 4; i++ {
+				decodeString = append([]byte{0}, decodeString...)
+			}
 			indexOffset = binary.BigEndian.Uint32(decodeString)
 		} else {
 			parsedIndexOffset, err := strconv.ParseUint(match["indexOffset"], 10, 32)
@@ -135,42 +177,96 @@ func (m FieldHandler) ParseQuery(query string) (apiModel.PlcField, error) {
 			}
 			indexOffset = uint32(parsedIndexOffset)
 		}
-
-		typeByName, ok := model2.AdsDataTypeByName(match["adsDataType"])
-		if !ok {
-			return nil, errors.Errorf("Unknown type %s", match["adsDataType"])
+		adsDataTypeName := match["adsDataType"]
+		if adsDataTypeName == "" {
+			return nil, errors.Errorf("Missing ads data type")
 		}
-		numberOfElements, err := strconv.ParseUint(match["numberOfElements"], 10, 32)
-		if err != nil {
-			log.Trace().Msg("Falling back to number of elements 1")
-			numberOfElements = 1
-		}
-		return newDirectAdsPlcField(indexGroup, indexOffset, typeByName, int32(0), uint32(numberOfElements))
-	} else if match := utils.GetSubgroupMatches(m.symbolicAdsStringField, query); match != nil {
-		stringLength, err := strconv.ParseInt(match["stringLength"], 10, 32)
-		if err != nil {
-			return nil, errors.Wrap(err, "Error decoding string length")
-		}
-		numberOfElements, err := strconv.ParseUint(match["numberOfElements"], 10, 32)
-		if err != nil {
-			return nil, errors.Wrap(err, "Error decoding number of elements")
-		}
-		typeByName, ok := model2.AdsDataTypeByName(match["adsDataType"])
+		adsDataType, ok := model.AdsDataTypeByName(adsDataTypeName)
 		if !ok {
-			return nil, errors.Errorf("Unknown type %s", match["adsDataType"])
+			return nil, fmt.Errorf("invalid ads data type")
 		}
-		return newAdsSymbolicPlcField(match["symbolicAddress"], typeByName, int32(stringLength), uint32(numberOfElements))
-	} else if match := utils.GetSubgroupMatches(m.symbolicAdsField, query); match != nil {
-		numberOfElements, err := strconv.ParseUint(match["numberOfElements"], 10, 32)
-		if err != nil {
-			log.Trace().Msg("Falling back to number of elements 1")
-			numberOfElements = 1
+
+		numElements := NONE
+		startElement := NONE
+		endElement := NONE
+
+		if match["numElements"] != "" {
+			tmpNumElements, err := strconv.ParseUint(match["numElements"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements': %v", err)
+			} else {
+				numElements = int32(tmpNumElements)
+			}
+		} else if match["startElement"] != "" && match["endElement"] != "" {
+			tmpStartElement, err := strconv.ParseUint(match["startElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement': %v", err)
+			} else {
+				startElement = int32(tmpStartElement)
+			}
+			tmpEndElement, err := strconv.ParseUint(match["endElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'endElement': %v", err)
+			} else {
+				endElement = int32(tmpEndElement)
+			}
+		} else if match["startElement2"] != "" && match["numElements2"] != "" {
+			tmpStartElement2, err := strconv.ParseUint(match["startElement2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement2': %v", err)
+			} else {
+				startElement = int32(tmpStartElement2)
+			}
+			tmpNumElements2, err := strconv.ParseUint(match["numElements2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements2': %v", err)
+			} else {
+				numElements = int32(tmpNumElements2)
+			}
 		}
-		typeByName, ok := model2.AdsDataTypeByName(match["adsDataType"])
-		if !ok {
-			return nil, errors.Errorf("Unknown type %s", match["adsDataType"])
+
+		return newDirectAdsPlcField(indexGroup, indexOffset, adsDataType, NONE, numElements, startElement, endElement)
+	} else if match := utils.GetSubgroupMatches(m.symbolicAdsField, query); match != nil {
+		numElements := NONE
+		startElement := NONE
+		endElement := NONE
+
+		if match["numElements"] != "" {
+			tmpNumElements, err := strconv.ParseUint(match["numElements"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements': %v", err)
+			} else {
+				numElements = int32(tmpNumElements)
+			}
+		} else if match["startElement"] != "" && match["endElement"] != "" {
+			tmpStartElement, err := strconv.ParseUint(match["startElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement': %v", err)
+			} else {
+				startElement = int32(tmpStartElement)
+			}
+			tmpEndElement, err := strconv.ParseUint(match["endElement"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'endElement': %v", err)
+			} else {
+				endElement = int32(tmpEndElement)
+			}
+		} else if match["startElement2"] != "" && match["numElements2"] != "" {
+			tmpStartElement2, err := strconv.ParseUint(match["startElement2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'startElement2': %v", err)
+			} else {
+				startElement = int32(tmpStartElement2)
+			}
+			tmpNumElements2, err := strconv.ParseUint(match["numElements2"], 10, 32)
+			if err != nil {
+				return nil, fmt.Errorf("invalid address format parsing 'numElements2': %v", err)
+			} else {
+				numElements = int32(tmpNumElements2)
+			}
 		}
-		return newAdsSymbolicPlcField(match["symbolicAddress"], typeByName, int32(0), uint32(numberOfElements))
+
+		return newAdsSymbolicPlcField(match["symbolicAddress"], numElements, startElement, endElement)
 	} else {
 		return nil, errors.Errorf("Invalid address format for address '%s'", query)
 	}
diff --git a/plc4go/internal/ads/FieldHandler_test.go b/plc4go/internal/ads/FieldHandler_test.go
new file mode 100644
index 000000000..c96e02b7a
--- /dev/null
+++ b/plc4go/internal/ads/FieldHandler_test.go
@@ -0,0 +1,374 @@
+package ads
+
+import (
+	"reflect"
+	"regexp"
+	"testing"
+
+	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	model2 "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
+)
+
+func TestFieldHandler_ParseQuery(t *testing.T) {
+	type fields struct {
+		directAdsStringField *regexp.Regexp
+		directAdsField       *regexp.Regexp
+		symbolicAdsField     *regexp.Regexp
+	}
+	type args struct {
+		query string
+	}
+	tests := []struct {
+		name    string
+		fields  fields
+		args    args
+		want    model.PlcField
+		wantErr bool
+	}{
+		// All tests without any array notation.
+		{
+			name: "simple direct numeric address",
+			args: args{
+				query: "1234/5678:BOOL",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "simple direct hex address",
+			args: args{
+				query: "0x04D2/0x162E:BOOL",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "simple direct numeric string address",
+			args: args{
+				query: "1234/5678:STRING(80)",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_STRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "simple direct hex string address",
+			args: args{
+				query: "0x04D2/0x162E:WSTRING(80)",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_WSTRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "simple symbolic address",
+			args: args{
+				query: "MAIN.testVariable",
+			},
+			want: SymbolicPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				SymbolicAddress: "MAIN.testVariable",
+			},
+		},
+		// All tests with simple array notation.
+		{
+			name: "simple array direct numeric address",
+			args: args{
+				query: "1234/5678:BOOL[42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "simple array direct hex address",
+			args: args{
+				query: "0x04D2/0x162E:BOOL[42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "simple array direct numeric string address",
+			args: args{
+				query: "1234/5678:STRING(80)[42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_STRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "simple array direct hex string address",
+			args: args{
+				query: "0x04D2/0x162E:WSTRING(80)[42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_WSTRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "simple array symbolic address",
+			args: args{
+				query: "MAIN.testVariable[42]",
+			},
+			want: SymbolicPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: NONE,
+					EndElement:   NONE,
+				},
+				SymbolicAddress: "MAIN.testVariable",
+			},
+		},
+		// All tests with range array notation.
+		{
+			name: "range array direct numeric address",
+			args: args{
+				query: "1234/5678:BOOL[23..42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: 23,
+					EndElement:   42,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "range array direct hex address",
+			args: args{
+				query: "0x04D2/0x162E:BOOL[23..42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: 23,
+					EndElement:   42,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "range array direct numeric string address",
+			args: args{
+				query: "1234/5678:STRING(80)[23..42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: 23,
+					EndElement:   42,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_STRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "range array direct hex string address",
+			args: args{
+				query: "0x04D2/0x162E:WSTRING(80)[23..42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: 23,
+					EndElement:   42,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_WSTRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "range array symbolic address",
+			args: args{
+				query: "MAIN.testVariable[23..42]",
+			},
+			want: SymbolicPlcField{
+				PlcField: PlcField{
+					NumElements:  NONE,
+					StartElement: 23,
+					EndElement:   42,
+				},
+				SymbolicAddress: "MAIN.testVariable",
+			},
+		},
+		// All tests with array with offset notation.
+		{
+			name: "array with offset direct numeric address",
+			args: args{
+				query: "1234/5678:BOOL[23:42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: 23,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "array with offset direct hex address",
+			args: args{
+				query: "0x04D2/0x162E:BOOL[23:42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: 23,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_BOOL,
+				StringLength: NONE,
+			},
+		},
+		{
+			name: "array with offset direct numeric string address",
+			args: args{
+				query: "1234/5678:STRING(80)[23:42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: 23,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_STRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "array with offset direct hex string address",
+			args: args{
+				query: "0x04D2/0x162E:WSTRING(80)[23:42]",
+			},
+			want: DirectPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: 23,
+					EndElement:   NONE,
+				},
+				IndexGroup:   1234,
+				IndexOffset:  5678,
+				AdsDatatype:  model2.AdsDataType_WSTRING,
+				StringLength: 80,
+			},
+		},
+		{
+			name: "array with offset symbolic address",
+			args: args{
+				query: "MAIN.testVariable[23:42]",
+			},
+			want: SymbolicPlcField{
+				PlcField: PlcField{
+					NumElements:  42,
+					StartElement: 23,
+					EndElement:   NONE,
+				},
+				SymbolicAddress: "MAIN.testVariable",
+			},
+		},
+	}
+	for _, tt := range tests {
+		t.Run(tt.name, func(t *testing.T) {
+			m := NewFieldHandler()
+			got, err := m.ParseQuery(tt.args.query)
+			if (err != nil) != tt.wantErr {
+				t.Errorf("ParseQuery() error = %v, wantErr %v", err, tt.wantErr)
+				return
+			}
+			if !reflect.DeepEqual(got, tt.want) {
+				t.Errorf("ParseQuery() got = %v, want %v", got, tt.want)
+			}
+		})
+	}
+}
diff --git a/plc4go/internal/ads/Reader.go b/plc4go/internal/ads/Reader.go
index 34219d6a1..215426c68 100644
--- a/plc4go/internal/ads/Reader.go
+++ b/plc4go/internal/ads/Reader.go
@@ -21,18 +21,17 @@ package ads
 
 import (
 	"context"
+	"math"
+	"sync"
+	"sync/atomic"
+	"time"
+
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
-	"github.com/apache/plc4x/plc4go/pkg/api/values"
 	readWriteModel "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
 	"github.com/apache/plc4x/plc4go/spi"
 	plc4goModel "github.com/apache/plc4x/plc4go/spi/model"
-	"github.com/apache/plc4x/plc4go/spi/utils"
 	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
-	"math"
-	"sync"
-	"sync/atomic"
-	"time"
 )
 
 type Reader struct {
@@ -73,30 +72,41 @@ func (m *Reader) Read(ctx context.Context, readRequest model.PlcReadRequest) <-c
 }
 
 func (m *Reader) singleRead(ctx context.Context, readRequest model.PlcReadRequest, result chan model.PlcReadRequestResult) {
-	if len(readRequest.GetFieldNames()) != 1 {
-		result <- &plc4goModel.DefaultPlcReadRequestResult{
-			Request:  readRequest,
-			Response: nil,
-			Err:      errors.New("ads only supports single-item requests"),
-		}
-		log.Debug().Msgf("ads only supports single-item requests. Got %d fields", len(readRequest.GetFieldNames()))
-		return
-	}
-	// If we are requesting only one field, use a
-	fieldName := readRequest.GetFieldNames()[0]
-	field := readRequest.GetField(fieldName)
-	if needsResolving(field) {
-		adsField, err := castToSymbolicPlcFieldFromPlcField(field)
-		if err != nil {
+	/*	if len(readRequest.GetFieldNames()) != 1 {
 			result <- &plc4goModel.DefaultPlcReadRequestResult{
 				Request:  readRequest,
 				Response: nil,
-				Err:      errors.Wrap(err, "invalid field item type"),
+				Err:      errors.New("ads only supports single-item requests"),
 			}
-			log.Debug().Msgf("Invalid field item type %T", field)
+			log.Debug().Msgf("ads only supports single-item requests. Got %d fields", len(readRequest.GetFieldNames()))
 			return
 		}
-		field, err = m.resolveField(ctx, adsField)
+		// If we are requesting only one field, use a
+		fieldName := readRequest.GetFieldNames()[0]
+		field := readRequest.GetField(fieldName)
+		if needsResolving(field) {
+			adsField, err := castToSymbolicPlcFieldFromPlcField(field)
+			if err != nil {
+				result <- &plc4goModel.DefaultPlcReadRequestResult{
+					Request:  readRequest,
+					Response: nil,
+					Err:      errors.Wrap(err, "invalid field item type"),
+				}
+				log.Debug().Msgf("Invalid field item type %T", field)
+				return
+			}
+			field, err = m.resolveField(ctx, adsField)
+			if err != nil {
+				result <- &plc4goModel.DefaultPlcReadRequestResult{
+					Request:  readRequest,
+					Response: nil,
+					Err:      errors.Wrap(err, "invalid field item type"),
+				}
+				log.Debug().Msgf("Invalid field item type %T", field)
+				return
+			}
+		}
+		adsField, err := castToDirectAdsFieldFromPlcField(field)
 		if err != nil {
 			result <- &plc4goModel.DefaultPlcReadRequestResult{
 				Request:  readRequest,
@@ -106,103 +116,103 @@ func (m *Reader) singleRead(ctx context.Context, readRequest model.PlcReadReques
 			log.Debug().Msgf("Invalid field item type %T", field)
 			return
 		}
-	}
-	adsField, err := castToDirectAdsFieldFromPlcField(field)
-	if err != nil {
-		result <- &plc4goModel.DefaultPlcReadRequestResult{
-			Request:  readRequest,
-			Response: nil,
-			Err:      errors.Wrap(err, "invalid field item type"),
-		}
-		log.Debug().Msgf("Invalid field item type %T", field)
-		return
-	}
-
-	readLength := uint32(adsField.Datatype.NumBytes())
-	switch {
-	case adsField.GetDatatype() == readWriteModel.AdsDataType_STRING:
-		// If an explicit size is given with the string, use this, if not use 256
-		if adsField.GetStringLength() != 0 {
-			readLength = uint32(adsField.GetStringLength())
-		} else {
-			readLength = 256
-		}
-	case adsField.GetDatatype() == readWriteModel.AdsDataType_WSTRING:
-		// If an explicit size is given with the string, use this, if not use 512
-		if adsField.GetStringLength() != 0 {
-			readLength = uint32(adsField.GetStringLength() * 2)
-		} else {
-			readLength = 512
-		}
-	default:
-		readLength = uint32(adsField.Datatype.NumBytes())
-	}
-	userdata := readWriteModel.NewAdsReadRequest(
-		adsField.IndexGroup,
-		adsField.IndexOffset,
-		readLength,
-		m.targetAmsNetId,
-		m.targetAmsPort,
-		m.sourceAmsNetId,
-		m.sourceAmsPort,
-		0,
-		m.getInvokeId())
-
-	m.sendOverTheWire(ctx, userdata, readRequest, result)
-}
 
-func (m *Reader) multiRead(ctx context.Context, readRequest model.PlcReadRequest, result chan model.PlcReadRequestResult) {
-	// Calculate the size of all fields together.
-	// Calculate the expected size of the response data.
-	expectedResponseDataSize := uint32(0)
-	for _, fieldName := range readRequest.GetFieldNames() {
-		field, err := castToAdsFieldFromPlcField(readRequest.GetField(fieldName))
-		if err != nil {
-			result <- &plc4goModel.DefaultPlcReadRequestResult{
-				Request:  readRequest,
-				Response: nil,
-				Err:      errors.Wrap(err, "error casting field"),
-			}
-			return
-		}
-		size := uint32(0)
-		switch field.GetDatatype() {
-		case readWriteModel.AdsDataType_STRING:
+		readLength := uint32(adsField.Datatype.NumBytes())
+		switch {
+		case adsField.GetDatatype() == readWriteModel.AdsDataType_STRING:
 			// If an explicit size is given with the string, use this, if not use 256
-			if field.GetStringLength() != 0 {
-				size = uint32(field.GetStringLength())
+			if adsField.GetStringLength() != 0 {
+				readLength = uint32(adsField.GetStringLength())
 			} else {
-				size = 256
+				readLength = 256
 			}
-		case readWriteModel.AdsDataType_WSTRING:
+		case adsField.GetDatatype() == readWriteModel.AdsDataType_WSTRING:
 			// If an explicit size is given with the string, use this, if not use 512
-			if field.GetStringLength() != 0 {
-				size = uint32(field.GetStringLength() * 2)
+			if adsField.GetStringLength() != 0 {
+				readLength = uint32(adsField.GetStringLength() * 2)
 			} else {
-				size = 512
+				readLength = 512
 			}
 		default:
-			size = uint32(field.GetDatatype().NumBytes())
+			readLength = uint32(adsField.Datatype.NumBytes())
 		}
-		// Status code + payload size
-		expectedResponseDataSize += 4 + (size * field.GetNumberOfElements())
-	}
+		userdata := readWriteModel.NewAdsReadRequest(
+			adsField.IndexGroup,
+			adsField.IndexOffset,
+			readLength,
+			m.targetAmsNetId,
+			m.targetAmsPort,
+			m.sourceAmsNetId,
+			m.sourceAmsPort,
+			0,
+			m.getInvokeId())
 
-	items := make([]readWriteModel.AdsMultiRequestItem, len(readRequest.GetFieldNames()))
-	for i, fieldName := range readRequest.GetFieldNames() {
-		field := readRequest.GetField(fieldName)
-		if needsResolving(field) {
-			adsField, err := castToSymbolicPlcFieldFromPlcField(field)
+		m.sendOverTheWire(ctx, userdata, readRequest, result)*/
+}
+
+func (m *Reader) multiRead(ctx context.Context, readRequest model.PlcReadRequest, result chan model.PlcReadRequestResult) {
+	/*	// Calculate the size of all fields together.
+		// Calculate the expected size of the response data.
+		expectedResponseDataSize := uint32(0)
+		for _, fieldName := range readRequest.GetFieldNames() {
+			field, err := castToAdsFieldFromPlcField(readRequest.GetField(fieldName))
 			if err != nil {
 				result <- &plc4goModel.DefaultPlcReadRequestResult{
 					Request:  readRequest,
 					Response: nil,
-					Err:      errors.Wrap(err, "invalid field item type"),
+					Err:      errors.Wrap(err, "error casting field"),
 				}
-				log.Debug().Msgf("Invalid field item type %T", field)
 				return
 			}
-			field, err = m.resolveField(ctx, adsField)
+			size := uint32(0)
+			switch field.GetDatatype() {
+			case readWriteModel.AdsDataType_STRING:
+				// If an explicit size is given with the string, use this, if not use 256
+				if field.GetStringLength() != 0 {
+					size = uint32(field.GetStringLength())
+				} else {
+					size = 256
+				}
+			case readWriteModel.AdsDataType_WSTRING:
+				// If an explicit size is given with the string, use this, if not use 512
+				if field.GetStringLength() != 0 {
+					size = uint32(field.GetStringLength() * 2)
+				} else {
+					size = 512
+				}
+			default:
+				size = uint32(field.GetDatatype().NumBytes())
+			}
+			// Status code + payload size
+			expectedResponseDataSize += 4 + (size * field.GetNumberOfElements())
+		}
+
+		items := make([]readWriteModel.AdsMultiRequestItem, len(readRequest.GetFieldNames()))
+		for i, fieldName := range readRequest.GetFieldNames() {
+			field := readRequest.GetField(fieldName)
+			if needsResolving(field) {
+				adsField, err := castToSymbolicPlcFieldFromPlcField(field)
+				if err != nil {
+					result <- &plc4goModel.DefaultPlcReadRequestResult{
+						Request:  readRequest,
+						Response: nil,
+						Err:      errors.Wrap(err, "invalid field item type"),
+					}
+					log.Debug().Msgf("Invalid field item type %T", field)
+					return
+				}
+				field, err = m.resolveField(ctx, adsField)
+				if err != nil {
+					result <- &plc4goModel.DefaultPlcReadRequestResult{
+						Request:  readRequest,
+						Response: nil,
+						Err:      errors.Wrap(err, "invalid field item type"),
+					}
+					log.Debug().Msgf("Invalid field item type %T", field)
+					return
+				}
+			}
+			adsField, err := castToDirectAdsFieldFromPlcField(field)
 			if err != nil {
 				result <- &plc4goModel.DefaultPlcReadRequestResult{
 					Request:  readRequest,
@@ -212,34 +222,23 @@ func (m *Reader) multiRead(ctx context.Context, readRequest model.PlcReadRequest
 				log.Debug().Msgf("Invalid field item type %T", field)
 				return
 			}
+			// With multi-requests, the index-group is fixed and the index offset indicates the number of elements.
+			items[i] = readWriteModel.NewAdsMultiRequestItemRead(adsField.IndexGroup, adsField.IndexOffset, uint32(adsField.GetDatatype().NumBytes())*adsField.NumberOfElements)
 		}
-		adsField, err := castToDirectAdsFieldFromPlcField(field)
-		if err != nil {
-			result <- &plc4goModel.DefaultPlcReadRequestResult{
-				Request:  readRequest,
-				Response: nil,
-				Err:      errors.Wrap(err, "invalid field item type"),
-			}
-			log.Debug().Msgf("Invalid field item type %T", field)
-			return
-		}
-		// With multi-requests, the index-group is fixed and the index offset indicates the number of elements.
-		items[i] = readWriteModel.NewAdsMultiRequestItemRead(adsField.IndexGroup, adsField.IndexOffset, uint32(adsField.GetDatatype().NumBytes())*adsField.NumberOfElements)
-	}
-	userdata := readWriteModel.NewAdsReadWriteRequest(
-		uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_MULTIPLE_READ),
-		uint32(len(readRequest.GetFieldNames())),
-		expectedResponseDataSize,
-		items,
-		nil,
-		m.targetAmsNetId,
-		m.targetAmsPort,
-		m.sourceAmsNetId,
-		m.sourceAmsPort,
-		0,
-		m.getInvokeId())
+		userdata := readWriteModel.NewAdsReadWriteRequest(
+			uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_MULTIPLE_READ),
+			uint32(len(readRequest.GetFieldNames())),
+			expectedResponseDataSize,
+			items,
+			nil,
+			m.targetAmsNetId,
+			m.targetAmsPort,
+			m.sourceAmsNetId,
+			m.sourceAmsPort,
+			0,
+			m.getInvokeId())
 
-	m.sendOverTheWire(ctx, userdata, readRequest, result)
+		m.sendOverTheWire(ctx, userdata, readRequest, result)*/
 }
 
 func (m *Reader) sendOverTheWire(ctx context.Context, userdata readWriteModel.AmsPacket, readRequest model.PlcReadRequest, result chan model.PlcReadRequestResult) {
@@ -297,60 +296,61 @@ func (m *Reader) sendOverTheWire(ctx context.Context, userdata readWriteModel.Am
 }
 
 func (m *Reader) resolveField(ctx context.Context, symbolicField SymbolicPlcField) (DirectPlcField, error) {
-	if directPlcField, ok := m.fieldMapping[symbolicField]; ok {
-		return directPlcField, nil
-	}
-	m.mappingLock.Lock()
-	defer m.mappingLock.Unlock()
-	// In case a previous one has already
-	if directPlcField, ok := m.fieldMapping[symbolicField]; ok {
-		return directPlcField, nil
-	}
-	userdata := readWriteModel.NewAdsReadWriteRequest(
-		uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_SYM_HNDBYNAME),
-		0,
-		4,
-		nil,
-		[]byte(symbolicField.SymbolicAddress+"\000"),
-		m.targetAmsNetId,
-		m.targetAmsPort,
-		m.sourceAmsNetId,
-		m.sourceAmsPort,
-		0,
-		m.getInvokeId())
-	result := make(chan model.PlcReadRequestResult)
-	go func() {
-		dummyRequest := plc4goModel.NewDefaultPlcReadRequest(map[string]model.PlcField{"dummy": DirectPlcField{PlcField: PlcField{Datatype: readWriteModel.AdsDataType_UINT32}}}, []string{"dummy"}, nil, nil)
-		m.sendOverTheWire(ctx, userdata, dummyRequest, result)
-	}()
-	// We wait synchronous for the resolution response before we can continue
-	response := <-result
-	if response.GetErr() != nil {
-		log.Debug().Err(response.GetErr()).Msg("Error during resolve")
-		return DirectPlcField{}, response.GetErr()
-	}
-	if response.GetResponse().GetResponseCode("dummy") != model.PlcResponseCode_OK {
-		return DirectPlcField{}, errors.Errorf("Got a response error %#v", response.GetResponse().GetResponseCode("dummy"))
-	}
-	handle := response.GetResponse().GetValue("dummy").GetUint32()
-	log.Debug().Uint32("handle", handle).Str("symbolicAddress", symbolicField.SymbolicAddress).Msg("Resolved symbolic address")
-	directPlcField := DirectPlcField{
-		IndexGroup:  uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_SYM_VALBYHND),
-		IndexOffset: handle,
-		PlcField:    symbolicField.PlcField,
-	}
-	switch directPlcField.FieldType {
-	case SymbolicAdsField:
-		directPlcField.FieldType = DirectAdsField
-	case SymbolicAdsStringField:
-		directPlcField.FieldType = DirectAdsStringField
-	}
-	m.fieldMapping[symbolicField] = directPlcField
-	return directPlcField, nil
+	/*	if directPlcField, ok := m.fieldMapping[symbolicField]; ok {
+			return directPlcField, nil
+		}
+		m.mappingLock.Lock()
+		defer m.mappingLock.Unlock()
+		// In case a previous one has already
+		if directPlcField, ok := m.fieldMapping[symbolicField]; ok {
+			return directPlcField, nil
+		}
+		userdata := readWriteModel.NewAdsReadWriteRequest(
+			uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_SYM_HNDBYNAME),
+			0,
+			4,
+			nil,
+			[]byte(symbolicField.SymbolicAddress+"\000"),
+			m.targetAmsNetId,
+			m.targetAmsPort,
+			m.sourceAmsNetId,
+			m.sourceAmsPort,
+			0,
+			m.getInvokeId())
+		result := make(chan model.PlcReadRequestResult)
+		go func() {
+			dummyRequest := plc4goModel.NewDefaultPlcReadRequest(map[string]model.PlcField{"dummy": DirectPlcField{PlcField: PlcField{Datatype: readWriteModel.AdsDataType_UINT32}}}, []string{"dummy"}, nil, nil)
+			m.sendOverTheWire(ctx, userdata, dummyRequest, result)
+		}()
+		// We wait synchronous for the resolution response before we can continue
+		response := <-result
+		if response.GetErr() != nil {
+			log.Debug().Err(response.GetErr()).Msg("Error during resolve")
+			return DirectPlcField{}, response.GetErr()
+		}
+		if response.GetResponse().GetResponseCode("dummy") != model.PlcResponseCode_OK {
+			return DirectPlcField{}, errors.Errorf("Got a response error %#v", response.GetResponse().GetResponseCode("dummy"))
+		}
+		handle := response.GetResponse().GetValue("dummy").GetUint32()
+		log.Debug().Uint32("handle", handle).Str("symbolicAddress", symbolicField.SymbolicAddress).Msg("Resolved symbolic address")
+		directPlcField := DirectPlcField{
+			IndexGroup:  uint32(readWriteModel.ReservedIndexGroups_ADSIGRP_SYM_VALBYHND),
+			IndexOffset: handle,
+			PlcField:    symbolicField.PlcField,
+		}
+		switch directPlcField.FieldType {
+		case SymbolicAdsField:
+			directPlcField.FieldType = DirectAdsField
+		case SymbolicAdsStringField:
+			directPlcField.FieldType = DirectAdsStringField
+		}
+		m.fieldMapping[symbolicField] = directPlcField
+		return directPlcField, nil*/
+	return DirectPlcField{}, nil
 }
 
 func (m *Reader) ToPlc4xReadResponse(amsTcpPaket readWriteModel.AmsTCPPacket, readRequest model.PlcReadRequest) (model.PlcReadResponse, error) {
-	var rb utils.ReadBuffer
+	/*var rb utils.ReadBuffer
 	responseCodes := map[string]model.PlcResponseCode{}
 	switch data := amsTcpPaket.GetUserdata().(type) {
 	case readWriteModel.AdsReadResponse:
@@ -410,7 +410,8 @@ func (m *Reader) ToPlc4xReadResponse(amsTcpPaket readWriteModel.AmsTCPPacket, re
 
 	// Return the response
 	log.Trace().Msg("Returning the response")
-	return plc4goModel.NewDefaultPlcReadResponse(readRequest, responseCodes, plcValues), nil
+	return plc4goModel.NewDefaultPlcReadResponse(readRequest, responseCodes, plcValues), nil*/
+	return nil, nil
 }
 
 func (m *Reader) getInvokeId() uint32 {
diff --git a/plc4go/internal/ads/Writer.go b/plc4go/internal/ads/Writer.go
index ca76a62fd..73377df7f 100644
--- a/plc4go/internal/ads/Writer.go
+++ b/plc4go/internal/ads/Writer.go
@@ -21,15 +21,15 @@ package ads
 
 import (
 	"context"
+	"math"
+	"sync/atomic"
+
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
 	readWriteModel "github.com/apache/plc4x/plc4go/protocols/ads/readwrite/model"
 	"github.com/apache/plc4x/plc4go/spi"
 	plc4goModel "github.com/apache/plc4x/plc4go/spi/model"
-	"github.com/apache/plc4x/plc4go/spi/utils"
 	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
-	"math"
-	"sync/atomic"
 )
 
 type Writer struct {
@@ -55,133 +55,135 @@ func NewWriter(messageCodec spi.MessageCodec, targetAmsNetId readWriteModel.AmsN
 }
 
 func (m *Writer) Write(ctx context.Context, writeRequest model.PlcWriteRequest) <-chan model.PlcWriteRequestResult {
-	// TODO: handle context
-	result := make(chan model.PlcWriteRequestResult)
-	go func() {
-		// If we are requesting only one field, use a
-		if len(writeRequest.GetFieldNames()) != 1 {
-			result <- &plc4goModel.DefaultPlcWriteRequestResult{
-				Request:  writeRequest,
-				Response: nil,
-				Err:      errors.New("ads only supports single-item requests"),
-			}
-			return
-		}
-		fieldName := writeRequest.GetFieldNames()[0]
-
-		// Get the ads field instance from the request
-		field := writeRequest.GetField(fieldName)
-		if needsResolving(field) {
-			adsField, err := castToSymbolicPlcFieldFromPlcField(field)
-			if err != nil {
+	/*	// TODO: handle context
+		result := make(chan model.PlcWriteRequestResult)
+		go func() {
+			// If we are requesting only one field, use a
+			if len(writeRequest.GetFieldNames()) != 1 {
 				result <- &plc4goModel.DefaultPlcWriteRequestResult{
 					Request:  writeRequest,
 					Response: nil,
-					Err:      errors.Wrap(err, "invalid field item type"),
+					Err:      errors.New("ads only supports single-item requests"),
 				}
-				log.Debug().Msgf("Invalid field item type %T", field)
 				return
 			}
-			field, err = m.reader.resolveField(ctx, adsField)
+			fieldName := writeRequest.GetFieldNames()[0]
+
+			// Get the ads field instance from the request
+			field := writeRequest.GetField(fieldName)
+			if needsResolving(field) {
+				adsField, err := castToSymbolicPlcFieldFromPlcField(field)
+				if err != nil {
+					result <- &plc4goModel.DefaultPlcWriteRequestResult{
+						Request:  writeRequest,
+						Response: nil,
+						Err:      errors.Wrap(err, "invalid field item type"),
+					}
+					log.Debug().Msgf("Invalid field item type %T", field)
+					return
+				}
+				field, err = m.reader.resolveField(ctx, adsField)
+				if err != nil {
+					result <- &plc4goModel.DefaultPlcWriteRequestResult{
+						Request:  writeRequest,
+						Response: nil,
+						Err:      errors.Wrap(err, "invalid field item type"),
+					}
+					log.Debug().Msgf("Invalid field item type %T", field)
+					return
+				}
+			}
+			adsField, err := castToDirectAdsFieldFromPlcField(field)
 			if err != nil {
 				result <- &plc4goModel.DefaultPlcWriteRequestResult{
 					Request:  writeRequest,
 					Response: nil,
 					Err:      errors.Wrap(err, "invalid field item type"),
 				}
-				log.Debug().Msgf("Invalid field item type %T", field)
 				return
 			}
-		}
-		adsField, err := castToDirectAdsFieldFromPlcField(field)
-		if err != nil {
-			result <- &plc4goModel.DefaultPlcWriteRequestResult{
-				Request:  writeRequest,
-				Response: nil,
-				Err:      errors.Wrap(err, "invalid field item type"),
-			}
-			return
-		}
-
-		// Get the value from the request and serialize it to a byte array
-		value := writeRequest.GetValue(fieldName)
-		io := utils.NewLittleEndianWriteBufferByteBased()
-		if err := readWriteModel.DataItemSerialize(io, value, adsField.Datatype.PlcValueType(), adsField.StringLength); err != nil {
-			result <- &plc4goModel.DefaultPlcWriteRequestResult{
-				Request:  writeRequest,
-				Response: nil,
-				Err:      errors.Wrap(err, "error serializing value"),
-			}
-			return
-		}
-		/*data := io.GetBytes()
-
-		userdata := readWriteModel.NewAmsPacket(
-			m.targetAmsNetId,
-			m.targetAmsPort,
-			m.sourceAmsNetId,
-			m.sourceAmsPort,
-			readWriteModel.CommandId_ADS_READ,
-			readWriteModel.NewState(false, false, false, false, false, true, false, false, false),
-			0,
-			0,
-			nil,
-		)*/
-		switch adsField.FieldType {
-		case DirectAdsStringField:
-			//userdata.Data = readWriteModel.NewAdsWriteRequest(adsField.IndexGroup, adsField.IndexOffset, data)
-			panic("implement me")
-		case DirectAdsField:
-			panic("implement me")
-		case SymbolicAdsStringField, SymbolicAdsField:
-			panic("we should never reach this point as symbols are resolved before")
-		default:
-			result <- &plc4goModel.DefaultPlcWriteRequestResult{
-				Request:  writeRequest,
-				Response: nil,
-				Err:      errors.New("unsupported field type"),
-			}
-			return
-		}
-
-		// Calculate a new unit identifier
-		/*userdata.InvokeId = m.getInvokeId()
-
-		// Assemble the finished amsTcpPaket
-		log.Trace().Msg("Assemble amsTcpPaket")
-		amsTcpPaket := readWriteModel.NewAmsTCPPacket(userdata)
-
-		// Send the TCP Paket over the wire
-		err = m.messageCodec.SendRequest(ctx, amsTcpPaket, func(message spi.Message) bool {
-			paket := readWriteModel.CastAmsTCPPacket(message)
-			return paket.GetUserdata().GetInvokeId() == transactionIdentifier
-		}, func(message spi.Message) error {
-			// Convert the response into an responseAmsTcpPaket
-			responseAmsTcpPaket := readWriteModel.CastAmsTCPPacket(message)
-			// Convert the ads response into a PLC4X response
-			readResponse, err := m.ToPlc4xWriteResponse(amsTcpPaket, responseAmsTcpPaket, writeRequest)
 
-			if err != nil {
+			// Get the value from the request and serialize it to a byte array
+			value := writeRequest.GetValue(fieldName)
+			io := utils.NewLittleEndianWriteBufferByteBased()
+			if err := readWriteModel.DataItemSerialize(io, value, adsField.Datatype.PlcValueType(), adsField.StringLength); err != nil {
 				result <- &plc4goModel.DefaultPlcWriteRequestResult{
-					Request: writeRequest,
-					Err:     errors.Wrap(err, "Error decoding response"),
+					Request:  writeRequest,
+					Response: nil,
+					Err:      errors.Wrap(err, "error serializing value"),
 				}
-			} else {
+				return
+			}
+			/data := io.GetBytes()
+
+			userdata := readWriteModel.NewAmsPacket(
+				m.targetAmsNetId,
+				m.targetAmsPort,
+				m.sourceAmsNetId,
+				m.sourceAmsPort,
+				readWriteModel.CommandId_ADS_READ,
+				readWriteModel.NewState(false, false, false, false, false, true, false, false, false),
+				0,
+				0,
+				nil,
+			)/
+			switch adsField.FieldType {
+			case DirectAdsStringField:
+				//userdata.Data = readWriteModel.NewAdsWriteRequest(adsField.IndexGroup, adsField.IndexOffset, data)
+				panic("implement me")
+			case DirectAdsField:
+				panic("implement me")
+			case SymbolicAdsStringField, SymbolicAdsField:
+				panic("we should never reach this point as symbols are resolved before")
+			default:
 				result <- &plc4goModel.DefaultPlcWriteRequestResult{
 					Request:  writeRequest,
-					Response: readResponse,
+					Response: nil,
+					Err:      errors.New("unsupported field type"),
 				}
+				return
 			}
-			return nil
-		}, func(err error) error {
-			result <- &plc4goModel.DefaultPlcWriteRequestResult{
-				Request: writeRequest,
-				Err:     errors.New("got timeout while waiting for response"),
-			}
-			return nil
-		}, time.Second*1)*/
-	}()
-	return result
+
+			// Calculate a new unit identifier
+			/userdata.InvokeId = m.getInvokeId()
+
+			// Assemble the finished amsTcpPaket
+			log.Trace().Msg("Assemble amsTcpPaket")
+			amsTcpPaket := readWriteModel.NewAmsTCPPacket(userdata)
+
+			// Send the TCP Paket over the wire
+			err = m.messageCodec.SendRequest(ctx, amsTcpPaket, func(message spi.Message) bool {
+				paket := readWriteModel.CastAmsTCPPacket(message)
+				return paket.GetUserdata().GetInvokeId() == transactionIdentifier
+			}, func(message spi.Message) error {
+				// Convert the response into an responseAmsTcpPaket
+				responseAmsTcpPaket := readWriteModel.CastAmsTCPPacket(message)
+				// Convert the ads response into a PLC4X response
+				readResponse, err := m.ToPlc4xWriteResponse(amsTcpPaket, responseAmsTcpPaket, writeRequest)
+
+				if err != nil {
+					result <- &plc4goModel.DefaultPlcWriteRequestResult{
+						Request: writeRequest,
+						Err:     errors.Wrap(err, "Error decoding response"),
+					}
+				} else {
+					result <- &plc4goModel.DefaultPlcWriteRequestResult{
+						Request:  writeRequest,
+						Response: readResponse,
+					}
+				}
+				return nil
+			}, func(err error) error {
+				result <- &plc4goModel.DefaultPlcWriteRequestResult{
+					Request: writeRequest,
+					Err:     errors.New("got timeout while waiting for response"),
+				}
+				return nil
+			}, time.Second*1)/
+		}()
+		return result
+	*/
+	return nil
 }
 
 func (m *Writer) ToPlc4xWriteResponse(requestTcpPaket readWriteModel.AmsTCPPacket, responseTcpPaket readWriteModel.AmsTCPPacket, writeRequest model.PlcWriteRequest) (model.PlcWriteResponse, error) {
diff --git a/plc4go/internal/ads/fieldtype_string.go b/plc4go/internal/ads/fieldtype_string.go
deleted file mode 100644
index 4b0da398a..000000000
--- a/plc4go/internal/ads/fieldtype_string.go
+++ /dev/null
@@ -1,54 +0,0 @@
-// Licensed to 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. Apache Software Foundation (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.
-
-// Code generated by "stringer -type FieldType"; DO NOT EDIT.
-
-package ads
-
-import "strconv"
-
-func _() {
-	// An "invalid array index" compiler error signifies that the constant values have changed.
-	// Re-run the stringer command to generate them again.
-	var x [1]struct{}
-	_ = x[DirectAdsStringField-0]
-	_ = x[DirectAdsField-1]
-	_ = x[SymbolicAdsStringField-3]
-	_ = x[SymbolicAdsField-4]
-}
-
-const (
-	_FieldType_name_0 = "DirectAdsStringFieldDirectAdsField"
-	_FieldType_name_1 = "SymbolicAdsStringFieldSymbolicAdsField"
-)
-
-var (
-	_FieldType_index_0 = [...]uint8{0, 20, 34}
-	_FieldType_index_1 = [...]uint8{0, 22, 38}
-)
-
-func (i FieldType) String() string {
-	switch {
-	case i <= 1:
-		return _FieldType_name_0[_FieldType_index_0[i]:_FieldType_index_0[i+1]]
-	case 3 <= i && i <= 4:
-		i -= 3
-		return _FieldType_name_1[_FieldType_index_1[i]:_FieldType_index_1[i+1]]
-	default:
-		return "FieldType(" + strconv.FormatInt(int64(i), 10) + ")"
-	}
-}
diff --git a/plc4go/internal/cbus/fieldtype_string.go b/plc4go/internal/cbus/fieldtype_string.go
index 8ce7ef761..7da7ae9fc 100644
--- a/plc4go/internal/cbus/fieldtype_string.go
+++ b/plc4go/internal/cbus/fieldtype_string.go
@@ -1,20 +1,3 @@
-// Licensed to 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. Apache Software Foundation (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.
-
 // Code generated by "stringer -type FieldType"; DO NOT EDIT.
 
 package cbus
diff --git a/plc4go/internal/knxnetip/ConnectionHelper.go b/plc4go/internal/knxnetip/ConnectionHelper.go
index 12d38c9fc..1b80a8d9c 100644
--- a/plc4go/internal/knxnetip/ConnectionHelper.go
+++ b/plc4go/internal/knxnetip/ConnectionHelper.go
@@ -200,7 +200,7 @@ func (m *Connection) getLocalAddress() (*net.UDPAddr, error) {
 	}
 
 	// Prepare a SearchReq
-	udpTransportInstance, ok := transportInstanceExposer.GetTransportInstance().(*udp.TransportInstance)
+	udpTransportInstance, ok := transportInstanceExposer.GetTransportInstance().(*udp.DirectTransportInstance)
 	if !ok {
 		return nil, errors.New("used transport, is not a UdpTransportInstance")
 	}
diff --git a/plc4go/internal/knxnetip/Discoverer.go b/plc4go/internal/knxnetip/Discoverer.go
index f57afc4f5..f0e728943 100644
--- a/plc4go/internal/knxnetip/Discoverer.go
+++ b/plc4go/internal/knxnetip/Discoverer.go
@@ -23,12 +23,13 @@ import (
 	"bytes"
 	"context"
 	"fmt"
-	"github.com/apache/plc4x/plc4go/spi/options"
-	"github.com/pkg/errors"
 	"net"
 	"net/url"
 	"time"
 
+	"github.com/apache/plc4x/plc4go/spi/options"
+	"github.com/pkg/errors"
+
 	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
 	driverModel "github.com/apache/plc4x/plc4go/protocols/knxnetip/readwrite/model"
 	"github.com/apache/plc4x/plc4go/spi"
@@ -78,7 +79,7 @@ func (d *Discoverer) Discover(ctx context.Context, callback func(event apiModel.
 		interfaces = allInterfaces
 	}
 
-	var tranportInstances []transports.TransportInstance
+	var transportInstances []transports.TransportInstance
 	// Iterate over all network devices of this system.
 	for _, interf := range interfaces {
 		addrs, err := interf.Addrs()
@@ -116,16 +117,16 @@ func (d *Discoverer) Discover(ctx context.Context, callback func(event apiModel.
 					continue
 				}
 
-				tranportInstances = append(tranportInstances, transportInstance)
+				transportInstances = append(transportInstances, transportInstance)
 			}
 		}
 	}
 
-	if len(tranportInstances) <= 0 {
+	if len(transportInstances) <= 0 {
 		return nil
 	}
 
-	for _, transportInstance := range tranportInstances {
+	for _, transportInstance := range transportInstances {
 		// Create a codec for sending and receiving messages.
 		codec := NewMessageCodec(transportInstance, nil)
 		// Explicitly start the worker
@@ -133,8 +134,8 @@ func (d *Discoverer) Discover(ctx context.Context, callback func(event apiModel.
 			return errors.Wrap(err, "Error connecting")
 		}
 
-		// Cast to the UDP transport instance so we can access information on the local port.
-		udpTransportInstance, ok := transportInstance.(*udp.TransportInstance)
+		// Cast to the UDP transport instance, so we can access information on the local port.
+		udpTransportInstance, ok := transportInstance.(*udp.DirectTransportInstance)
 		if !ok {
 			return errors.New("couldn't cast transport instance to UDP transport instance")
 		}
diff --git a/plc4go/internal/modbus/fieldtype_string.go b/plc4go/internal/modbus/fieldtype_string.go
index ec3ec5099..4ef728bd5 100644
--- a/plc4go/internal/modbus/fieldtype_string.go
+++ b/plc4go/internal/modbus/fieldtype_string.go
@@ -1,20 +1,3 @@
-// Licensed to 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. Apache Software Foundation (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.
-
 // Code generated by "stringer -type FieldType"; DO NOT EDIT.
 
 package modbus
diff --git a/plc4go/internal/s7/fieldtype_string.go b/plc4go/internal/s7/fieldtype_string.go
index 3045fc100..02ae83feb 100644
--- a/plc4go/internal/s7/fieldtype_string.go
+++ b/plc4go/internal/s7/fieldtype_string.go
@@ -1,20 +1,3 @@
-// Licensed to 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. Apache Software Foundation (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.
-
 // Code generated by "stringer -type FieldType"; DO NOT EDIT.
 
 package s7
diff --git a/plc4go/pkg/api/model/plc_field.go b/plc4go/pkg/api/model/plc_field.go
index 91b8caed3..c84f8be06 100644
--- a/plc4go/pkg/api/model/plc_field.go
+++ b/plc4go/pkg/api/model/plc_field.go
@@ -20,7 +20,6 @@
 package model
 
 type PlcField interface {
-	GetAddressString() string
-	GetTypeName() string
-	GetQuantity() uint16
+	GetPlcDataType() string
+	GetNumberOfElements() int
 }
diff --git a/plc4go/pom.xml b/plc4go/pom.xml
index 619bd009a..bfcca79c9 100644
--- a/plc4go/pom.xml
+++ b/plc4go/pom.xml
@@ -197,6 +197,19 @@
               <outputDir>${project.basedir}/protocols</outputDir>
             </configuration>
           </execution>
+          <execution>
+            <id>generate-driver-ads-discovery</id>
+            <phase>generate-sources</phase>
+            <goals>
+              <goal>generate-driver</goal>
+            </goals>
+            <configuration>
+              <protocolName>ads.discovery</protocolName>
+              <languageName>go</languageName>
+              <outputFlavor>read-write</outputFlavor>
+              <outputDir>${project.basedir}/protocols</outputDir>
+            </configuration>
+          </execution>
           <execution>
             <id>generate-driver-bacnetip</id>
             <phase>generate-sources</phase>
diff --git a/plc4go/pkg/api/model/plc_field.go b/plc4go/protocols/ads/discovery/readwrite/ParserHelper.go
similarity index 51%
copy from plc4go/pkg/api/model/plc_field.go
copy to plc4go/protocols/ads/discovery/readwrite/ParserHelper.go
index 91b8caed3..61ca7452b 100644
--- a/plc4go/pkg/api/model/plc_field.go
+++ b/plc4go/protocols/ads/discovery/readwrite/ParserHelper.go
@@ -17,10 +17,31 @@
  * under the License.
  */
 
-package model
+package readwrite
 
-type PlcField interface {
-	GetAddressString() string
-	GetTypeName() string
-	GetQuantity() uint16
+import (
+	"github.com/apache/plc4x/plc4go/protocols/ads.discovery/readwrite/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+type Ads.discoveryParserHelper struct {
+}
+
+func (m Ads.discoveryParserHelper) Parse(typeName string, arguments []string, io utils.ReadBuffer) (interface{}, error) {
+	switch typeName {
+	case "AdsDiscovery":
+		return model.AdsDiscoveryParse(io)
+	case "AdsDiscoveryBlock":
+		return model.AdsDiscoveryBlockParse(io)
+	case "AdsDiscoveryConstants":
+		return model.AdsDiscoveryConstantsParse(io)
+	case "AmsNetId":
+		return model.AmsNetIdParse(io)
+	case "AmsString":
+		return model.AmsStringParse(io)
+	}
+	return nil, errors.Errorf("Unsupported type %s", typeName)
 }
diff --git a/plc4go/protocols/ads/discovery/readwrite/XmlParserHelper.go b/plc4go/protocols/ads/discovery/readwrite/XmlParserHelper.go
new file mode 100644
index 000000000..e20787878
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/XmlParserHelper.go
@@ -0,0 +1,57 @@
+/*
+ * 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 readwrite
+
+import (
+	"github.com/apache/plc4x/plc4go/protocols/ads.discovery/readwrite/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+    "strings"
+    "strconv"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+type Ads.discoveryXmlParserHelper struct {
+}
+
+// Temporary imports to silent compiler warnings (TODO: migrate from static to emission based imports)
+func init() {
+	_ = strconv.ParseUint
+	_ = strconv.ParseInt
+	_ = strings.Join
+	_ = utils.Dump
+}
+
+func (m Ads.discoveryXmlParserHelper) Parse(typeName string, xmlString string, parserArguments ...string) (interface{}, error) {
+    switch typeName {
+        case "AdsDiscovery":
+			return model.AdsDiscoveryParse(utils.NewXmlReadBuffer(strings.NewReader(xmlString)))
+        case "AdsDiscoveryBlock":
+			return model.AdsDiscoveryBlockParse(utils.NewXmlReadBuffer(strings.NewReader(xmlString)))
+        case "AdsDiscoveryConstants":
+			return model.AdsDiscoveryConstantsParse(utils.NewXmlReadBuffer(strings.NewReader(xmlString)))
+        case "AmsNetId":
+			return model.AmsNetIdParse(utils.NewXmlReadBuffer(strings.NewReader(xmlString)))
+        case "AmsString":
+			return model.AmsStringParse(utils.NewXmlReadBuffer(strings.NewReader(xmlString)))
+    }
+    return nil, errors.Errorf("Unsupported type %s", typeName)
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscovery.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscovery.go
new file mode 100644
index 000000000..d6cb1dd4b
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscovery.go
@@ -0,0 +1,372 @@
+/*
+ * 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 model
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Constant values.
+const AdsDiscovery_HEADER uint32 = 0x71146603
+
+// AdsDiscovery is the corresponding interface of AdsDiscovery
+type AdsDiscovery interface {
+	utils.LengthAware
+	utils.Serializable
+	// GetRequestId returns RequestId (property field)
+	GetRequestId() uint32
+	// GetOperation returns Operation (property field)
+	GetOperation() Operation
+	// GetAmsNetId returns AmsNetId (property field)
+	GetAmsNetId() AmsNetId
+	// GetPortNumber returns PortNumber (property field)
+	GetPortNumber() AdsPortNumbers
+	// GetBlocks returns Blocks (property field)
+	GetBlocks() []AdsDiscoveryBlock
+}
+
+// AdsDiscoveryExactly can be used when we want exactly this type and not a type which fulfills AdsDiscovery.
+// This is useful for switch cases.
+type AdsDiscoveryExactly interface {
+	AdsDiscovery
+	isAdsDiscovery() bool
+}
+
+// _AdsDiscovery is the data-structure of this message
+type _AdsDiscovery struct {
+	RequestId  uint32
+	Operation  Operation
+	AmsNetId   AmsNetId
+	PortNumber AdsPortNumbers
+	Blocks     []AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscovery) GetRequestId() uint32 {
+	return m.RequestId
+}
+
+func (m *_AdsDiscovery) GetOperation() Operation {
+	return m.Operation
+}
+
+func (m *_AdsDiscovery) GetAmsNetId() AmsNetId {
+	return m.AmsNetId
+}
+
+func (m *_AdsDiscovery) GetPortNumber() AdsPortNumbers {
+	return m.PortNumber
+}
+
+func (m *_AdsDiscovery) GetBlocks() []AdsDiscoveryBlock {
+	return m.Blocks
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for const fields.
+///////////////////////
+
+func (m *_AdsDiscovery) GetHeader() uint32 {
+	return AdsDiscovery_HEADER
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscovery factory function for _AdsDiscovery
+func NewAdsDiscovery(requestId uint32, operation Operation, amsNetId AmsNetId, portNumber AdsPortNumbers, blocks []AdsDiscoveryBlock) *_AdsDiscovery {
+	return &_AdsDiscovery{RequestId: requestId, Operation: operation, AmsNetId: amsNetId, PortNumber: portNumber, Blocks: blocks}
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscovery(structType interface{}) AdsDiscovery {
+	if casted, ok := structType.(AdsDiscovery); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscovery); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscovery) GetTypeName() string {
+	return "AdsDiscovery"
+}
+
+func (m *_AdsDiscovery) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscovery) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(0)
+
+	// Const Field (header)
+	lengthInBits += 32
+
+	// Simple field (requestId)
+	lengthInBits += 32
+
+	// Simple field (operation)
+	lengthInBits += 32
+
+	// Simple field (amsNetId)
+	lengthInBits += m.AmsNetId.GetLengthInBits()
+
+	// Simple field (portNumber)
+	lengthInBits += 16
+
+	// Implicit Field (numBlocks)
+	lengthInBits += 32
+
+	// Array field
+	if len(m.Blocks) > 0 {
+		for i, element := range m.Blocks {
+			last := i == len(m.Blocks)-1
+			lengthInBits += element.(interface{ GetLengthInBitsConditional(bool) uint16 }).GetLengthInBitsConditional(last)
+		}
+	}
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscovery) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryParse(readBuffer utils.ReadBuffer) (AdsDiscovery, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscovery"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscovery")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Const Field (header)
+	header, _headerErr := readBuffer.ReadUint32("header", 32)
+	if _headerErr != nil {
+		return nil, errors.Wrap(_headerErr, "Error parsing 'header' field of AdsDiscovery")
+	}
+	if header != AdsDiscovery_HEADER {
+		return nil, errors.New("Expected constant value " + fmt.Sprintf("%d", AdsDiscovery_HEADER) + " but got " + fmt.Sprintf("%d", header))
+	}
+
+	// Simple Field (requestId)
+	_requestId, _requestIdErr := readBuffer.ReadUint32("requestId", 32)
+	if _requestIdErr != nil {
+		return nil, errors.Wrap(_requestIdErr, "Error parsing 'requestId' field of AdsDiscovery")
+	}
+	requestId := _requestId
+
+	// Simple Field (operation)
+	if pullErr := readBuffer.PullContext("operation"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for operation")
+	}
+	_operation, _operationErr := OperationParse(readBuffer)
+	if _operationErr != nil {
+		return nil, errors.Wrap(_operationErr, "Error parsing 'operation' field of AdsDiscovery")
+	}
+	operation := _operation
+	if closeErr := readBuffer.CloseContext("operation"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for operation")
+	}
+
+	// Simple Field (amsNetId)
+	if pullErr := readBuffer.PullContext("amsNetId"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for amsNetId")
+	}
+	_amsNetId, _amsNetIdErr := AmsNetIdParse(readBuffer)
+	if _amsNetIdErr != nil {
+		return nil, errors.Wrap(_amsNetIdErr, "Error parsing 'amsNetId' field of AdsDiscovery")
+	}
+	amsNetId := _amsNetId.(AmsNetId)
+	if closeErr := readBuffer.CloseContext("amsNetId"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for amsNetId")
+	}
+
+	// Simple Field (portNumber)
+	if pullErr := readBuffer.PullContext("portNumber"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for portNumber")
+	}
+	_portNumber, _portNumberErr := AdsPortNumbersParse(readBuffer)
+	if _portNumberErr != nil {
+		return nil, errors.Wrap(_portNumberErr, "Error parsing 'portNumber' field of AdsDiscovery")
+	}
+	portNumber := _portNumber
+	if closeErr := readBuffer.CloseContext("portNumber"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for portNumber")
+	}
+
+	// Implicit Field (numBlocks) (Used for parsing, but its value is not stored as it's implicitly given by the objects content)
+	numBlocks, _numBlocksErr := readBuffer.ReadUint32("numBlocks", 32)
+	_ = numBlocks
+	if _numBlocksErr != nil {
+		return nil, errors.Wrap(_numBlocksErr, "Error parsing 'numBlocks' field of AdsDiscovery")
+	}
+
+	// Array field (blocks)
+	if pullErr := readBuffer.PullContext("blocks", utils.WithRenderAsList(true)); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for blocks")
+	}
+	// Count array
+	blocks := make([]AdsDiscoveryBlock, numBlocks)
+	// This happens when the size is set conditional to 0
+	if len(blocks) == 0 {
+		blocks = nil
+	}
+	{
+		for curItem := uint16(0); curItem < uint16(numBlocks); curItem++ {
+			_item, _err := AdsDiscoveryBlockParse(readBuffer)
+			if _err != nil {
+				return nil, errors.Wrap(_err, "Error parsing 'blocks' field of AdsDiscovery")
+			}
+			blocks[curItem] = _item.(AdsDiscoveryBlock)
+		}
+	}
+	if closeErr := readBuffer.CloseContext("blocks", utils.WithRenderAsList(true)); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for blocks")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscovery"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscovery")
+	}
+
+	// Create the instance
+	return &_AdsDiscovery{
+		RequestId:  requestId,
+		Operation:  operation,
+		AmsNetId:   amsNetId,
+		PortNumber: portNumber,
+		Blocks:     blocks,
+	}, nil
+}
+
+func (m *_AdsDiscovery) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	if pushErr := writeBuffer.PushContext("AdsDiscovery"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for AdsDiscovery")
+	}
+
+	// Const Field (header)
+	_headerErr := writeBuffer.WriteUint32("header", 32, 0x71146603)
+	if _headerErr != nil {
+		return errors.Wrap(_headerErr, "Error serializing 'header' field")
+	}
+
+	// Simple Field (requestId)
+	requestId := uint32(m.GetRequestId())
+	_requestIdErr := writeBuffer.WriteUint32("requestId", 32, (requestId))
+	if _requestIdErr != nil {
+		return errors.Wrap(_requestIdErr, "Error serializing 'requestId' field")
+	}
+
+	// Simple Field (operation)
+	if pushErr := writeBuffer.PushContext("operation"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for operation")
+	}
+	_operationErr := writeBuffer.WriteSerializable(m.GetOperation())
+	if popErr := writeBuffer.PopContext("operation"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for operation")
+	}
+	if _operationErr != nil {
+		return errors.Wrap(_operationErr, "Error serializing 'operation' field")
+	}
+
+	// Simple Field (amsNetId)
+	if pushErr := writeBuffer.PushContext("amsNetId"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for amsNetId")
+	}
+	_amsNetIdErr := writeBuffer.WriteSerializable(m.GetAmsNetId())
+	if popErr := writeBuffer.PopContext("amsNetId"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for amsNetId")
+	}
+	if _amsNetIdErr != nil {
+		return errors.Wrap(_amsNetIdErr, "Error serializing 'amsNetId' field")
+	}
+
+	// Simple Field (portNumber)
+	if pushErr := writeBuffer.PushContext("portNumber"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for portNumber")
+	}
+	_portNumberErr := writeBuffer.WriteSerializable(m.GetPortNumber())
+	if popErr := writeBuffer.PopContext("portNumber"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for portNumber")
+	}
+	if _portNumberErr != nil {
+		return errors.Wrap(_portNumberErr, "Error serializing 'portNumber' field")
+	}
+
+	// Implicit Field (numBlocks) (Used for parsing, but it's value is not stored as it's implicitly given by the objects content)
+	numBlocks := uint32(uint32(len(m.GetBlocks())))
+	_numBlocksErr := writeBuffer.WriteUint32("numBlocks", 32, (numBlocks))
+	if _numBlocksErr != nil {
+		return errors.Wrap(_numBlocksErr, "Error serializing 'numBlocks' field")
+	}
+
+	// Array Field (blocks)
+	if pushErr := writeBuffer.PushContext("blocks", utils.WithRenderAsList(true)); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for blocks")
+	}
+	for _, _element := range m.GetBlocks() {
+		_elementErr := writeBuffer.WriteSerializable(_element)
+		if _elementErr != nil {
+			return errors.Wrap(_elementErr, "Error serializing 'blocks' field")
+		}
+	}
+	if popErr := writeBuffer.PopContext("blocks", utils.WithRenderAsList(true)); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for blocks")
+	}
+
+	if popErr := writeBuffer.PopContext("AdsDiscovery"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for AdsDiscovery")
+	}
+	return nil
+}
+
+func (m *_AdsDiscovery) isAdsDiscovery() bool {
+	return true
+}
+
+func (m *_AdsDiscovery) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlock.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlock.go
new file mode 100644
index 000000000..9600a3263
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlock.go
@@ -0,0 +1,217 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlock is the corresponding interface of AdsDiscoveryBlock
+type AdsDiscoveryBlock interface {
+	utils.LengthAware
+	utils.Serializable
+	// GetBlockType returns BlockType (discriminator field)
+	GetBlockType() AdsDiscoveryBlockType
+}
+
+// AdsDiscoveryBlockExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlock.
+// This is useful for switch cases.
+type AdsDiscoveryBlockExactly interface {
+	AdsDiscoveryBlock
+	isAdsDiscoveryBlock() bool
+}
+
+// _AdsDiscoveryBlock is the data-structure of this message
+type _AdsDiscoveryBlock struct {
+	_AdsDiscoveryBlockChildRequirements
+}
+
+type _AdsDiscoveryBlockChildRequirements interface {
+	utils.Serializable
+	GetLengthInBits() uint16
+	GetLengthInBitsConditional(lastItem bool) uint16
+	GetBlockType() AdsDiscoveryBlockType
+}
+
+type AdsDiscoveryBlockParent interface {
+	SerializeParent(writeBuffer utils.WriteBuffer, child AdsDiscoveryBlock, serializeChildFunction func() error) error
+	GetTypeName() string
+}
+
+type AdsDiscoveryBlockChild interface {
+	utils.Serializable
+	InitializeParent(parent AdsDiscoveryBlock)
+	GetParent() *AdsDiscoveryBlock
+
+	GetTypeName() string
+	AdsDiscoveryBlock
+}
+
+// NewAdsDiscoveryBlock factory function for _AdsDiscoveryBlock
+func NewAdsDiscoveryBlock() *_AdsDiscoveryBlock {
+	return &_AdsDiscoveryBlock{}
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlock(structType interface{}) AdsDiscoveryBlock {
+	if casted, ok := structType.(AdsDiscoveryBlock); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlock); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlock) GetTypeName() string {
+	return "AdsDiscoveryBlock"
+}
+
+func (m *_AdsDiscoveryBlock) GetParentLengthInBits() uint16 {
+	lengthInBits := uint16(0)
+	// Discriminator Field (blockType)
+	lengthInBits += 16
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlock) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlock, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlock"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlock")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Discriminator Field (blockType) (Used as input to a switch field)
+	if pullErr := readBuffer.PullContext("blockType"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for blockType")
+	}
+	blockType_temp, _blockTypeErr := AdsDiscoveryBlockTypeParse(readBuffer)
+	var blockType AdsDiscoveryBlockType = blockType_temp
+	if closeErr := readBuffer.CloseContext("blockType"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for blockType")
+	}
+	if _blockTypeErr != nil {
+		return nil, errors.Wrap(_blockTypeErr, "Error parsing 'blockType' field of AdsDiscoveryBlock")
+	}
+
+	// Switch Field (Depending on the discriminator values, passes the instantiation to a sub-type)
+	type AdsDiscoveryBlockChildSerializeRequirement interface {
+		AdsDiscoveryBlock
+		InitializeParent(AdsDiscoveryBlock)
+		GetParent() AdsDiscoveryBlock
+	}
+	var _childTemp interface{}
+	var _child AdsDiscoveryBlockChildSerializeRequirement
+	var typeSwitchError error
+	switch {
+	case blockType == AdsDiscoveryBlockType_STATUS: // AdsDiscoveryBlockStatus
+		_childTemp, typeSwitchError = AdsDiscoveryBlockStatusParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_PASSWORD: // AdsDiscoveryBlockPassword
+		_childTemp, typeSwitchError = AdsDiscoveryBlockPasswordParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_VERSION: // AdsDiscoveryBlockVersion
+		_childTemp, typeSwitchError = AdsDiscoveryBlockVersionParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_OS_DATA: // AdsDiscoveryBlockOsData
+		_childTemp, typeSwitchError = AdsDiscoveryBlockOsDataParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_HOST_NAME: // AdsDiscoveryBlockHostName
+		_childTemp, typeSwitchError = AdsDiscoveryBlockHostNameParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_AMS_NET_ID: // AdsDiscoveryBlockAmsNetId
+		_childTemp, typeSwitchError = AdsDiscoveryBlockAmsNetIdParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_ROUTE_NAME: // AdsDiscoveryBlockRouteName
+		_childTemp, typeSwitchError = AdsDiscoveryBlockRouteNameParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_USER_NAME: // AdsDiscoveryBlockUserName
+		_childTemp, typeSwitchError = AdsDiscoveryBlockUserNameParse(readBuffer)
+	case blockType == AdsDiscoveryBlockType_FINGERPRINT: // AdsDiscoveryBlockFingerprint
+		_childTemp, typeSwitchError = AdsDiscoveryBlockFingerprintParse(readBuffer)
+	default:
+		typeSwitchError = errors.Errorf("Unmapped type for parameters [blockType=%v]", blockType)
+	}
+	if typeSwitchError != nil {
+		return nil, errors.Wrap(typeSwitchError, "Error parsing sub-type for type-switch of AdsDiscoveryBlock")
+	}
+	_child = _childTemp.(AdsDiscoveryBlockChildSerializeRequirement)
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlock"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlock")
+	}
+
+	// Finish initializing
+	_child.InitializeParent(_child)
+	return _child, nil
+}
+
+func (pm *_AdsDiscoveryBlock) SerializeParent(writeBuffer utils.WriteBuffer, child AdsDiscoveryBlock, serializeChildFunction func() error) error {
+	// We redirect all calls through client as some methods are only implemented there
+	m := child
+	_ = m
+	positionAware := writeBuffer
+	_ = positionAware
+	if pushErr := writeBuffer.PushContext("AdsDiscoveryBlock"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlock")
+	}
+
+	// Discriminator Field (blockType) (Used as input to a switch field)
+	blockType := AdsDiscoveryBlockType(child.GetBlockType())
+	if pushErr := writeBuffer.PushContext("blockType"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for blockType")
+	}
+	_blockTypeErr := writeBuffer.WriteSerializable(blockType)
+	if popErr := writeBuffer.PopContext("blockType"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for blockType")
+	}
+
+	if _blockTypeErr != nil {
+		return errors.Wrap(_blockTypeErr, "Error serializing 'blockType' field")
+	}
+
+	// Switch field (Depending on the discriminator values, passes the serialization to a sub-type)
+	if _typeSwitchErr := serializeChildFunction(); _typeSwitchErr != nil {
+		return errors.Wrap(_typeSwitchErr, "Error serializing sub-type field")
+	}
+
+	if popErr := writeBuffer.PopContext("AdsDiscoveryBlock"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlock")
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlock) isAdsDiscoveryBlock() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlock) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockAmsNetId.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockAmsNetId.go
new file mode 100644
index 000000000..17ae5f7f4
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockAmsNetId.go
@@ -0,0 +1,238 @@
+/*
+ * 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 model
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Constant values.
+const AdsDiscoveryBlockAmsNetId_AMSNETIDLENGTH uint16 = 0x0006
+
+// AdsDiscoveryBlockAmsNetId is the corresponding interface of AdsDiscoveryBlockAmsNetId
+type AdsDiscoveryBlockAmsNetId interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetAmsNetId returns AmsNetId (property field)
+	GetAmsNetId() AmsNetId
+}
+
+// AdsDiscoveryBlockAmsNetIdExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockAmsNetId.
+// This is useful for switch cases.
+type AdsDiscoveryBlockAmsNetIdExactly interface {
+	AdsDiscoveryBlockAmsNetId
+	isAdsDiscoveryBlockAmsNetId() bool
+}
+
+// _AdsDiscoveryBlockAmsNetId is the data-structure of this message
+type _AdsDiscoveryBlockAmsNetId struct {
+	*_AdsDiscoveryBlock
+	AmsNetId AmsNetId
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_AMS_NET_ID
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockAmsNetId) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetAmsNetId() AmsNetId {
+	return m.AmsNetId
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for const fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetAmsNetIdLength() uint16 {
+	return AdsDiscoveryBlockAmsNetId_AMSNETIDLENGTH
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockAmsNetId factory function for _AdsDiscoveryBlockAmsNetId
+func NewAdsDiscoveryBlockAmsNetId(amsNetId AmsNetId) *_AdsDiscoveryBlockAmsNetId {
+	_result := &_AdsDiscoveryBlockAmsNetId{
+		AmsNetId:           amsNetId,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockAmsNetId(structType interface{}) AdsDiscoveryBlockAmsNetId {
+	if casted, ok := structType.(AdsDiscoveryBlockAmsNetId); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockAmsNetId); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetTypeName() string {
+	return "AdsDiscoveryBlockAmsNetId"
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Const Field (amsNetIdLength)
+	lengthInBits += 16
+
+	// Simple field (amsNetId)
+	lengthInBits += m.AmsNetId.GetLengthInBits()
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockAmsNetIdParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockAmsNetId, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockAmsNetId"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockAmsNetId")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Const Field (amsNetIdLength)
+	amsNetIdLength, _amsNetIdLengthErr := readBuffer.ReadUint16("amsNetIdLength", 16)
+	if _amsNetIdLengthErr != nil {
+		return nil, errors.Wrap(_amsNetIdLengthErr, "Error parsing 'amsNetIdLength' field of AdsDiscoveryBlockAmsNetId")
+	}
+	if amsNetIdLength != AdsDiscoveryBlockAmsNetId_AMSNETIDLENGTH {
+		return nil, errors.New("Expected constant value " + fmt.Sprintf("%d", AdsDiscoveryBlockAmsNetId_AMSNETIDLENGTH) + " but got " + fmt.Sprintf("%d", amsNetIdLength))
+	}
+
+	// Simple Field (amsNetId)
+	if pullErr := readBuffer.PullContext("amsNetId"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for amsNetId")
+	}
+	_amsNetId, _amsNetIdErr := AmsNetIdParse(readBuffer)
+	if _amsNetIdErr != nil {
+		return nil, errors.Wrap(_amsNetIdErr, "Error parsing 'amsNetId' field of AdsDiscoveryBlockAmsNetId")
+	}
+	amsNetId := _amsNetId.(AmsNetId)
+	if closeErr := readBuffer.CloseContext("amsNetId"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for amsNetId")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockAmsNetId"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockAmsNetId")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockAmsNetId{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		AmsNetId:           amsNetId,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockAmsNetId"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockAmsNetId")
+		}
+
+		// Const Field (amsNetIdLength)
+		_amsNetIdLengthErr := writeBuffer.WriteUint16("amsNetIdLength", 16, 0x0006)
+		if _amsNetIdLengthErr != nil {
+			return errors.Wrap(_amsNetIdLengthErr, "Error serializing 'amsNetIdLength' field")
+		}
+
+		// Simple Field (amsNetId)
+		if pushErr := writeBuffer.PushContext("amsNetId"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for amsNetId")
+		}
+		_amsNetIdErr := writeBuffer.WriteSerializable(m.GetAmsNetId())
+		if popErr := writeBuffer.PopContext("amsNetId"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for amsNetId")
+		}
+		if _amsNetIdErr != nil {
+			return errors.Wrap(_amsNetIdErr, "Error serializing 'amsNetId' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockAmsNetId"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockAmsNetId")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) isAdsDiscoveryBlockAmsNetId() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockAmsNetId) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockFingerprint.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockFingerprint.go
new file mode 100644
index 000000000..6ef5d3fef
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockFingerprint.go
@@ -0,0 +1,209 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockFingerprint is the corresponding interface of AdsDiscoveryBlockFingerprint
+type AdsDiscoveryBlockFingerprint interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetData returns Data (property field)
+	GetData() []byte
+}
+
+// AdsDiscoveryBlockFingerprintExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockFingerprint.
+// This is useful for switch cases.
+type AdsDiscoveryBlockFingerprintExactly interface {
+	AdsDiscoveryBlockFingerprint
+	isAdsDiscoveryBlockFingerprint() bool
+}
+
+// _AdsDiscoveryBlockFingerprint is the data-structure of this message
+type _AdsDiscoveryBlockFingerprint struct {
+	*_AdsDiscoveryBlock
+	Data []byte
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockFingerprint) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_FINGERPRINT
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockFingerprint) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockFingerprint) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockFingerprint) GetData() []byte {
+	return m.Data
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockFingerprint factory function for _AdsDiscoveryBlockFingerprint
+func NewAdsDiscoveryBlockFingerprint(data []byte) *_AdsDiscoveryBlockFingerprint {
+	_result := &_AdsDiscoveryBlockFingerprint{
+		Data:               data,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockFingerprint(structType interface{}) AdsDiscoveryBlockFingerprint {
+	if casted, ok := structType.(AdsDiscoveryBlockFingerprint); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockFingerprint); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) GetTypeName() string {
+	return "AdsDiscoveryBlockFingerprint"
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Implicit Field (dataLen)
+	lengthInBits += 16
+
+	// Array field
+	if len(m.Data) > 0 {
+		lengthInBits += 8 * uint16(len(m.Data))
+	}
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockFingerprintParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockFingerprint, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockFingerprint"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockFingerprint")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Implicit Field (dataLen) (Used for parsing, but its value is not stored as it's implicitly given by the objects content)
+	dataLen, _dataLenErr := readBuffer.ReadUint16("dataLen", 16)
+	_ = dataLen
+	if _dataLenErr != nil {
+		return nil, errors.Wrap(_dataLenErr, "Error parsing 'dataLen' field of AdsDiscoveryBlockFingerprint")
+	}
+	// Byte Array field (data)
+	numberOfBytesdata := int(dataLen)
+	data, _readArrayErr := readBuffer.ReadByteArray("data", numberOfBytesdata)
+	if _readArrayErr != nil {
+		return nil, errors.Wrap(_readArrayErr, "Error parsing 'data' field of AdsDiscoveryBlockFingerprint")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockFingerprint"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockFingerprint")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockFingerprint{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		Data:               data,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockFingerprint"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockFingerprint")
+		}
+
+		// Implicit Field (dataLen) (Used for parsing, but it's value is not stored as it's implicitly given by the objects content)
+		dataLen := uint16(uint16(len(m.GetData())))
+		_dataLenErr := writeBuffer.WriteUint16("dataLen", 16, (dataLen))
+		if _dataLenErr != nil {
+			return errors.Wrap(_dataLenErr, "Error serializing 'dataLen' field")
+		}
+
+		// Array Field (data)
+		// Byte Array field (data)
+		if err := writeBuffer.WriteByteArray("data", m.GetData()); err != nil {
+			return errors.Wrap(err, "Error serializing 'data' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockFingerprint"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockFingerprint")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) isAdsDiscoveryBlockFingerprint() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockFingerprint) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockHostName.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockHostName.go
new file mode 100644
index 000000000..9ba311f8a
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockHostName.go
@@ -0,0 +1,203 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockHostName is the corresponding interface of AdsDiscoveryBlockHostName
+type AdsDiscoveryBlockHostName interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetHostName returns HostName (property field)
+	GetHostName() AmsString
+}
+
+// AdsDiscoveryBlockHostNameExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockHostName.
+// This is useful for switch cases.
+type AdsDiscoveryBlockHostNameExactly interface {
+	AdsDiscoveryBlockHostName
+	isAdsDiscoveryBlockHostName() bool
+}
+
+// _AdsDiscoveryBlockHostName is the data-structure of this message
+type _AdsDiscoveryBlockHostName struct {
+	*_AdsDiscoveryBlock
+	HostName AmsString
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockHostName) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_HOST_NAME
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockHostName) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockHostName) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockHostName) GetHostName() AmsString {
+	return m.HostName
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockHostName factory function for _AdsDiscoveryBlockHostName
+func NewAdsDiscoveryBlockHostName(hostName AmsString) *_AdsDiscoveryBlockHostName {
+	_result := &_AdsDiscoveryBlockHostName{
+		HostName:           hostName,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockHostName(structType interface{}) AdsDiscoveryBlockHostName {
+	if casted, ok := structType.(AdsDiscoveryBlockHostName); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockHostName); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockHostName) GetTypeName() string {
+	return "AdsDiscoveryBlockHostName"
+}
+
+func (m *_AdsDiscoveryBlockHostName) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockHostName) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Simple field (hostName)
+	lengthInBits += m.HostName.GetLengthInBits()
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockHostName) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockHostNameParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockHostName, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockHostName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockHostName")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Simple Field (hostName)
+	if pullErr := readBuffer.PullContext("hostName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for hostName")
+	}
+	_hostName, _hostNameErr := AmsStringParse(readBuffer)
+	if _hostNameErr != nil {
+		return nil, errors.Wrap(_hostNameErr, "Error parsing 'hostName' field of AdsDiscoveryBlockHostName")
+	}
+	hostName := _hostName.(AmsString)
+	if closeErr := readBuffer.CloseContext("hostName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for hostName")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockHostName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockHostName")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockHostName{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		HostName:           hostName,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockHostName) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockHostName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockHostName")
+		}
+
+		// Simple Field (hostName)
+		if pushErr := writeBuffer.PushContext("hostName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for hostName")
+		}
+		_hostNameErr := writeBuffer.WriteSerializable(m.GetHostName())
+		if popErr := writeBuffer.PopContext("hostName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for hostName")
+		}
+		if _hostNameErr != nil {
+			return errors.Wrap(_hostNameErr, "Error serializing 'hostName' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockHostName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockHostName")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockHostName) isAdsDiscoveryBlockHostName() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockHostName) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockOsData.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockOsData.go
new file mode 100644
index 000000000..b68f77d0f
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockOsData.go
@@ -0,0 +1,209 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockOsData is the corresponding interface of AdsDiscoveryBlockOsData
+type AdsDiscoveryBlockOsData interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetOsData returns OsData (property field)
+	GetOsData() []byte
+}
+
+// AdsDiscoveryBlockOsDataExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockOsData.
+// This is useful for switch cases.
+type AdsDiscoveryBlockOsDataExactly interface {
+	AdsDiscoveryBlockOsData
+	isAdsDiscoveryBlockOsData() bool
+}
+
+// _AdsDiscoveryBlockOsData is the data-structure of this message
+type _AdsDiscoveryBlockOsData struct {
+	*_AdsDiscoveryBlock
+	OsData []byte
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockOsData) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_OS_DATA
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockOsData) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockOsData) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockOsData) GetOsData() []byte {
+	return m.OsData
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockOsData factory function for _AdsDiscoveryBlockOsData
+func NewAdsDiscoveryBlockOsData(osData []byte) *_AdsDiscoveryBlockOsData {
+	_result := &_AdsDiscoveryBlockOsData{
+		OsData:             osData,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockOsData(structType interface{}) AdsDiscoveryBlockOsData {
+	if casted, ok := structType.(AdsDiscoveryBlockOsData); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockOsData); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockOsData) GetTypeName() string {
+	return "AdsDiscoveryBlockOsData"
+}
+
+func (m *_AdsDiscoveryBlockOsData) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockOsData) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Implicit Field (osDataLen)
+	lengthInBits += 16
+
+	// Array field
+	if len(m.OsData) > 0 {
+		lengthInBits += 8 * uint16(len(m.OsData))
+	}
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockOsData) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockOsDataParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockOsData, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockOsData"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockOsData")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Implicit Field (osDataLen) (Used for parsing, but its value is not stored as it's implicitly given by the objects content)
+	osDataLen, _osDataLenErr := readBuffer.ReadUint16("osDataLen", 16)
+	_ = osDataLen
+	if _osDataLenErr != nil {
+		return nil, errors.Wrap(_osDataLenErr, "Error parsing 'osDataLen' field of AdsDiscoveryBlockOsData")
+	}
+	// Byte Array field (osData)
+	numberOfBytesosData := int(osDataLen)
+	osData, _readArrayErr := readBuffer.ReadByteArray("osData", numberOfBytesosData)
+	if _readArrayErr != nil {
+		return nil, errors.Wrap(_readArrayErr, "Error parsing 'osData' field of AdsDiscoveryBlockOsData")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockOsData"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockOsData")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockOsData{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		OsData:             osData,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockOsData) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockOsData"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockOsData")
+		}
+
+		// Implicit Field (osDataLen) (Used for parsing, but it's value is not stored as it's implicitly given by the objects content)
+		osDataLen := uint16(uint16(len(m.GetOsData())))
+		_osDataLenErr := writeBuffer.WriteUint16("osDataLen", 16, (osDataLen))
+		if _osDataLenErr != nil {
+			return errors.Wrap(_osDataLenErr, "Error serializing 'osDataLen' field")
+		}
+
+		// Array Field (osData)
+		// Byte Array field (osData)
+		if err := writeBuffer.WriteByteArray("osData", m.GetOsData()); err != nil {
+			return errors.Wrap(err, "Error serializing 'osData' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockOsData"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockOsData")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockOsData) isAdsDiscoveryBlockOsData() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockOsData) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockPassword.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockPassword.go
new file mode 100644
index 000000000..d0d5cf6e8
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockPassword.go
@@ -0,0 +1,203 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockPassword is the corresponding interface of AdsDiscoveryBlockPassword
+type AdsDiscoveryBlockPassword interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetPassword returns Password (property field)
+	GetPassword() AmsString
+}
+
+// AdsDiscoveryBlockPasswordExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockPassword.
+// This is useful for switch cases.
+type AdsDiscoveryBlockPasswordExactly interface {
+	AdsDiscoveryBlockPassword
+	isAdsDiscoveryBlockPassword() bool
+}
+
+// _AdsDiscoveryBlockPassword is the data-structure of this message
+type _AdsDiscoveryBlockPassword struct {
+	*_AdsDiscoveryBlock
+	Password AmsString
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockPassword) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_PASSWORD
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockPassword) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockPassword) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockPassword) GetPassword() AmsString {
+	return m.Password
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockPassword factory function for _AdsDiscoveryBlockPassword
+func NewAdsDiscoveryBlockPassword(password AmsString) *_AdsDiscoveryBlockPassword {
+	_result := &_AdsDiscoveryBlockPassword{
+		Password:           password,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockPassword(structType interface{}) AdsDiscoveryBlockPassword {
+	if casted, ok := structType.(AdsDiscoveryBlockPassword); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockPassword); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockPassword) GetTypeName() string {
+	return "AdsDiscoveryBlockPassword"
+}
+
+func (m *_AdsDiscoveryBlockPassword) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockPassword) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Simple field (password)
+	lengthInBits += m.Password.GetLengthInBits()
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockPassword) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockPasswordParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockPassword, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockPassword"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockPassword")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Simple Field (password)
+	if pullErr := readBuffer.PullContext("password"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for password")
+	}
+	_password, _passwordErr := AmsStringParse(readBuffer)
+	if _passwordErr != nil {
+		return nil, errors.Wrap(_passwordErr, "Error parsing 'password' field of AdsDiscoveryBlockPassword")
+	}
+	password := _password.(AmsString)
+	if closeErr := readBuffer.CloseContext("password"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for password")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockPassword"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockPassword")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockPassword{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		Password:           password,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockPassword) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockPassword"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockPassword")
+		}
+
+		// Simple Field (password)
+		if pushErr := writeBuffer.PushContext("password"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for password")
+		}
+		_passwordErr := writeBuffer.WriteSerializable(m.GetPassword())
+		if popErr := writeBuffer.PopContext("password"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for password")
+		}
+		if _passwordErr != nil {
+			return errors.Wrap(_passwordErr, "Error serializing 'password' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockPassword"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockPassword")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockPassword) isAdsDiscoveryBlockPassword() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockPassword) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockRouteName.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockRouteName.go
new file mode 100644
index 000000000..7ac1ad313
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockRouteName.go
@@ -0,0 +1,203 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockRouteName is the corresponding interface of AdsDiscoveryBlockRouteName
+type AdsDiscoveryBlockRouteName interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetRouteName returns RouteName (property field)
+	GetRouteName() AmsString
+}
+
+// AdsDiscoveryBlockRouteNameExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockRouteName.
+// This is useful for switch cases.
+type AdsDiscoveryBlockRouteNameExactly interface {
+	AdsDiscoveryBlockRouteName
+	isAdsDiscoveryBlockRouteName() bool
+}
+
+// _AdsDiscoveryBlockRouteName is the data-structure of this message
+type _AdsDiscoveryBlockRouteName struct {
+	*_AdsDiscoveryBlock
+	RouteName AmsString
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockRouteName) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_ROUTE_NAME
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockRouteName) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockRouteName) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockRouteName) GetRouteName() AmsString {
+	return m.RouteName
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockRouteName factory function for _AdsDiscoveryBlockRouteName
+func NewAdsDiscoveryBlockRouteName(routeName AmsString) *_AdsDiscoveryBlockRouteName {
+	_result := &_AdsDiscoveryBlockRouteName{
+		RouteName:          routeName,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockRouteName(structType interface{}) AdsDiscoveryBlockRouteName {
+	if casted, ok := structType.(AdsDiscoveryBlockRouteName); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockRouteName); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockRouteName) GetTypeName() string {
+	return "AdsDiscoveryBlockRouteName"
+}
+
+func (m *_AdsDiscoveryBlockRouteName) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockRouteName) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Simple field (routeName)
+	lengthInBits += m.RouteName.GetLengthInBits()
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockRouteName) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockRouteNameParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockRouteName, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockRouteName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockRouteName")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Simple Field (routeName)
+	if pullErr := readBuffer.PullContext("routeName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for routeName")
+	}
+	_routeName, _routeNameErr := AmsStringParse(readBuffer)
+	if _routeNameErr != nil {
+		return nil, errors.Wrap(_routeNameErr, "Error parsing 'routeName' field of AdsDiscoveryBlockRouteName")
+	}
+	routeName := _routeName.(AmsString)
+	if closeErr := readBuffer.CloseContext("routeName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for routeName")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockRouteName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockRouteName")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockRouteName{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		RouteName:          routeName,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockRouteName) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockRouteName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockRouteName")
+		}
+
+		// Simple Field (routeName)
+		if pushErr := writeBuffer.PushContext("routeName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for routeName")
+		}
+		_routeNameErr := writeBuffer.WriteSerializable(m.GetRouteName())
+		if popErr := writeBuffer.PopContext("routeName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for routeName")
+		}
+		if _routeNameErr != nil {
+			return errors.Wrap(_routeNameErr, "Error serializing 'routeName' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockRouteName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockRouteName")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockRouteName) isAdsDiscoveryBlockRouteName() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockRouteName) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockStatus.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockStatus.go
new file mode 100644
index 000000000..fc7b087b1
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockStatus.go
@@ -0,0 +1,238 @@
+/*
+ * 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 model
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Constant values.
+const AdsDiscoveryBlockStatus_STATUSLENGTH uint16 = 0x0004
+
+// AdsDiscoveryBlockStatus is the corresponding interface of AdsDiscoveryBlockStatus
+type AdsDiscoveryBlockStatus interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetStatus returns Status (property field)
+	GetStatus() Status
+}
+
+// AdsDiscoveryBlockStatusExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockStatus.
+// This is useful for switch cases.
+type AdsDiscoveryBlockStatusExactly interface {
+	AdsDiscoveryBlockStatus
+	isAdsDiscoveryBlockStatus() bool
+}
+
+// _AdsDiscoveryBlockStatus is the data-structure of this message
+type _AdsDiscoveryBlockStatus struct {
+	*_AdsDiscoveryBlock
+	Status Status
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockStatus) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_STATUS
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockStatus) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockStatus) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockStatus) GetStatus() Status {
+	return m.Status
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for const fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockStatus) GetStatusLength() uint16 {
+	return AdsDiscoveryBlockStatus_STATUSLENGTH
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockStatus factory function for _AdsDiscoveryBlockStatus
+func NewAdsDiscoveryBlockStatus(status Status) *_AdsDiscoveryBlockStatus {
+	_result := &_AdsDiscoveryBlockStatus{
+		Status:             status,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockStatus(structType interface{}) AdsDiscoveryBlockStatus {
+	if casted, ok := structType.(AdsDiscoveryBlockStatus); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockStatus); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockStatus) GetTypeName() string {
+	return "AdsDiscoveryBlockStatus"
+}
+
+func (m *_AdsDiscoveryBlockStatus) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockStatus) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Const Field (statusLength)
+	lengthInBits += 16
+
+	// Simple field (status)
+	lengthInBits += 32
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockStatus) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockStatusParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockStatus, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockStatus"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockStatus")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Const Field (statusLength)
+	statusLength, _statusLengthErr := readBuffer.ReadUint16("statusLength", 16)
+	if _statusLengthErr != nil {
+		return nil, errors.Wrap(_statusLengthErr, "Error parsing 'statusLength' field of AdsDiscoveryBlockStatus")
+	}
+	if statusLength != AdsDiscoveryBlockStatus_STATUSLENGTH {
+		return nil, errors.New("Expected constant value " + fmt.Sprintf("%d", AdsDiscoveryBlockStatus_STATUSLENGTH) + " but got " + fmt.Sprintf("%d", statusLength))
+	}
+
+	// Simple Field (status)
+	if pullErr := readBuffer.PullContext("status"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for status")
+	}
+	_status, _statusErr := StatusParse(readBuffer)
+	if _statusErr != nil {
+		return nil, errors.Wrap(_statusErr, "Error parsing 'status' field of AdsDiscoveryBlockStatus")
+	}
+	status := _status
+	if closeErr := readBuffer.CloseContext("status"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for status")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockStatus"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockStatus")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockStatus{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		Status:             status,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockStatus) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockStatus"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockStatus")
+		}
+
+		// Const Field (statusLength)
+		_statusLengthErr := writeBuffer.WriteUint16("statusLength", 16, 0x0004)
+		if _statusLengthErr != nil {
+			return errors.Wrap(_statusLengthErr, "Error serializing 'statusLength' field")
+		}
+
+		// Simple Field (status)
+		if pushErr := writeBuffer.PushContext("status"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for status")
+		}
+		_statusErr := writeBuffer.WriteSerializable(m.GetStatus())
+		if popErr := writeBuffer.PopContext("status"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for status")
+		}
+		if _statusErr != nil {
+			return errors.Wrap(_statusErr, "Error serializing 'status' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockStatus"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockStatus")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockStatus) isAdsDiscoveryBlockStatus() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockStatus) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockType.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockType.go
new file mode 100644
index 000000000..88bd4509c
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockType.go
@@ -0,0 +1,184 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockType is an enum
+type AdsDiscoveryBlockType uint16
+
+type IAdsDiscoveryBlockType interface {
+	Serialize(writeBuffer utils.WriteBuffer) error
+}
+
+const (
+	AdsDiscoveryBlockType_STATUS      AdsDiscoveryBlockType = 0x0001
+	AdsDiscoveryBlockType_PASSWORD    AdsDiscoveryBlockType = 0x0002
+	AdsDiscoveryBlockType_VERSION     AdsDiscoveryBlockType = 0x0003
+	AdsDiscoveryBlockType_OS_DATA     AdsDiscoveryBlockType = 0x0004
+	AdsDiscoveryBlockType_HOST_NAME   AdsDiscoveryBlockType = 0x0005
+	AdsDiscoveryBlockType_AMS_NET_ID  AdsDiscoveryBlockType = 0x0007
+	AdsDiscoveryBlockType_ROUTE_NAME  AdsDiscoveryBlockType = 0x000C
+	AdsDiscoveryBlockType_USER_NAME   AdsDiscoveryBlockType = 0x000D
+	AdsDiscoveryBlockType_FINGERPRINT AdsDiscoveryBlockType = 0x0012
+)
+
+var AdsDiscoveryBlockTypeValues []AdsDiscoveryBlockType
+
+func init() {
+	_ = errors.New
+	AdsDiscoveryBlockTypeValues = []AdsDiscoveryBlockType{
+		AdsDiscoveryBlockType_STATUS,
+		AdsDiscoveryBlockType_PASSWORD,
+		AdsDiscoveryBlockType_VERSION,
+		AdsDiscoveryBlockType_OS_DATA,
+		AdsDiscoveryBlockType_HOST_NAME,
+		AdsDiscoveryBlockType_AMS_NET_ID,
+		AdsDiscoveryBlockType_ROUTE_NAME,
+		AdsDiscoveryBlockType_USER_NAME,
+		AdsDiscoveryBlockType_FINGERPRINT,
+	}
+}
+
+func AdsDiscoveryBlockTypeByValue(value uint16) (enum AdsDiscoveryBlockType, ok bool) {
+	switch value {
+	case 0x0001:
+		return AdsDiscoveryBlockType_STATUS, true
+	case 0x0002:
+		return AdsDiscoveryBlockType_PASSWORD, true
+	case 0x0003:
+		return AdsDiscoveryBlockType_VERSION, true
+	case 0x0004:
+		return AdsDiscoveryBlockType_OS_DATA, true
+	case 0x0005:
+		return AdsDiscoveryBlockType_HOST_NAME, true
+	case 0x0007:
+		return AdsDiscoveryBlockType_AMS_NET_ID, true
+	case 0x000C:
+		return AdsDiscoveryBlockType_ROUTE_NAME, true
+	case 0x000D:
+		return AdsDiscoveryBlockType_USER_NAME, true
+	case 0x0012:
+		return AdsDiscoveryBlockType_FINGERPRINT, true
+	}
+	return 0, false
+}
+
+func AdsDiscoveryBlockTypeByName(value string) (enum AdsDiscoveryBlockType, ok bool) {
+	switch value {
+	case "STATUS":
+		return AdsDiscoveryBlockType_STATUS, true
+	case "PASSWORD":
+		return AdsDiscoveryBlockType_PASSWORD, true
+	case "VERSION":
+		return AdsDiscoveryBlockType_VERSION, true
+	case "OS_DATA":
+		return AdsDiscoveryBlockType_OS_DATA, true
+	case "HOST_NAME":
+		return AdsDiscoveryBlockType_HOST_NAME, true
+	case "AMS_NET_ID":
+		return AdsDiscoveryBlockType_AMS_NET_ID, true
+	case "ROUTE_NAME":
+		return AdsDiscoveryBlockType_ROUTE_NAME, true
+	case "USER_NAME":
+		return AdsDiscoveryBlockType_USER_NAME, true
+	case "FINGERPRINT":
+		return AdsDiscoveryBlockType_FINGERPRINT, true
+	}
+	return 0, false
+}
+
+func AdsDiscoveryBlockTypeKnows(value uint16) bool {
+	for _, typeValue := range AdsDiscoveryBlockTypeValues {
+		if uint16(typeValue) == value {
+			return true
+		}
+	}
+	return false
+}
+
+func CastAdsDiscoveryBlockType(structType interface{}) AdsDiscoveryBlockType {
+	castFunc := func(typ interface{}) AdsDiscoveryBlockType {
+		if sAdsDiscoveryBlockType, ok := typ.(AdsDiscoveryBlockType); ok {
+			return sAdsDiscoveryBlockType
+		}
+		return 0
+	}
+	return castFunc(structType)
+}
+
+func (m AdsDiscoveryBlockType) GetLengthInBits() uint16 {
+	return 16
+}
+
+func (m AdsDiscoveryBlockType) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockTypeParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockType, error) {
+	val, err := readBuffer.ReadUint16("AdsDiscoveryBlockType", 16)
+	if err != nil {
+		return 0, errors.Wrap(err, "error reading AdsDiscoveryBlockType")
+	}
+	if enum, ok := AdsDiscoveryBlockTypeByValue(val); !ok {
+		Plc4xModelLog.Debug().Msgf("no value %x found for RequestType", val)
+		return AdsDiscoveryBlockType(val), nil
+	} else {
+		return enum, nil
+	}
+}
+
+func (e AdsDiscoveryBlockType) Serialize(writeBuffer utils.WriteBuffer) error {
+	return writeBuffer.WriteUint16("AdsDiscoveryBlockType", 16, uint16(e), utils.WithAdditionalStringRepresentation(e.PLC4XEnumName()))
+}
+
+// PLC4XEnumName returns the name that is used in code to identify this enum
+func (e AdsDiscoveryBlockType) PLC4XEnumName() string {
+	switch e {
+	case AdsDiscoveryBlockType_STATUS:
+		return "STATUS"
+	case AdsDiscoveryBlockType_PASSWORD:
+		return "PASSWORD"
+	case AdsDiscoveryBlockType_VERSION:
+		return "VERSION"
+	case AdsDiscoveryBlockType_OS_DATA:
+		return "OS_DATA"
+	case AdsDiscoveryBlockType_HOST_NAME:
+		return "HOST_NAME"
+	case AdsDiscoveryBlockType_AMS_NET_ID:
+		return "AMS_NET_ID"
+	case AdsDiscoveryBlockType_ROUTE_NAME:
+		return "ROUTE_NAME"
+	case AdsDiscoveryBlockType_USER_NAME:
+		return "USER_NAME"
+	case AdsDiscoveryBlockType_FINGERPRINT:
+		return "FINGERPRINT"
+	}
+	return ""
+}
+
+func (e AdsDiscoveryBlockType) String() string {
+	return e.PLC4XEnumName()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockUserName.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockUserName.go
new file mode 100644
index 000000000..86cc2da29
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockUserName.go
@@ -0,0 +1,203 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockUserName is the corresponding interface of AdsDiscoveryBlockUserName
+type AdsDiscoveryBlockUserName interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetUserName returns UserName (property field)
+	GetUserName() AmsString
+}
+
+// AdsDiscoveryBlockUserNameExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockUserName.
+// This is useful for switch cases.
+type AdsDiscoveryBlockUserNameExactly interface {
+	AdsDiscoveryBlockUserName
+	isAdsDiscoveryBlockUserName() bool
+}
+
+// _AdsDiscoveryBlockUserName is the data-structure of this message
+type _AdsDiscoveryBlockUserName struct {
+	*_AdsDiscoveryBlock
+	UserName AmsString
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockUserName) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_USER_NAME
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockUserName) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockUserName) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockUserName) GetUserName() AmsString {
+	return m.UserName
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockUserName factory function for _AdsDiscoveryBlockUserName
+func NewAdsDiscoveryBlockUserName(userName AmsString) *_AdsDiscoveryBlockUserName {
+	_result := &_AdsDiscoveryBlockUserName{
+		UserName:           userName,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockUserName(structType interface{}) AdsDiscoveryBlockUserName {
+	if casted, ok := structType.(AdsDiscoveryBlockUserName); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockUserName); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockUserName) GetTypeName() string {
+	return "AdsDiscoveryBlockUserName"
+}
+
+func (m *_AdsDiscoveryBlockUserName) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockUserName) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Simple field (userName)
+	lengthInBits += m.UserName.GetLengthInBits()
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockUserName) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockUserNameParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockUserName, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockUserName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockUserName")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Simple Field (userName)
+	if pullErr := readBuffer.PullContext("userName"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for userName")
+	}
+	_userName, _userNameErr := AmsStringParse(readBuffer)
+	if _userNameErr != nil {
+		return nil, errors.Wrap(_userNameErr, "Error parsing 'userName' field of AdsDiscoveryBlockUserName")
+	}
+	userName := _userName.(AmsString)
+	if closeErr := readBuffer.CloseContext("userName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for userName")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockUserName"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockUserName")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockUserName{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		UserName:           userName,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockUserName) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockUserName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockUserName")
+		}
+
+		// Simple Field (userName)
+		if pushErr := writeBuffer.PushContext("userName"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for userName")
+		}
+		_userNameErr := writeBuffer.WriteSerializable(m.GetUserName())
+		if popErr := writeBuffer.PopContext("userName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for userName")
+		}
+		if _userNameErr != nil {
+			return errors.Wrap(_userNameErr, "Error serializing 'userName' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockUserName"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockUserName")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockUserName) isAdsDiscoveryBlockUserName() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockUserName) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockVersion.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockVersion.go
new file mode 100644
index 000000000..97a73fa49
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryBlockVersion.go
@@ -0,0 +1,209 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsDiscoveryBlockVersion is the corresponding interface of AdsDiscoveryBlockVersion
+type AdsDiscoveryBlockVersion interface {
+	utils.LengthAware
+	utils.Serializable
+	AdsDiscoveryBlock
+	// GetVersionData returns VersionData (property field)
+	GetVersionData() []byte
+}
+
+// AdsDiscoveryBlockVersionExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryBlockVersion.
+// This is useful for switch cases.
+type AdsDiscoveryBlockVersionExactly interface {
+	AdsDiscoveryBlockVersion
+	isAdsDiscoveryBlockVersion() bool
+}
+
+// _AdsDiscoveryBlockVersion is the data-structure of this message
+type _AdsDiscoveryBlockVersion struct {
+	*_AdsDiscoveryBlock
+	VersionData []byte
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for discriminator values.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockVersion) GetBlockType() AdsDiscoveryBlockType {
+	return AdsDiscoveryBlockType_VERSION
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+func (m *_AdsDiscoveryBlockVersion) InitializeParent(parent AdsDiscoveryBlock) {}
+
+func (m *_AdsDiscoveryBlockVersion) GetParent() AdsDiscoveryBlock {
+	return m._AdsDiscoveryBlock
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AdsDiscoveryBlockVersion) GetVersionData() []byte {
+	return m.VersionData
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryBlockVersion factory function for _AdsDiscoveryBlockVersion
+func NewAdsDiscoveryBlockVersion(versionData []byte) *_AdsDiscoveryBlockVersion {
+	_result := &_AdsDiscoveryBlockVersion{
+		VersionData:        versionData,
+		_AdsDiscoveryBlock: NewAdsDiscoveryBlock(),
+	}
+	_result._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _result
+	return _result
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryBlockVersion(structType interface{}) AdsDiscoveryBlockVersion {
+	if casted, ok := structType.(AdsDiscoveryBlockVersion); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryBlockVersion); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryBlockVersion) GetTypeName() string {
+	return "AdsDiscoveryBlockVersion"
+}
+
+func (m *_AdsDiscoveryBlockVersion) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryBlockVersion) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(m.GetParentLengthInBits())
+
+	// Implicit Field (versionDataLen)
+	lengthInBits += 16
+
+	// Array field
+	if len(m.VersionData) > 0 {
+		lengthInBits += 8 * uint16(len(m.VersionData))
+	}
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryBlockVersion) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryBlockVersionParse(readBuffer utils.ReadBuffer) (AdsDiscoveryBlockVersion, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryBlockVersion"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryBlockVersion")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Implicit Field (versionDataLen) (Used for parsing, but its value is not stored as it's implicitly given by the objects content)
+	versionDataLen, _versionDataLenErr := readBuffer.ReadUint16("versionDataLen", 16)
+	_ = versionDataLen
+	if _versionDataLenErr != nil {
+		return nil, errors.Wrap(_versionDataLenErr, "Error parsing 'versionDataLen' field of AdsDiscoveryBlockVersion")
+	}
+	// Byte Array field (versionData)
+	numberOfBytesversionData := int(versionDataLen)
+	versionData, _readArrayErr := readBuffer.ReadByteArray("versionData", numberOfBytesversionData)
+	if _readArrayErr != nil {
+		return nil, errors.Wrap(_readArrayErr, "Error parsing 'versionData' field of AdsDiscoveryBlockVersion")
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryBlockVersion"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryBlockVersion")
+	}
+
+	// Create a partially initialized instance
+	_child := &_AdsDiscoveryBlockVersion{
+		_AdsDiscoveryBlock: &_AdsDiscoveryBlock{},
+		VersionData:        versionData,
+	}
+	_child._AdsDiscoveryBlock._AdsDiscoveryBlockChildRequirements = _child
+	return _child, nil
+}
+
+func (m *_AdsDiscoveryBlockVersion) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	ser := func() error {
+		if pushErr := writeBuffer.PushContext("AdsDiscoveryBlockVersion"); pushErr != nil {
+			return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryBlockVersion")
+		}
+
+		// Implicit Field (versionDataLen) (Used for parsing, but it's value is not stored as it's implicitly given by the objects content)
+		versionDataLen := uint16(uint16(len(m.GetVersionData())))
+		_versionDataLenErr := writeBuffer.WriteUint16("versionDataLen", 16, (versionDataLen))
+		if _versionDataLenErr != nil {
+			return errors.Wrap(_versionDataLenErr, "Error serializing 'versionDataLen' field")
+		}
+
+		// Array Field (versionData)
+		// Byte Array field (versionData)
+		if err := writeBuffer.WriteByteArray("versionData", m.GetVersionData()); err != nil {
+			return errors.Wrap(err, "Error serializing 'versionData' field")
+		}
+
+		if popErr := writeBuffer.PopContext("AdsDiscoveryBlockVersion"); popErr != nil {
+			return errors.Wrap(popErr, "Error popping for AdsDiscoveryBlockVersion")
+		}
+		return nil
+	}
+	return m.SerializeParent(writeBuffer, m, ser)
+}
+
+func (m *_AdsDiscoveryBlockVersion) isAdsDiscoveryBlockVersion() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryBlockVersion) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryConstants.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryConstants.go
new file mode 100644
index 000000000..23a63f807
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsDiscoveryConstants.go
@@ -0,0 +1,159 @@
+/*
+ * 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 model
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Constant values.
+const AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT uint16 = uint16(48899)
+
+// AdsDiscoveryConstants is the corresponding interface of AdsDiscoveryConstants
+type AdsDiscoveryConstants interface {
+	utils.LengthAware
+	utils.Serializable
+}
+
+// AdsDiscoveryConstantsExactly can be used when we want exactly this type and not a type which fulfills AdsDiscoveryConstants.
+// This is useful for switch cases.
+type AdsDiscoveryConstantsExactly interface {
+	AdsDiscoveryConstants
+	isAdsDiscoveryConstants() bool
+}
+
+// _AdsDiscoveryConstants is the data-structure of this message
+type _AdsDiscoveryConstants struct {
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for const fields.
+///////////////////////
+
+func (m *_AdsDiscoveryConstants) GetAdsDiscoveryUdpDefaultPort() uint16 {
+	return AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAdsDiscoveryConstants factory function for _AdsDiscoveryConstants
+func NewAdsDiscoveryConstants() *_AdsDiscoveryConstants {
+	return &_AdsDiscoveryConstants{}
+}
+
+// Deprecated: use the interface for direct cast
+func CastAdsDiscoveryConstants(structType interface{}) AdsDiscoveryConstants {
+	if casted, ok := structType.(AdsDiscoveryConstants); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AdsDiscoveryConstants); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryConstants) GetTypeName() string {
+	return "AdsDiscoveryConstants"
+}
+
+func (m *_AdsDiscoveryConstants) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AdsDiscoveryConstants) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(0)
+
+	// Const Field (adsDiscoveryUdpDefaultPort)
+	lengthInBits += 16
+
+	return lengthInBits
+}
+
+func (m *_AdsDiscoveryConstants) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsDiscoveryConstantsParse(readBuffer utils.ReadBuffer) (AdsDiscoveryConstants, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AdsDiscoveryConstants"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AdsDiscoveryConstants")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Const Field (adsDiscoveryUdpDefaultPort)
+	adsDiscoveryUdpDefaultPort, _adsDiscoveryUdpDefaultPortErr := readBuffer.ReadUint16("adsDiscoveryUdpDefaultPort", 16)
+	if _adsDiscoveryUdpDefaultPortErr != nil {
+		return nil, errors.Wrap(_adsDiscoveryUdpDefaultPortErr, "Error parsing 'adsDiscoveryUdpDefaultPort' field of AdsDiscoveryConstants")
+	}
+	if adsDiscoveryUdpDefaultPort != AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT {
+		return nil, errors.New("Expected constant value " + fmt.Sprintf("%d", AdsDiscoveryConstants_ADSDISCOVERYUDPDEFAULTPORT) + " but got " + fmt.Sprintf("%d", adsDiscoveryUdpDefaultPort))
+	}
+
+	if closeErr := readBuffer.CloseContext("AdsDiscoveryConstants"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AdsDiscoveryConstants")
+	}
+
+	// Create the instance
+	return &_AdsDiscoveryConstants{}, nil
+}
+
+func (m *_AdsDiscoveryConstants) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	if pushErr := writeBuffer.PushContext("AdsDiscoveryConstants"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for AdsDiscoveryConstants")
+	}
+
+	// Const Field (adsDiscoveryUdpDefaultPort)
+	_adsDiscoveryUdpDefaultPortErr := writeBuffer.WriteUint16("adsDiscoveryUdpDefaultPort", 16, 48899)
+	if _adsDiscoveryUdpDefaultPortErr != nil {
+		return errors.Wrap(_adsDiscoveryUdpDefaultPortErr, "Error serializing 'adsDiscoveryUdpDefaultPort' field")
+	}
+
+	if popErr := writeBuffer.PopContext("AdsDiscoveryConstants"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for AdsDiscoveryConstants")
+	}
+	return nil
+}
+
+func (m *_AdsDiscoveryConstants) isAdsDiscoveryConstants() bool {
+	return true
+}
+
+func (m *_AdsDiscoveryConstants) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AdsPortNumbers.go b/plc4go/protocols/ads/discovery/readwrite/model/AdsPortNumbers.go
new file mode 100644
index 000000000..a9636f43d
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AdsPortNumbers.go
@@ -0,0 +1,216 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AdsPortNumbers is an enum
+type AdsPortNumbers uint16
+
+type IAdsPortNumbers interface {
+	Serialize(writeBuffer utils.WriteBuffer) error
+}
+
+const (
+	AdsPortNumbers_LOGGER               AdsPortNumbers = 100
+	AdsPortNumbers_EVENT_LOGGER         AdsPortNumbers = 110
+	AdsPortNumbers_IO                   AdsPortNumbers = 300
+	AdsPortNumbers_ADDITIONAL_TASK_1    AdsPortNumbers = 301
+	AdsPortNumbers_ADDITIONAL_TASK_2    AdsPortNumbers = 302
+	AdsPortNumbers_NC                   AdsPortNumbers = 500
+	AdsPortNumbers_PLC_RUNTIME_SYSTEM_1 AdsPortNumbers = 801
+	AdsPortNumbers_PLC_RUNTIME_SYSTEM_2 AdsPortNumbers = 811
+	AdsPortNumbers_PLC_RUNTIME_SYSTEM_3 AdsPortNumbers = 821
+	AdsPortNumbers_PLC_RUNTIME_SYSTEM_4 AdsPortNumbers = 831
+	AdsPortNumbers_CAM_SWITCH           AdsPortNumbers = 900
+	AdsPortNumbers_SYSTEM_SERVICE       AdsPortNumbers = 10000
+	AdsPortNumbers_SCOPE                AdsPortNumbers = 14000
+)
+
+var AdsPortNumbersValues []AdsPortNumbers
+
+func init() {
+	_ = errors.New
+	AdsPortNumbersValues = []AdsPortNumbers{
+		AdsPortNumbers_LOGGER,
+		AdsPortNumbers_EVENT_LOGGER,
+		AdsPortNumbers_IO,
+		AdsPortNumbers_ADDITIONAL_TASK_1,
+		AdsPortNumbers_ADDITIONAL_TASK_2,
+		AdsPortNumbers_NC,
+		AdsPortNumbers_PLC_RUNTIME_SYSTEM_1,
+		AdsPortNumbers_PLC_RUNTIME_SYSTEM_2,
+		AdsPortNumbers_PLC_RUNTIME_SYSTEM_3,
+		AdsPortNumbers_PLC_RUNTIME_SYSTEM_4,
+		AdsPortNumbers_CAM_SWITCH,
+		AdsPortNumbers_SYSTEM_SERVICE,
+		AdsPortNumbers_SCOPE,
+	}
+}
+
+func AdsPortNumbersByValue(value uint16) (enum AdsPortNumbers, ok bool) {
+	switch value {
+	case 100:
+		return AdsPortNumbers_LOGGER, true
+	case 10000:
+		return AdsPortNumbers_SYSTEM_SERVICE, true
+	case 110:
+		return AdsPortNumbers_EVENT_LOGGER, true
+	case 14000:
+		return AdsPortNumbers_SCOPE, true
+	case 300:
+		return AdsPortNumbers_IO, true
+	case 301:
+		return AdsPortNumbers_ADDITIONAL_TASK_1, true
+	case 302:
+		return AdsPortNumbers_ADDITIONAL_TASK_2, true
+	case 500:
+		return AdsPortNumbers_NC, true
+	case 801:
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_1, true
+	case 811:
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_2, true
+	case 821:
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_3, true
+	case 831:
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_4, true
+	case 900:
+		return AdsPortNumbers_CAM_SWITCH, true
+	}
+	return 0, false
+}
+
+func AdsPortNumbersByName(value string) (enum AdsPortNumbers, ok bool) {
+	switch value {
+	case "LOGGER":
+		return AdsPortNumbers_LOGGER, true
+	case "SYSTEM_SERVICE":
+		return AdsPortNumbers_SYSTEM_SERVICE, true
+	case "EVENT_LOGGER":
+		return AdsPortNumbers_EVENT_LOGGER, true
+	case "SCOPE":
+		return AdsPortNumbers_SCOPE, true
+	case "IO":
+		return AdsPortNumbers_IO, true
+	case "ADDITIONAL_TASK_1":
+		return AdsPortNumbers_ADDITIONAL_TASK_1, true
+	case "ADDITIONAL_TASK_2":
+		return AdsPortNumbers_ADDITIONAL_TASK_2, true
+	case "NC":
+		return AdsPortNumbers_NC, true
+	case "PLC_RUNTIME_SYSTEM_1":
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_1, true
+	case "PLC_RUNTIME_SYSTEM_2":
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_2, true
+	case "PLC_RUNTIME_SYSTEM_3":
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_3, true
+	case "PLC_RUNTIME_SYSTEM_4":
+		return AdsPortNumbers_PLC_RUNTIME_SYSTEM_4, true
+	case "CAM_SWITCH":
+		return AdsPortNumbers_CAM_SWITCH, true
+	}
+	return 0, false
+}
+
+func AdsPortNumbersKnows(value uint16) bool {
+	for _, typeValue := range AdsPortNumbersValues {
+		if uint16(typeValue) == value {
+			return true
+		}
+	}
+	return false
+}
+
+func CastAdsPortNumbers(structType interface{}) AdsPortNumbers {
+	castFunc := func(typ interface{}) AdsPortNumbers {
+		if sAdsPortNumbers, ok := typ.(AdsPortNumbers); ok {
+			return sAdsPortNumbers
+		}
+		return 0
+	}
+	return castFunc(structType)
+}
+
+func (m AdsPortNumbers) GetLengthInBits() uint16 {
+	return 16
+}
+
+func (m AdsPortNumbers) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AdsPortNumbersParse(readBuffer utils.ReadBuffer) (AdsPortNumbers, error) {
+	val, err := readBuffer.ReadUint16("AdsPortNumbers", 16)
+	if err != nil {
+		return 0, errors.Wrap(err, "error reading AdsPortNumbers")
+	}
+	if enum, ok := AdsPortNumbersByValue(val); !ok {
+		Plc4xModelLog.Debug().Msgf("no value %x found for RequestType", val)
+		return AdsPortNumbers(val), nil
+	} else {
+		return enum, nil
+	}
+}
+
+func (e AdsPortNumbers) Serialize(writeBuffer utils.WriteBuffer) error {
+	return writeBuffer.WriteUint16("AdsPortNumbers", 16, uint16(e), utils.WithAdditionalStringRepresentation(e.PLC4XEnumName()))
+}
+
+// PLC4XEnumName returns the name that is used in code to identify this enum
+func (e AdsPortNumbers) PLC4XEnumName() string {
+	switch e {
+	case AdsPortNumbers_LOGGER:
+		return "LOGGER"
+	case AdsPortNumbers_SYSTEM_SERVICE:
+		return "SYSTEM_SERVICE"
+	case AdsPortNumbers_EVENT_LOGGER:
+		return "EVENT_LOGGER"
+	case AdsPortNumbers_SCOPE:
+		return "SCOPE"
+	case AdsPortNumbers_IO:
+		return "IO"
+	case AdsPortNumbers_ADDITIONAL_TASK_1:
+		return "ADDITIONAL_TASK_1"
+	case AdsPortNumbers_ADDITIONAL_TASK_2:
+		return "ADDITIONAL_TASK_2"
+	case AdsPortNumbers_NC:
+		return "NC"
+	case AdsPortNumbers_PLC_RUNTIME_SYSTEM_1:
+		return "PLC_RUNTIME_SYSTEM_1"
+	case AdsPortNumbers_PLC_RUNTIME_SYSTEM_2:
+		return "PLC_RUNTIME_SYSTEM_2"
+	case AdsPortNumbers_PLC_RUNTIME_SYSTEM_3:
+		return "PLC_RUNTIME_SYSTEM_3"
+	case AdsPortNumbers_PLC_RUNTIME_SYSTEM_4:
+		return "PLC_RUNTIME_SYSTEM_4"
+	case AdsPortNumbers_CAM_SWITCH:
+		return "CAM_SWITCH"
+	}
+	return ""
+}
+
+func (e AdsPortNumbers) String() string {
+	return e.PLC4XEnumName()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AmsNetId.go b/plc4go/protocols/ads/discovery/readwrite/model/AmsNetId.go
new file mode 100644
index 000000000..5c5401182
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AmsNetId.go
@@ -0,0 +1,284 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AmsNetId is the corresponding interface of AmsNetId
+type AmsNetId interface {
+	utils.LengthAware
+	utils.Serializable
+	// GetOctet1 returns Octet1 (property field)
+	GetOctet1() uint8
+	// GetOctet2 returns Octet2 (property field)
+	GetOctet2() uint8
+	// GetOctet3 returns Octet3 (property field)
+	GetOctet3() uint8
+	// GetOctet4 returns Octet4 (property field)
+	GetOctet4() uint8
+	// GetOctet5 returns Octet5 (property field)
+	GetOctet5() uint8
+	// GetOctet6 returns Octet6 (property field)
+	GetOctet6() uint8
+}
+
+// AmsNetIdExactly can be used when we want exactly this type and not a type which fulfills AmsNetId.
+// This is useful for switch cases.
+type AmsNetIdExactly interface {
+	AmsNetId
+	isAmsNetId() bool
+}
+
+// _AmsNetId is the data-structure of this message
+type _AmsNetId struct {
+	Octet1 uint8
+	Octet2 uint8
+	Octet3 uint8
+	Octet4 uint8
+	Octet5 uint8
+	Octet6 uint8
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AmsNetId) GetOctet1() uint8 {
+	return m.Octet1
+}
+
+func (m *_AmsNetId) GetOctet2() uint8 {
+	return m.Octet2
+}
+
+func (m *_AmsNetId) GetOctet3() uint8 {
+	return m.Octet3
+}
+
+func (m *_AmsNetId) GetOctet4() uint8 {
+	return m.Octet4
+}
+
+func (m *_AmsNetId) GetOctet5() uint8 {
+	return m.Octet5
+}
+
+func (m *_AmsNetId) GetOctet6() uint8 {
+	return m.Octet6
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAmsNetId factory function for _AmsNetId
+func NewAmsNetId(octet1 uint8, octet2 uint8, octet3 uint8, octet4 uint8, octet5 uint8, octet6 uint8) *_AmsNetId {
+	return &_AmsNetId{Octet1: octet1, Octet2: octet2, Octet3: octet3, Octet4: octet4, Octet5: octet5, Octet6: octet6}
+}
+
+// Deprecated: use the interface for direct cast
+func CastAmsNetId(structType interface{}) AmsNetId {
+	if casted, ok := structType.(AmsNetId); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AmsNetId); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AmsNetId) GetTypeName() string {
+	return "AmsNetId"
+}
+
+func (m *_AmsNetId) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AmsNetId) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(0)
+
+	// Simple field (octet1)
+	lengthInBits += 8
+
+	// Simple field (octet2)
+	lengthInBits += 8
+
+	// Simple field (octet3)
+	lengthInBits += 8
+
+	// Simple field (octet4)
+	lengthInBits += 8
+
+	// Simple field (octet5)
+	lengthInBits += 8
+
+	// Simple field (octet6)
+	lengthInBits += 8
+
+	return lengthInBits
+}
+
+func (m *_AmsNetId) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AmsNetIdParse(readBuffer utils.ReadBuffer) (AmsNetId, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AmsNetId"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AmsNetId")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Simple Field (octet1)
+	_octet1, _octet1Err := readBuffer.ReadUint8("octet1", 8)
+	if _octet1Err != nil {
+		return nil, errors.Wrap(_octet1Err, "Error parsing 'octet1' field of AmsNetId")
+	}
+	octet1 := _octet1
+
+	// Simple Field (octet2)
+	_octet2, _octet2Err := readBuffer.ReadUint8("octet2", 8)
+	if _octet2Err != nil {
+		return nil, errors.Wrap(_octet2Err, "Error parsing 'octet2' field of AmsNetId")
+	}
+	octet2 := _octet2
+
+	// Simple Field (octet3)
+	_octet3, _octet3Err := readBuffer.ReadUint8("octet3", 8)
+	if _octet3Err != nil {
+		return nil, errors.Wrap(_octet3Err, "Error parsing 'octet3' field of AmsNetId")
+	}
+	octet3 := _octet3
+
+	// Simple Field (octet4)
+	_octet4, _octet4Err := readBuffer.ReadUint8("octet4", 8)
+	if _octet4Err != nil {
+		return nil, errors.Wrap(_octet4Err, "Error parsing 'octet4' field of AmsNetId")
+	}
+	octet4 := _octet4
+
+	// Simple Field (octet5)
+	_octet5, _octet5Err := readBuffer.ReadUint8("octet5", 8)
+	if _octet5Err != nil {
+		return nil, errors.Wrap(_octet5Err, "Error parsing 'octet5' field of AmsNetId")
+	}
+	octet5 := _octet5
+
+	// Simple Field (octet6)
+	_octet6, _octet6Err := readBuffer.ReadUint8("octet6", 8)
+	if _octet6Err != nil {
+		return nil, errors.Wrap(_octet6Err, "Error parsing 'octet6' field of AmsNetId")
+	}
+	octet6 := _octet6
+
+	if closeErr := readBuffer.CloseContext("AmsNetId"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AmsNetId")
+	}
+
+	// Create the instance
+	return &_AmsNetId{
+		Octet1: octet1,
+		Octet2: octet2,
+		Octet3: octet3,
+		Octet4: octet4,
+		Octet5: octet5,
+		Octet6: octet6,
+	}, nil
+}
+
+func (m *_AmsNetId) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	if pushErr := writeBuffer.PushContext("AmsNetId"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for AmsNetId")
+	}
+
+	// Simple Field (octet1)
+	octet1 := uint8(m.GetOctet1())
+	_octet1Err := writeBuffer.WriteUint8("octet1", 8, (octet1))
+	if _octet1Err != nil {
+		return errors.Wrap(_octet1Err, "Error serializing 'octet1' field")
+	}
+
+	// Simple Field (octet2)
+	octet2 := uint8(m.GetOctet2())
+	_octet2Err := writeBuffer.WriteUint8("octet2", 8, (octet2))
+	if _octet2Err != nil {
+		return errors.Wrap(_octet2Err, "Error serializing 'octet2' field")
+	}
+
+	// Simple Field (octet3)
+	octet3 := uint8(m.GetOctet3())
+	_octet3Err := writeBuffer.WriteUint8("octet3", 8, (octet3))
+	if _octet3Err != nil {
+		return errors.Wrap(_octet3Err, "Error serializing 'octet3' field")
+	}
+
+	// Simple Field (octet4)
+	octet4 := uint8(m.GetOctet4())
+	_octet4Err := writeBuffer.WriteUint8("octet4", 8, (octet4))
+	if _octet4Err != nil {
+		return errors.Wrap(_octet4Err, "Error serializing 'octet4' field")
+	}
+
+	// Simple Field (octet5)
+	octet5 := uint8(m.GetOctet5())
+	_octet5Err := writeBuffer.WriteUint8("octet5", 8, (octet5))
+	if _octet5Err != nil {
+		return errors.Wrap(_octet5Err, "Error serializing 'octet5' field")
+	}
+
+	// Simple Field (octet6)
+	octet6 := uint8(m.GetOctet6())
+	_octet6Err := writeBuffer.WriteUint8("octet6", 8, (octet6))
+	if _octet6Err != nil {
+		return errors.Wrap(_octet6Err, "Error serializing 'octet6' field")
+	}
+
+	if popErr := writeBuffer.PopContext("AmsNetId"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for AmsNetId")
+	}
+	return nil
+}
+
+func (m *_AmsNetId) isAmsNetId() bool {
+	return true
+}
+
+func (m *_AmsNetId) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/AmsString.go b/plc4go/protocols/ads/discovery/readwrite/model/AmsString.go
new file mode 100644
index 000000000..501574de8
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/AmsString.go
@@ -0,0 +1,215 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// AmsString is the corresponding interface of AmsString
+type AmsString interface {
+	utils.LengthAware
+	utils.Serializable
+	// GetText returns Text (property field)
+	GetText() string
+}
+
+// AmsStringExactly can be used when we want exactly this type and not a type which fulfills AmsString.
+// This is useful for switch cases.
+type AmsStringExactly interface {
+	AmsString
+	isAmsString() bool
+}
+
+// _AmsString is the data-structure of this message
+type _AmsString struct {
+	Text string
+	// Reserved Fields
+	reservedField0 *uint8
+}
+
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+/////////////////////// Accessors for property fields.
+///////////////////////
+
+func (m *_AmsString) GetText() string {
+	return m.Text
+}
+
+///////////////////////
+///////////////////////
+///////////////////////////////////////////////////////////
+///////////////////////////////////////////////////////////
+
+// NewAmsString factory function for _AmsString
+func NewAmsString(text string) *_AmsString {
+	return &_AmsString{Text: text}
+}
+
+// Deprecated: use the interface for direct cast
+func CastAmsString(structType interface{}) AmsString {
+	if casted, ok := structType.(AmsString); ok {
+		return casted
+	}
+	if casted, ok := structType.(*AmsString); ok {
+		return *casted
+	}
+	return nil
+}
+
+func (m *_AmsString) GetTypeName() string {
+	return "AmsString"
+}
+
+func (m *_AmsString) GetLengthInBits() uint16 {
+	return m.GetLengthInBitsConditional(false)
+}
+
+func (m *_AmsString) GetLengthInBitsConditional(lastItem bool) uint16 {
+	lengthInBits := uint16(0)
+
+	// Implicit Field (strLen)
+	lengthInBits += 16
+
+	// Simple field (text)
+	lengthInBits += uint16(int32(int32(8)) * int32((int32(uint16(uint16(len(m.GetText())))+uint16(uint16(1))) - int32(int32(1)))))
+
+	// Reserved Field (reserved)
+	lengthInBits += 8
+
+	return lengthInBits
+}
+
+func (m *_AmsString) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func AmsStringParse(readBuffer utils.ReadBuffer) (AmsString, error) {
+	positionAware := readBuffer
+	_ = positionAware
+	if pullErr := readBuffer.PullContext("AmsString"); pullErr != nil {
+		return nil, errors.Wrap(pullErr, "Error pulling for AmsString")
+	}
+	currentPos := positionAware.GetPos()
+	_ = currentPos
+
+	// Implicit Field (strLen) (Used for parsing, but its value is not stored as it's implicitly given by the objects content)
+	strLen, _strLenErr := readBuffer.ReadUint16("strLen", 16)
+	_ = strLen
+	if _strLenErr != nil {
+		return nil, errors.Wrap(_strLenErr, "Error parsing 'strLen' field of AmsString")
+	}
+
+	// Simple Field (text)
+	_text, _textErr := readBuffer.ReadString("text", uint32((8)*((strLen)-(1))), "UTF-8")
+	if _textErr != nil {
+		return nil, errors.Wrap(_textErr, "Error parsing 'text' field of AmsString")
+	}
+	text := _text
+
+	var reservedField0 *uint8
+	// Reserved Field (Compartmentalized so the "reserved" variable can't leak)
+	{
+		reserved, _err := readBuffer.ReadUint8("reserved", 8)
+		if _err != nil {
+			return nil, errors.Wrap(_err, "Error parsing 'reserved' field of AmsString")
+		}
+		if reserved != uint8(0x00) {
+			Plc4xModelLog.Info().Fields(map[string]interface{}{
+				"expected value": uint8(0x00),
+				"got value":      reserved,
+			}).Msg("Got unexpected response for reserved field.")
+			// We save the value, so it can be re-serialized
+			reservedField0 = &reserved
+		}
+	}
+
+	if closeErr := readBuffer.CloseContext("AmsString"); closeErr != nil {
+		return nil, errors.Wrap(closeErr, "Error closing for AmsString")
+	}
+
+	// Create the instance
+	return &_AmsString{
+		Text:           text,
+		reservedField0: reservedField0,
+	}, nil
+}
+
+func (m *_AmsString) Serialize(writeBuffer utils.WriteBuffer) error {
+	positionAware := writeBuffer
+	_ = positionAware
+	if pushErr := writeBuffer.PushContext("AmsString"); pushErr != nil {
+		return errors.Wrap(pushErr, "Error pushing for AmsString")
+	}
+
+	// Implicit Field (strLen) (Used for parsing, but it's value is not stored as it's implicitly given by the objects content)
+	strLen := uint16(uint16(uint16(len(m.GetText()))) + uint16(uint16(1)))
+	_strLenErr := writeBuffer.WriteUint16("strLen", 16, (strLen))
+	if _strLenErr != nil {
+		return errors.Wrap(_strLenErr, "Error serializing 'strLen' field")
+	}
+
+	// Simple Field (text)
+	text := string(m.GetText())
+	_textErr := writeBuffer.WriteString("text", uint32((8)*((uint16(uint16(len(m.GetText())))+uint16(uint16(1)))-(1))), "UTF-8", (text))
+	if _textErr != nil {
+		return errors.Wrap(_textErr, "Error serializing 'text' field")
+	}
+
+	// Reserved Field (reserved)
+	{
+		var reserved uint8 = uint8(0x00)
+		if m.reservedField0 != nil {
+			Plc4xModelLog.Info().Fields(map[string]interface{}{
+				"expected value": uint8(0x00),
+				"got value":      reserved,
+			}).Msg("Overriding reserved field with unexpected value.")
+			reserved = *m.reservedField0
+		}
+		_err := writeBuffer.WriteUint8("reserved", 8, reserved)
+		if _err != nil {
+			return errors.Wrap(_err, "Error serializing 'reserved' field")
+		}
+	}
+
+	if popErr := writeBuffer.PopContext("AmsString"); popErr != nil {
+		return errors.Wrap(popErr, "Error popping for AmsString")
+	}
+	return nil
+}
+
+func (m *_AmsString) isAmsString() bool {
+	return true
+}
+
+func (m *_AmsString) String() string {
+	if m == nil {
+		return "<nil>"
+	}
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/Operation.go b/plc4go/protocols/ads/discovery/readwrite/model/Operation.go
new file mode 100644
index 000000000..c066b08e9
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/Operation.go
@@ -0,0 +1,176 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Operation is an enum
+type Operation uint32
+
+type IOperation interface {
+	Serialize(writeBuffer utils.WriteBuffer) error
+}
+
+const (
+	Operation_DISCOVERY_REQUEST            Operation = 0x00000001
+	Operation_DISCOVERY_RESPONSE           Operation = 0x80000001
+	Operation_ADD_OR_UPDATE_ROUTE_REQUEST  Operation = 0x00000006
+	Operation_ADD_OR_UPDATE_ROUTE_RESPONSE Operation = 0x80000006
+	Operation_DEL_ROUTE_REQUEST            Operation = 0x00000007
+	Operation_DEL_ROUTE_RESPONSE           Operation = 0x80000007
+	Operation_UNKNOWN_REQUEST              Operation = 0x00000008
+	Operation_UNKNOWN_RESPONSE             Operation = 0x80000008
+)
+
+var OperationValues []Operation
+
+func init() {
+	_ = errors.New
+	OperationValues = []Operation{
+		Operation_DISCOVERY_REQUEST,
+		Operation_DISCOVERY_RESPONSE,
+		Operation_ADD_OR_UPDATE_ROUTE_REQUEST,
+		Operation_ADD_OR_UPDATE_ROUTE_RESPONSE,
+		Operation_DEL_ROUTE_REQUEST,
+		Operation_DEL_ROUTE_RESPONSE,
+		Operation_UNKNOWN_REQUEST,
+		Operation_UNKNOWN_RESPONSE,
+	}
+}
+
+func OperationByValue(value uint32) (enum Operation, ok bool) {
+	switch value {
+	case 0x00000001:
+		return Operation_DISCOVERY_REQUEST, true
+	case 0x00000006:
+		return Operation_ADD_OR_UPDATE_ROUTE_REQUEST, true
+	case 0x00000007:
+		return Operation_DEL_ROUTE_REQUEST, true
+	case 0x00000008:
+		return Operation_UNKNOWN_REQUEST, true
+	case 0x80000001:
+		return Operation_DISCOVERY_RESPONSE, true
+	case 0x80000006:
+		return Operation_ADD_OR_UPDATE_ROUTE_RESPONSE, true
+	case 0x80000007:
+		return Operation_DEL_ROUTE_RESPONSE, true
+	case 0x80000008:
+		return Operation_UNKNOWN_RESPONSE, true
+	}
+	return 0, false
+}
+
+func OperationByName(value string) (enum Operation, ok bool) {
+	switch value {
+	case "DISCOVERY_REQUEST":
+		return Operation_DISCOVERY_REQUEST, true
+	case "ADD_OR_UPDATE_ROUTE_REQUEST":
+		return Operation_ADD_OR_UPDATE_ROUTE_REQUEST, true
+	case "DEL_ROUTE_REQUEST":
+		return Operation_DEL_ROUTE_REQUEST, true
+	case "UNKNOWN_REQUEST":
+		return Operation_UNKNOWN_REQUEST, true
+	case "DISCOVERY_RESPONSE":
+		return Operation_DISCOVERY_RESPONSE, true
+	case "ADD_OR_UPDATE_ROUTE_RESPONSE":
+		return Operation_ADD_OR_UPDATE_ROUTE_RESPONSE, true
+	case "DEL_ROUTE_RESPONSE":
+		return Operation_DEL_ROUTE_RESPONSE, true
+	case "UNKNOWN_RESPONSE":
+		return Operation_UNKNOWN_RESPONSE, true
+	}
+	return 0, false
+}
+
+func OperationKnows(value uint32) bool {
+	for _, typeValue := range OperationValues {
+		if uint32(typeValue) == value {
+			return true
+		}
+	}
+	return false
+}
+
+func CastOperation(structType interface{}) Operation {
+	castFunc := func(typ interface{}) Operation {
+		if sOperation, ok := typ.(Operation); ok {
+			return sOperation
+		}
+		return 0
+	}
+	return castFunc(structType)
+}
+
+func (m Operation) GetLengthInBits() uint16 {
+	return 32
+}
+
+func (m Operation) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func OperationParse(readBuffer utils.ReadBuffer) (Operation, error) {
+	val, err := readBuffer.ReadUint32("Operation", 32)
+	if err != nil {
+		return 0, errors.Wrap(err, "error reading Operation")
+	}
+	if enum, ok := OperationByValue(val); !ok {
+		Plc4xModelLog.Debug().Msgf("no value %x found for RequestType", val)
+		return Operation(val), nil
+	} else {
+		return enum, nil
+	}
+}
+
+func (e Operation) Serialize(writeBuffer utils.WriteBuffer) error {
+	return writeBuffer.WriteUint32("Operation", 32, uint32(e), utils.WithAdditionalStringRepresentation(e.PLC4XEnumName()))
+}
+
+// PLC4XEnumName returns the name that is used in code to identify this enum
+func (e Operation) PLC4XEnumName() string {
+	switch e {
+	case Operation_DISCOVERY_REQUEST:
+		return "DISCOVERY_REQUEST"
+	case Operation_ADD_OR_UPDATE_ROUTE_REQUEST:
+		return "ADD_OR_UPDATE_ROUTE_REQUEST"
+	case Operation_DEL_ROUTE_REQUEST:
+		return "DEL_ROUTE_REQUEST"
+	case Operation_UNKNOWN_REQUEST:
+		return "UNKNOWN_REQUEST"
+	case Operation_DISCOVERY_RESPONSE:
+		return "DISCOVERY_RESPONSE"
+	case Operation_ADD_OR_UPDATE_ROUTE_RESPONSE:
+		return "ADD_OR_UPDATE_ROUTE_RESPONSE"
+	case Operation_DEL_ROUTE_RESPONSE:
+		return "DEL_ROUTE_RESPONSE"
+	case Operation_UNKNOWN_RESPONSE:
+		return "UNKNOWN_RESPONSE"
+	}
+	return ""
+}
+
+func (e Operation) String() string {
+	return e.PLC4XEnumName()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/Status.go b/plc4go/protocols/ads/discovery/readwrite/model/Status.go
new file mode 100644
index 000000000..de4fcf57a
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/Status.go
@@ -0,0 +1,136 @@
+/*
+ * 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 model
+
+import (
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
+)
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Status is an enum
+type Status uint32
+
+type IStatus interface {
+	Serialize(writeBuffer utils.WriteBuffer) error
+}
+
+const (
+	Status_SUCCESS              Status = 0x00000000
+	Status_FAILURE_INVALID_DATA Status = 0x00000704
+	Status_FAILURE_MISSING_DATA Status = 0x00000706
+)
+
+var StatusValues []Status
+
+func init() {
+	_ = errors.New
+	StatusValues = []Status{
+		Status_SUCCESS,
+		Status_FAILURE_INVALID_DATA,
+		Status_FAILURE_MISSING_DATA,
+	}
+}
+
+func StatusByValue(value uint32) (enum Status, ok bool) {
+	switch value {
+	case 0x00000000:
+		return Status_SUCCESS, true
+	case 0x00000704:
+		return Status_FAILURE_INVALID_DATA, true
+	case 0x00000706:
+		return Status_FAILURE_MISSING_DATA, true
+	}
+	return 0, false
+}
+
+func StatusByName(value string) (enum Status, ok bool) {
+	switch value {
+	case "SUCCESS":
+		return Status_SUCCESS, true
+	case "FAILURE_INVALID_DATA":
+		return Status_FAILURE_INVALID_DATA, true
+	case "FAILURE_MISSING_DATA":
+		return Status_FAILURE_MISSING_DATA, true
+	}
+	return 0, false
+}
+
+func StatusKnows(value uint32) bool {
+	for _, typeValue := range StatusValues {
+		if uint32(typeValue) == value {
+			return true
+		}
+	}
+	return false
+}
+
+func CastStatus(structType interface{}) Status {
+	castFunc := func(typ interface{}) Status {
+		if sStatus, ok := typ.(Status); ok {
+			return sStatus
+		}
+		return 0
+	}
+	return castFunc(structType)
+}
+
+func (m Status) GetLengthInBits() uint16 {
+	return 32
+}
+
+func (m Status) GetLengthInBytes() uint16 {
+	return m.GetLengthInBits() / 8
+}
+
+func StatusParse(readBuffer utils.ReadBuffer) (Status, error) {
+	val, err := readBuffer.ReadUint32("Status", 32)
+	if err != nil {
+		return 0, errors.Wrap(err, "error reading Status")
+	}
+	if enum, ok := StatusByValue(val); !ok {
+		Plc4xModelLog.Debug().Msgf("no value %x found for RequestType", val)
+		return Status(val), nil
+	} else {
+		return enum, nil
+	}
+}
+
+func (e Status) Serialize(writeBuffer utils.WriteBuffer) error {
+	return writeBuffer.WriteUint32("Status", 32, uint32(e), utils.WithAdditionalStringRepresentation(e.PLC4XEnumName()))
+}
+
+// PLC4XEnumName returns the name that is used in code to identify this enum
+func (e Status) PLC4XEnumName() string {
+	switch e {
+	case Status_SUCCESS:
+		return "SUCCESS"
+	case Status_FAILURE_INVALID_DATA:
+		return "FAILURE_INVALID_DATA"
+	case Status_FAILURE_MISSING_DATA:
+		return "FAILURE_MISSING_DATA"
+	}
+	return ""
+}
+
+func (e Status) String() string {
+	return e.PLC4XEnumName()
+}
diff --git a/plc4go/protocols/ads/discovery/readwrite/model/plc4x_common.go b/plc4go/protocols/ads/discovery/readwrite/model/plc4x_common.go
new file mode 100644
index 000000000..d2d66e8bd
--- /dev/null
+++ b/plc4go/protocols/ads/discovery/readwrite/model/plc4x_common.go
@@ -0,0 +1,27 @@
+/*
+* 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 model
+
+import "github.com/rs/zerolog/log"
+
+// Code generated by code-generation. DO NOT EDIT.
+
+// Plc4xModelLog is the Logger used by the Parse/Serialize methods
+var Plc4xModelLog = &log.Logger
diff --git a/plc4go/protocols/bacnetip/readwrite/model/BACnetVendorId.go b/plc4go/protocols/bacnetip/readwrite/model/BACnetVendorId.go
index 71f9a52b9..f1ca77183 100644
--- a/plc4go/protocols/bacnetip/readwrite/model/BACnetVendorId.go
+++ b/plc4go/protocols/bacnetip/readwrite/model/BACnetVendorId.go
@@ -1388,6 +1388,35 @@ const (
 	BACnetVendorId_THERMOPLASTIC_ENGINEERING_LTDTPE                                             BACnetVendorId = 1356
 	BACnetVendorId_WIRTH_RESEARCH_LTD                                                           BACnetVendorId = 1357
 	BACnetVendorId_SST_AUTOMATION                                                               BACnetVendorId = 1358
+	BACnetVendorId_SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD                                 BACnetVendorId = 1359
+	BACnetVendorId_AIWAA_SYSTEMS_PRIVATE_LIMITED                                                BACnetVendorId = 1360
+	BACnetVendorId_ENLESS_WIRELESS                                                              BACnetVendorId = 1361
+	BACnetVendorId_OZUNO_ENGINEERING_PTY_LTD                                                    BACnetVendorId = 1362
+	BACnetVendorId_HUBBELL_THE_ELECTRIC_HEATER_COMPANY                                          BACnetVendorId = 1363
+	BACnetVendorId_INDUSTRIAL_TURNAROUND_CORPORATIONITAC                                        BACnetVendorId = 1364
+	BACnetVendorId_WADSWORTH_CONTROL_SYSTEMS                                                    BACnetVendorId = 1365
+	BACnetVendorId_SERVICES_HILO_INC                                                            BACnetVendorId = 1366
+	BACnetVendorId_IDM_ENERGIESYSTEME_GMBH                                                      BACnetVendorId = 1367
+	BACnetVendorId_BE_NEXTBV                                                                    BACnetVendorId = 1368
+	BACnetVendorId_CLEAN_AIRAI_CORPORATION                                                      BACnetVendorId = 1369
+	BACnetVendorId_REVOLUTION_MICROELECTRONICS_AMERICA_INC                                      BACnetVendorId = 1370
+	BACnetVendorId_ARENDARIT_SECURITY_GMBH                                                      BACnetVendorId = 1371
+	BACnetVendorId_ZED_BEE_TECHNOLOGIES_PVT_LTD                                                 BACnetVendorId = 1372
+	BACnetVendorId_WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD                                         BACnetVendorId = 1373
+	BACnetVendorId_SENTICON_LTD                                                                 BACnetVendorId = 1374
+	BACnetVendorId_ROSSAKERAB                                                                   BACnetVendorId = 1375
+	BACnetVendorId_OPIT_SOLUTIONS_LTD                                                           BACnetVendorId = 1376
+	BACnetVendorId_HOTOWELL_INTERNATIONAL_CO_LIMITED                                            BACnetVendorId = 1377
+	BACnetVendorId_INIM_ELECTRONICSSRL_UNIPERSONALE                                             BACnetVendorId = 1378
+	BACnetVendorId_AIRTHINGSASA                                                                 BACnetVendorId = 1379
+	BACnetVendorId_ANALOG_DEVICES_INC                                                           BACnetVendorId = 1380
+	BACnetVendorId_AI_DIRECTIONSDMCC                                                            BACnetVendorId = 1381
+	BACnetVendorId_PRIMA_ELECTRO_SPA                                                            BACnetVendorId = 1382
+	BACnetVendorId_KLT_CONTROL_SYSTEM_LTD                                                       BACnetVendorId = 1383
+	BACnetVendorId_EVOLUTION_CONTROLS_INC                                                       BACnetVendorId = 1384
+	BACnetVendorId_BEVER_INNOVATIONS                                                            BACnetVendorId = 1385
+	BACnetVendorId_PELICAN_WIRELESS_SYSTEMS                                                     BACnetVendorId = 1386
+	BACnetVendorId_CONTROL_CONCEPTS_INC                                                         BACnetVendorId = 1387
 	BACnetVendorId_UNKNOWN_VENDOR                                                               BACnetVendorId = 0xFFFF
 )
 
@@ -2748,6 +2777,35 @@ func init() {
 		BACnetVendorId_THERMOPLASTIC_ENGINEERING_LTDTPE,
 		BACnetVendorId_WIRTH_RESEARCH_LTD,
 		BACnetVendorId_SST_AUTOMATION,
+		BACnetVendorId_SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD,
+		BACnetVendorId_AIWAA_SYSTEMS_PRIVATE_LIMITED,
+		BACnetVendorId_ENLESS_WIRELESS,
+		BACnetVendorId_OZUNO_ENGINEERING_PTY_LTD,
+		BACnetVendorId_HUBBELL_THE_ELECTRIC_HEATER_COMPANY,
+		BACnetVendorId_INDUSTRIAL_TURNAROUND_CORPORATIONITAC,
+		BACnetVendorId_WADSWORTH_CONTROL_SYSTEMS,
+		BACnetVendorId_SERVICES_HILO_INC,
+		BACnetVendorId_IDM_ENERGIESYSTEME_GMBH,
+		BACnetVendorId_BE_NEXTBV,
+		BACnetVendorId_CLEAN_AIRAI_CORPORATION,
+		BACnetVendorId_REVOLUTION_MICROELECTRONICS_AMERICA_INC,
+		BACnetVendorId_ARENDARIT_SECURITY_GMBH,
+		BACnetVendorId_ZED_BEE_TECHNOLOGIES_PVT_LTD,
+		BACnetVendorId_WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD,
+		BACnetVendorId_SENTICON_LTD,
+		BACnetVendorId_ROSSAKERAB,
+		BACnetVendorId_OPIT_SOLUTIONS_LTD,
+		BACnetVendorId_HOTOWELL_INTERNATIONAL_CO_LIMITED,
+		BACnetVendorId_INIM_ELECTRONICSSRL_UNIPERSONALE,
+		BACnetVendorId_AIRTHINGSASA,
+		BACnetVendorId_ANALOG_DEVICES_INC,
+		BACnetVendorId_AI_DIRECTIONSDMCC,
+		BACnetVendorId_PRIMA_ELECTRO_SPA,
+		BACnetVendorId_KLT_CONTROL_SYSTEM_LTD,
+		BACnetVendorId_EVOLUTION_CONTROLS_INC,
+		BACnetVendorId_BEVER_INNOVATIONS,
+		BACnetVendorId_PELICAN_WIRELESS_SYSTEMS,
+		BACnetVendorId_CONTROL_CONCEPTS_INC,
 		BACnetVendorId_UNKNOWN_VENDOR,
 	}
 }
@@ -4358,18 +4416,134 @@ func (e BACnetVendorId) VendorId() uint16 {
 		{ /* '1358' */
 			return 1358
 		}
+	case 1359:
+		{ /* '1359' */
+			return 1359
+		}
 	case 136:
 		{ /* '136' */
 			return 136
 		}
+	case 1360:
+		{ /* '1360' */
+			return 1360
+		}
+	case 1361:
+		{ /* '1361' */
+			return 1361
+		}
+	case 1362:
+		{ /* '1362' */
+			return 1362
+		}
+	case 1363:
+		{ /* '1363' */
+			return 1363
+		}
+	case 1364:
+		{ /* '1364' */
+			return 1364
+		}
+	case 1365:
+		{ /* '1365' */
+			return 1365
+		}
+	case 1366:
+		{ /* '1366' */
+			return 1366
+		}
+	case 1367:
+		{ /* '1367' */
+			return 1367
+		}
+	case 1368:
+		{ /* '1368' */
+			return 1368
+		}
+	case 1369:
+		{ /* '1369' */
+			return 1369
+		}
 	case 137:
 		{ /* '137' */
 			return 137
 		}
+	case 1370:
+		{ /* '1370' */
+			return 1370
+		}
+	case 1371:
+		{ /* '1371' */
+			return 1371
+		}
+	case 1372:
+		{ /* '1372' */
+			return 1372
+		}
+	case 1373:
+		{ /* '1373' */
+			return 1373
+		}
+	case 1374:
+		{ /* '1374' */
+			return 1374
+		}
+	case 1375:
+		{ /* '1375' */
+			return 1375
+		}
+	case 1376:
+		{ /* '1376' */
+			return 1376
+		}
+	case 1377:
+		{ /* '1377' */
+			return 1377
+		}
+	case 1378:
+		{ /* '1378' */
+			return 1378
+		}
+	case 1379:
+		{ /* '1379' */
+			return 1379
+		}
 	case 138:
 		{ /* '138' */
 			return 138
 		}
+	case 1380:
+		{ /* '1380' */
+			return 1380
+		}
+	case 1381:
+		{ /* '1381' */
+			return 1381
+		}
+	case 1382:
+		{ /* '1382' */
+			return 1382
+		}
+	case 1383:
+		{ /* '1383' */
+			return 1383
+		}
+	case 1384:
+		{ /* '1384' */
+			return 1384
+		}
+	case 1385:
+		{ /* '1385' */
+			return 1385
+		}
+	case 1386:
+		{ /* '1386' */
+			return 1386
+		}
+	case 1387:
+		{ /* '1387' */
+			return 1387
+		}
 	case 139:
 		{ /* '139' */
 			return 139
@@ -9788,18 +9962,134 @@ func (e BACnetVendorId) VendorName() string {
 		{ /* '1358' */
 			return "SST Automation"
 		}
+	case 1359:
+		{ /* '1359' */
+			return "Shanghai Bencol Electronic Technology Co., Ltd"
+		}
 	case 136:
 		{ /* '136' */
 			return "Regulvar, Inc."
 		}
+	case 1360:
+		{ /* '1360' */
+			return "AIWAA Systems Private Limited"
+		}
+	case 1361:
+		{ /* '1361' */
+			return "Enless Wireless"
+		}
+	case 1362:
+		{ /* '1362' */
+			return "Ozuno Engineering Pty Ltd"
+		}
+	case 1363:
+		{ /* '1363' */
+			return "Hubbell, The Electric Heater Company"
+		}
+	case 1364:
+		{ /* '1364' */
+			return "Industrial Turnaround Corporation (ITAC)"
+		}
+	case 1365:
+		{ /* '1365' */
+			return "Wadsworth Control Systems"
+		}
+	case 1366:
+		{ /* '1366' */
+			return "Services Hilo Inc."
+		}
+	case 1367:
+		{ /* '1367' */
+			return "iDM Energiesysteme GmbH"
+		}
+	case 1368:
+		{ /* '1368' */
+			return "BeNext B.V."
+		}
+	case 1369:
+		{ /* '1369' */
+			return "CleanAir.ai Corporation"
+		}
 	case 137:
 		{ /* '137' */
 			return "AFDtek Division of Fastek International Inc."
 		}
+	case 1370:
+		{ /* '1370' */
+			return "Revolution Microelectronics (America) Inc."
+		}
+	case 1371:
+		{ /* '1371' */
+			return "Arendar IT-Security GmbH"
+		}
+	case 1372:
+		{ /* '1372' */
+			return "ZedBee Technologies Pvt Ltd"
+		}
+	case 1373:
+		{ /* '1373' */
+			return "Winmate Technology Solutions Pvt. Ltd."
+		}
+	case 1374:
+		{ /* '1374' */
+			return "Senticon Ltd."
+		}
+	case 1375:
+		{ /* '1375' */
+			return "Rossaker AB"
+		}
+	case 1376:
+		{ /* '1376' */
+			return "OPIT Solutions Ltd"
+		}
+	case 1377:
+		{ /* '1377' */
+			return "Hotowell International Co., Limited"
+		}
+	case 1378:
+		{ /* '1378' */
+			return "Inim Electronics S.R.L. Unipersonale"
+		}
+	case 1379:
+		{ /* '1379' */
+			return "Airthings ASA"
+		}
 	case 138:
 		{ /* '138' */
 			return "PowerCold Comfort Air Solutions, Inc."
 		}
+	case 1380:
+		{ /* '1380' */
+			return "Analog Devices, Inc."
+		}
+	case 1381:
+		{ /* '1381' */
+			return "AIDirections DMCC"
+		}
+	case 1382:
+		{ /* '1382' */
+			return "Prima Electro S.p.A."
+		}
+	case 1383:
+		{ /* '1383' */
+			return "KLT Control System Ltd."
+		}
+	case 1384:
+		{ /* '1384' */
+			return "Evolution Controls Inc."
+		}
+	case 1385:
+		{ /* '1385' */
+			return "Bever Innovations"
+		}
+	case 1386:
+		{ /* '1386' */
+			return "Pelican Wireless Systems"
+		}
+	case 1387:
+		{ /* '1387' */
+			return "Control Concepts Inc."
+		}
 	case 139:
 		{ /* '139' */
 			return "I Controls"
@@ -14415,12 +14705,70 @@ func BACnetVendorIdByValue(value uint16) (enum BACnetVendorId, ok bool) {
 		return BACnetVendorId_WIRTH_RESEARCH_LTD, true
 	case 1358:
 		return BACnetVendorId_SST_AUTOMATION, true
+	case 1359:
+		return BACnetVendorId_SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD, true
 	case 136:
 		return BACnetVendorId_REGULVAR_INC, true
+	case 1360:
+		return BACnetVendorId_AIWAA_SYSTEMS_PRIVATE_LIMITED, true
+	case 1361:
+		return BACnetVendorId_ENLESS_WIRELESS, true
+	case 1362:
+		return BACnetVendorId_OZUNO_ENGINEERING_PTY_LTD, true
+	case 1363:
+		return BACnetVendorId_HUBBELL_THE_ELECTRIC_HEATER_COMPANY, true
+	case 1364:
+		return BACnetVendorId_INDUSTRIAL_TURNAROUND_CORPORATIONITAC, true
+	case 1365:
+		return BACnetVendorId_WADSWORTH_CONTROL_SYSTEMS, true
+	case 1366:
+		return BACnetVendorId_SERVICES_HILO_INC, true
+	case 1367:
+		return BACnetVendorId_IDM_ENERGIESYSTEME_GMBH, true
+	case 1368:
+		return BACnetVendorId_BE_NEXTBV, true
+	case 1369:
+		return BACnetVendorId_CLEAN_AIRAI_CORPORATION, true
 	case 137:
 		return BACnetVendorId_AF_DTEK_DIVISIONOF_FASTEK_INTERNATIONAL_INC, true
+	case 1370:
+		return BACnetVendorId_REVOLUTION_MICROELECTRONICS_AMERICA_INC, true
+	case 1371:
+		return BACnetVendorId_ARENDARIT_SECURITY_GMBH, true
+	case 1372:
+		return BACnetVendorId_ZED_BEE_TECHNOLOGIES_PVT_LTD, true
+	case 1373:
+		return BACnetVendorId_WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD, true
+	case 1374:
+		return BACnetVendorId_SENTICON_LTD, true
+	case 1375:
+		return BACnetVendorId_ROSSAKERAB, true
+	case 1376:
+		return BACnetVendorId_OPIT_SOLUTIONS_LTD, true
+	case 1377:
+		return BACnetVendorId_HOTOWELL_INTERNATIONAL_CO_LIMITED, true
+	case 1378:
+		return BACnetVendorId_INIM_ELECTRONICSSRL_UNIPERSONALE, true
+	case 1379:
+		return BACnetVendorId_AIRTHINGSASA, true
 	case 138:
 		return BACnetVendorId_POWER_COLD_COMFORT_AIR_SOLUTIONS_INC, true
+	case 1380:
+		return BACnetVendorId_ANALOG_DEVICES_INC, true
+	case 1381:
+		return BACnetVendorId_AI_DIRECTIONSDMCC, true
+	case 1382:
+		return BACnetVendorId_PRIMA_ELECTRO_SPA, true
+	case 1383:
+		return BACnetVendorId_KLT_CONTROL_SYSTEM_LTD, true
+	case 1384:
+		return BACnetVendorId_EVOLUTION_CONTROLS_INC, true
+	case 1385:
+		return BACnetVendorId_BEVER_INNOVATIONS, true
+	case 1386:
+		return BACnetVendorId_PELICAN_WIRELESS_SYSTEMS, true
+	case 1387:
+		return BACnetVendorId_CONTROL_CONCEPTS_INC, true
 	case 139:
 		return BACnetVendorId_I_CONTROLS, true
 	case 14:
@@ -17127,12 +17475,70 @@ func BACnetVendorIdByName(value string) (enum BACnetVendorId, ok bool) {
 		return BACnetVendorId_WIRTH_RESEARCH_LTD, true
 	case "SST_AUTOMATION":
 		return BACnetVendorId_SST_AUTOMATION, true
+	case "SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD":
+		return BACnetVendorId_SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD, true
 	case "REGULVAR_INC":
 		return BACnetVendorId_REGULVAR_INC, true
+	case "AIWAA_SYSTEMS_PRIVATE_LIMITED":
+		return BACnetVendorId_AIWAA_SYSTEMS_PRIVATE_LIMITED, true
+	case "ENLESS_WIRELESS":
+		return BACnetVendorId_ENLESS_WIRELESS, true
+	case "OZUNO_ENGINEERING_PTY_LTD":
+		return BACnetVendorId_OZUNO_ENGINEERING_PTY_LTD, true
+	case "HUBBELL_THE_ELECTRIC_HEATER_COMPANY":
+		return BACnetVendorId_HUBBELL_THE_ELECTRIC_HEATER_COMPANY, true
+	case "INDUSTRIAL_TURNAROUND_CORPORATIONITAC":
+		return BACnetVendorId_INDUSTRIAL_TURNAROUND_CORPORATIONITAC, true
+	case "WADSWORTH_CONTROL_SYSTEMS":
+		return BACnetVendorId_WADSWORTH_CONTROL_SYSTEMS, true
+	case "SERVICES_HILO_INC":
+		return BACnetVendorId_SERVICES_HILO_INC, true
+	case "IDM_ENERGIESYSTEME_GMBH":
+		return BACnetVendorId_IDM_ENERGIESYSTEME_GMBH, true
+	case "BE_NEXTBV":
+		return BACnetVendorId_BE_NEXTBV, true
+	case "CLEAN_AIRAI_CORPORATION":
+		return BACnetVendorId_CLEAN_AIRAI_CORPORATION, true
 	case "AF_DTEK_DIVISIONOF_FASTEK_INTERNATIONAL_INC":
 		return BACnetVendorId_AF_DTEK_DIVISIONOF_FASTEK_INTERNATIONAL_INC, true
+	case "REVOLUTION_MICROELECTRONICS_AMERICA_INC":
+		return BACnetVendorId_REVOLUTION_MICROELECTRONICS_AMERICA_INC, true
+	case "ARENDARIT_SECURITY_GMBH":
+		return BACnetVendorId_ARENDARIT_SECURITY_GMBH, true
+	case "ZED_BEE_TECHNOLOGIES_PVT_LTD":
+		return BACnetVendorId_ZED_BEE_TECHNOLOGIES_PVT_LTD, true
+	case "WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD":
+		return BACnetVendorId_WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD, true
+	case "SENTICON_LTD":
+		return BACnetVendorId_SENTICON_LTD, true
+	case "ROSSAKERAB":
+		return BACnetVendorId_ROSSAKERAB, true
+	case "OPIT_SOLUTIONS_LTD":
+		return BACnetVendorId_OPIT_SOLUTIONS_LTD, true
+	case "HOTOWELL_INTERNATIONAL_CO_LIMITED":
+		return BACnetVendorId_HOTOWELL_INTERNATIONAL_CO_LIMITED, true
+	case "INIM_ELECTRONICSSRL_UNIPERSONALE":
+		return BACnetVendorId_INIM_ELECTRONICSSRL_UNIPERSONALE, true
+	case "AIRTHINGSASA":
+		return BACnetVendorId_AIRTHINGSASA, true
 	case "POWER_COLD_COMFORT_AIR_SOLUTIONS_INC":
 		return BACnetVendorId_POWER_COLD_COMFORT_AIR_SOLUTIONS_INC, true
+	case "ANALOG_DEVICES_INC":
+		return BACnetVendorId_ANALOG_DEVICES_INC, true
+	case "AI_DIRECTIONSDMCC":
+		return BACnetVendorId_AI_DIRECTIONSDMCC, true
+	case "PRIMA_ELECTRO_SPA":
+		return BACnetVendorId_PRIMA_ELECTRO_SPA, true
+	case "KLT_CONTROL_SYSTEM_LTD":
+		return BACnetVendorId_KLT_CONTROL_SYSTEM_LTD, true
+	case "EVOLUTION_CONTROLS_INC":
+		return BACnetVendorId_EVOLUTION_CONTROLS_INC, true
+	case "BEVER_INNOVATIONS":
+		return BACnetVendorId_BEVER_INNOVATIONS, true
+	case "PELICAN_WIRELESS_SYSTEMS":
+		return BACnetVendorId_PELICAN_WIRELESS_SYSTEMS, true
+	case "CONTROL_CONCEPTS_INC":
+		return BACnetVendorId_CONTROL_CONCEPTS_INC, true
 	case "I_CONTROLS":
 		return BACnetVendorId_I_CONTROLS, true
 	case "CIMETRICS_TECHNOLOGY":
@@ -19884,12 +20290,70 @@ func (e BACnetVendorId) PLC4XEnumName() string {
 		return "WIRTH_RESEARCH_LTD"
 	case BACnetVendorId_SST_AUTOMATION:
 		return "SST_AUTOMATION"
+	case BACnetVendorId_SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD:
+		return "SHANGHAI_BENCOL_ELECTRONIC_TECHNOLOGY_CO_LTD"
 	case BACnetVendorId_REGULVAR_INC:
 		return "REGULVAR_INC"
+	case BACnetVendorId_AIWAA_SYSTEMS_PRIVATE_LIMITED:
+		return "AIWAA_SYSTEMS_PRIVATE_LIMITED"
+	case BACnetVendorId_ENLESS_WIRELESS:
+		return "ENLESS_WIRELESS"
+	case BACnetVendorId_OZUNO_ENGINEERING_PTY_LTD:
+		return "OZUNO_ENGINEERING_PTY_LTD"
+	case BACnetVendorId_HUBBELL_THE_ELECTRIC_HEATER_COMPANY:
+		return "HUBBELL_THE_ELECTRIC_HEATER_COMPANY"
+	case BACnetVendorId_INDUSTRIAL_TURNAROUND_CORPORATIONITAC:
+		return "INDUSTRIAL_TURNAROUND_CORPORATIONITAC"
+	case BACnetVendorId_WADSWORTH_CONTROL_SYSTEMS:
+		return "WADSWORTH_CONTROL_SYSTEMS"
+	case BACnetVendorId_SERVICES_HILO_INC:
+		return "SERVICES_HILO_INC"
+	case BACnetVendorId_IDM_ENERGIESYSTEME_GMBH:
+		return "IDM_ENERGIESYSTEME_GMBH"
+	case BACnetVendorId_BE_NEXTBV:
+		return "BE_NEXTBV"
+	case BACnetVendorId_CLEAN_AIRAI_CORPORATION:
+		return "CLEAN_AIRAI_CORPORATION"
 	case BACnetVendorId_AF_DTEK_DIVISIONOF_FASTEK_INTERNATIONAL_INC:
 		return "AF_DTEK_DIVISIONOF_FASTEK_INTERNATIONAL_INC"
+	case BACnetVendorId_REVOLUTION_MICROELECTRONICS_AMERICA_INC:
+		return "REVOLUTION_MICROELECTRONICS_AMERICA_INC"
+	case BACnetVendorId_ARENDARIT_SECURITY_GMBH:
+		return "ARENDARIT_SECURITY_GMBH"
+	case BACnetVendorId_ZED_BEE_TECHNOLOGIES_PVT_LTD:
+		return "ZED_BEE_TECHNOLOGIES_PVT_LTD"
+	case BACnetVendorId_WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD:
+		return "WINMATE_TECHNOLOGY_SOLUTIONS_PVT_LTD"
+	case BACnetVendorId_SENTICON_LTD:
+		return "SENTICON_LTD"
+	case BACnetVendorId_ROSSAKERAB:
+		return "ROSSAKERAB"
+	case BACnetVendorId_OPIT_SOLUTIONS_LTD:
+		return "OPIT_SOLUTIONS_LTD"
+	case BACnetVendorId_HOTOWELL_INTERNATIONAL_CO_LIMITED:
+		return "HOTOWELL_INTERNATIONAL_CO_LIMITED"
+	case BACnetVendorId_INIM_ELECTRONICSSRL_UNIPERSONALE:
+		return "INIM_ELECTRONICSSRL_UNIPERSONALE"
+	case BACnetVendorId_AIRTHINGSASA:
+		return "AIRTHINGSASA"
 	case BACnetVendorId_POWER_COLD_COMFORT_AIR_SOLUTIONS_INC:
 		return "POWER_COLD_COMFORT_AIR_SOLUTIONS_INC"
+	case BACnetVendorId_ANALOG_DEVICES_INC:
+		return "ANALOG_DEVICES_INC"
+	case BACnetVendorId_AI_DIRECTIONSDMCC:
+		return "AI_DIRECTIONSDMCC"
+	case BACnetVendorId_PRIMA_ELECTRO_SPA:
+		return "PRIMA_ELECTRO_SPA"
+	case BACnetVendorId_KLT_CONTROL_SYSTEM_LTD:
+		return "KLT_CONTROL_SYSTEM_LTD"
+	case BACnetVendorId_EVOLUTION_CONTROLS_INC:
+		return "EVOLUTION_CONTROLS_INC"
+	case BACnetVendorId_BEVER_INNOVATIONS:
+		return "BEVER_INNOVATIONS"
+	case BACnetVendorId_PELICAN_WIRELESS_SYSTEMS:
+		return "PELICAN_WIRELESS_SYSTEMS"
+	case BACnetVendorId_CONTROL_CONCEPTS_INC:
+		return "CONTROL_CONCEPTS_INC"
 	case BACnetVendorId_I_CONTROLS:
 		return "I_CONTROLS"
 	case BACnetVendorId_CIMETRICS_TECHNOLOGY:
diff --git a/plc4go/protocols/knxnetip/readwrite/model/KnxManufacturer.go b/plc4go/protocols/knxnetip/readwrite/model/KnxManufacturer.go
index 1d5e3591a..38a702233 100644
--- a/plc4go/protocols/knxnetip/readwrite/model/KnxManufacturer.go
+++ b/plc4go/protocols/knxnetip/readwrite/model/KnxManufacturer.go
@@ -637,8 +637,11 @@ const (
 	KnxManufacturer_M_RAMIREZ_ENGINEERING_GMBH                           KnxManufacturer = 598
 	KnxManufacturer_M_ZHONGSHAN_TAIYANG_IMPANDEXP__CO_LTD                KnxManufacturer = 599
 	KnxManufacturer_M_VIHAN_ELECTRIC_PVT_LTD                             KnxManufacturer = 600
-	KnxManufacturer_M_ABB___RESERVED                                     KnxManufacturer = 601
-	KnxManufacturer_M_BUSCH_JAEGER_ELEKTRO___RESERVED                    KnxManufacturer = 602
+	KnxManufacturer_M_SPLENDID_MINDS_GMBH                                KnxManufacturer = 601
+	KnxManufacturer_M_ESTADA                                             KnxManufacturer = 602
+	KnxManufacturer_M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI            KnxManufacturer = 603
+	KnxManufacturer_M_ABB___RESERVED                                     KnxManufacturer = 604
+	KnxManufacturer_M_BUSCH_JAEGER_ELEKTRO___RESERVED                    KnxManufacturer = 605
 )
 
 var KnxManufacturerValues []KnxManufacturer
@@ -1247,6 +1250,9 @@ func init() {
 		KnxManufacturer_M_RAMIREZ_ENGINEERING_GMBH,
 		KnxManufacturer_M_ZHONGSHAN_TAIYANG_IMPANDEXP__CO_LTD,
 		KnxManufacturer_M_VIHAN_ELECTRIC_PVT_LTD,
+		KnxManufacturer_M_SPLENDID_MINDS_GMBH,
+		KnxManufacturer_M_ESTADA,
+		KnxManufacturer_M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI,
 		KnxManufacturer_M_ABB___RESERVED,
 		KnxManufacturer_M_BUSCH_JAEGER_ELEKTRO___RESERVED,
 	}
@@ -3492,10 +3498,22 @@ func (e KnxManufacturer) Number() uint16 {
 		}
 	case 601:
 		{ /* '601' */
-			return 43954
+			return 659
 		}
 	case 602:
 		{ /* '602' */
+			return 660
+		}
+	case 603:
+		{ /* '603' */
+			return 661
+		}
+	case 604:
+		{ /* '604' */
+			return 43954
+		}
+	case 605:
+		{ /* '605' */
 			return 43959
 		}
 	case 61:
@@ -5922,10 +5940,22 @@ func (e KnxManufacturer) Name() string {
 		}
 	case 601:
 		{ /* '601' */
-			return "ABB - reserved"
+			return "Splendid Minds GmbH"
 		}
 	case 602:
 		{ /* '602' */
+			return "Estada"
+		}
+	case 603:
+		{ /* '603' */
+			return "zhongyunxinzhikonggujituanyouxiangongsi"
+		}
+	case 604:
+		{ /* '604' */
+			return "ABB - reserved"
+		}
+	case 605:
+		{ /* '605' */
 			return "Busch-Jaeger Elektro - reserved"
 		}
 	case 61:
@@ -7232,8 +7262,14 @@ func KnxManufacturerByValue(value uint16) (enum KnxManufacturer, ok bool) {
 	case 600:
 		return KnxManufacturer_M_VIHAN_ELECTRIC_PVT_LTD, true
 	case 601:
-		return KnxManufacturer_M_ABB___RESERVED, true
+		return KnxManufacturer_M_SPLENDID_MINDS_GMBH, true
 	case 602:
+		return KnxManufacturer_M_ESTADA, true
+	case 603:
+		return KnxManufacturer_M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI, true
+	case 604:
+		return KnxManufacturer_M_ABB___RESERVED, true
+	case 605:
 		return KnxManufacturer_M_BUSCH_JAEGER_ELEKTRO___RESERVED, true
 	case 61:
 		return KnxManufacturer_M_SCHNEIDER_ELECTRIC_INDUSTRIES_SAS, true
@@ -8443,6 +8479,12 @@ func KnxManufacturerByName(value string) (enum KnxManufacturer, ok bool) {
 		return KnxManufacturer_M_TECHEM, true
 	case "M_VIHAN_ELECTRIC_PVT_LTD":
 		return KnxManufacturer_M_VIHAN_ELECTRIC_PVT_LTD, true
+	case "M_SPLENDID_MINDS_GMBH":
+		return KnxManufacturer_M_SPLENDID_MINDS_GMBH, true
+	case "M_ESTADA":
+		return KnxManufacturer_M_ESTADA, true
+	case "M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI":
+		return KnxManufacturer_M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI, true
 	case "M_ABB___RESERVED":
 		return KnxManufacturer_M_ABB___RESERVED, true
 	case "M_BUSCH_JAEGER_ELEKTRO___RESERVED":
@@ -9700,6 +9742,12 @@ func (e KnxManufacturer) PLC4XEnumName() string {
 		return "M_TECHEM"
 	case KnxManufacturer_M_VIHAN_ELECTRIC_PVT_LTD:
 		return "M_VIHAN_ELECTRIC_PVT_LTD"
+	case KnxManufacturer_M_SPLENDID_MINDS_GMBH:
+		return "M_SPLENDID_MINDS_GMBH"
+	case KnxManufacturer_M_ESTADA:
+		return "M_ESTADA"
+	case KnxManufacturer_M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI:
+		return "M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI"
 	case KnxManufacturer_M_ABB___RESERVED:
 		return "M_ABB___RESERVED"
 	case KnxManufacturer_M_BUSCH_JAEGER_ELEKTRO___RESERVED:
diff --git a/plc4go/spi/model/DefaultPlcWriteRequest.go b/plc4go/spi/model/DefaultPlcWriteRequest.go
index 6dbcfbf6f..66f551816 100644
--- a/plc4go/spi/model/DefaultPlcWriteRequest.go
+++ b/plc4go/spi/model/DefaultPlcWriteRequest.go
@@ -21,12 +21,13 @@ package model
 
 import (
 	"context"
+	"time"
+
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
 	"github.com/apache/plc4x/plc4go/pkg/api/values"
 	"github.com/apache/plc4x/plc4go/spi"
 	"github.com/apache/plc4x/plc4go/spi/interceptors"
 	"github.com/pkg/errors"
-	"time"
 )
 
 //go:generate go run ../../tools/plc4xgenerator/gen.go -type=DefaultPlcWriteRequestBuilder
@@ -107,7 +108,7 @@ func (m *DefaultPlcWriteRequestBuilder) Build() (model.PlcWriteRequest, error) {
 	for name, field := range m.fields {
 		value, err := m.valueHandler.NewPlcValue(field, m.values[name])
 		if err != nil {
-			return nil, errors.Wrapf(err, "Error parsing value of type: %s", field.GetTypeName())
+			//			return nil, errors.Wrapf(err, "Error parsing value of type: %s", field.GetTypeName())
 		}
 		plcValues[name] = value
 	}
diff --git a/plc4go/spi/testutils/steptype_string.go b/plc4go/spi/testutils/steptype_string.go
index c99ce3e0f..8506728bb 100644
--- a/plc4go/spi/testutils/steptype_string.go
+++ b/plc4go/spi/testutils/steptype_string.go
@@ -1,20 +1,3 @@
-// Licensed to 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. Apache Software Foundation (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.
-
 // Code generated by "stringer -type StepType"; DO NOT EDIT.
 
 package testutils
diff --git a/plc4go/spi/values/IEC61131ValueHandler.go b/plc4go/spi/values/IEC61131ValueHandler.go
index c6da3aa47..d1ade0437 100644
--- a/plc4go/spi/values/IEC61131ValueHandler.go
+++ b/plc4go/spi/values/IEC61131ValueHandler.go
@@ -21,11 +21,12 @@ package values
 
 import (
 	"errors"
-	"github.com/apache/plc4x/plc4go/pkg/api/model"
-	"github.com/apache/plc4x/plc4go/pkg/api/values"
 	"reflect"
 	"strconv"
 	"time"
+
+	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/pkg/api/values"
 )
 
 const (
@@ -87,34 +88,36 @@ type IEC61131ValueHandler struct {
 }
 
 func (m IEC61131ValueHandler) NewPlcValue(field model.PlcField, value interface{}) (values.PlcValue, error) {
-	typeName := field.GetTypeName()
-	quantity := field.GetQuantity()
-	if quantity > 1 {
-		s := reflect.ValueOf(value)
-		if s.Kind() != reflect.Slice {
-			return nil, errors.New("couldn't cast value to []interface{}")
-		}
-		curValues := make([]interface{}, s.Len())
-		for i := 0; i < s.Len(); i++ {
-			curValues[i] = s.Index(i).Interface()
-		}
-
-		if len(curValues) != int(quantity) {
-			return nil, errors.New("number of actual values " + strconv.Itoa(len(curValues)) +
-				" doesn't match field size " + strconv.Itoa(int(quantity)))
-		}
-		var plcValues []values.PlcValue
-		for i := uint16(0); i < quantity; i++ {
-			curValue := curValues[i]
-			plcValue, err := m.NewPlcValueFromType(typeName, curValue)
-			if err != nil {
-				return nil, errors.New("error parsing PlcValue: " + err.Error())
+	/*	typeName := field.GetTypeName()
+		quantity := field.GetQuantity()
+		if quantity > 1 {
+			s := reflect.ValueOf(value)
+			if s.Kind() != reflect.Slice {
+				return nil, errors.New("couldn't cast value to []interface{}")
 			}
-			plcValues = append(plcValues, plcValue)
+			curValues := make([]interface{}, s.Len())
+			for i := 0; i < s.Len(); i++ {
+				curValues[i] = s.Index(i).Interface()
+			}
+
+			if len(curValues) != int(quantity) {
+				return nil, errors.New("number of actual values " + strconv.Itoa(len(curValues)) +
+					" doesn't match field size " + strconv.Itoa(int(quantity)))
+			}
+			var plcValues []values.PlcValue
+			for i := uint16(0); i < quantity; i++ {
+				curValue := curValues[i]
+				plcValue, err := m.NewPlcValueFromType(typeName, curValue)
+				if err != nil {
+					return nil, errors.New("error parsing PlcValue: " + err.Error())
+				}
+				plcValues = append(plcValues, plcValue)
+			}
+			return NewPlcList(plcValues), nil
 		}
-		return NewPlcList(plcValues), nil
-	}
-	return m.NewPlcValueFromType(typeName, value)
+		return m.NewPlcValueFromType(typeName, value)
+	*/
+	return nil, nil
 }
 
 func (m IEC61131ValueHandler) NewPlcValueFromType(typeName string, value interface{}) (values.PlcValue, error) {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/transaction/RequestTransactionManager.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/transaction/RequestTransactionManager.java
index bdb307d13..4b42caa33 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/transaction/RequestTransactionManager.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/transaction/RequestTransactionManager.java
@@ -48,7 +48,7 @@ public class RequestTransactionManager {
     private static final Logger logger = LoggerFactory.getLogger(RequestTransactionManager.class);
 
     /** Executor that performs all operations */
-    static final ExecutorService executor = Executors.newFixedThreadPool(4);
+    static final ExecutorService executor = Executors.newScheduledThreadPool(4);
     private final Set<RequestTransaction> runningRequests;
     /** How many Transactions are allowed to run at the same time? */
     private int numberOfConcurrentRequests;
@@ -94,7 +94,7 @@ public class RequestTransactionManager {
         assert handle.operation != null;
         // Add this Request with this handle i the Worklog
         // Put Transaction into Worklog
-        this.workLog.add(handle);
+        workLog.add(handle);
         // Try to Process the Worklog
         processWorklog();
     }
@@ -103,7 +103,7 @@ public class RequestTransactionManager {
         while (runningRequests.size() < getNumberOfConcurrentRequests() && !workLog.isEmpty()) {
             RequestTransaction next = workLog.poll();
             if (next != null) {
-                this.runningRequests.add(next);
+                runningRequests.add(next);
                 Future<?> completionFuture = executor.submit(next.operation);
                 next.setCompletionFuture(completionFuture);
             }
@@ -112,11 +112,11 @@ public class RequestTransactionManager {
 
 
     public RequestTransaction startRequest() {
-        return new RequestTransaction(this, this.transactionId.getAndIncrement());
+        return new RequestTransaction(this, transactionId.getAndIncrement());
     }
 
     public int getNumberOfActiveRequests() {
-        return this.runningRequests.size();
+        return runningRequests.size();
     }
 
     private void failRequest(RequestTransaction transaction) {
@@ -127,10 +127,10 @@ public class RequestTransactionManager {
     }
 
     private void endRequest(RequestTransaction transaction) {
-        if (!this.runningRequests.contains(transaction)) {
+        if (!runningRequests.contains(transaction)) {
             throw new IllegalArgumentException("Unknown Transaction or Transaction already finished!");
         }
-        this.runningRequests.remove(transaction);
+        runningRequests.remove(transaction);
         // Process the worklog, a slot should be free now
         processWorklog();
     }
@@ -154,12 +154,12 @@ public class RequestTransactionManager {
         }
 
         public void failRequest(Throwable t) {
-            this.parent.failRequest(this);
+            parent.failRequest(this);
         }
 
         public void endRequest() {
             // Remove it from Running Requests
-            this.parent.endRequest(this);
+            parent.endRequest(this);
         }
 
         public void setOperation(Runnable operation) {
@@ -176,8 +176,8 @@ public class RequestTransactionManager {
 
         public void submit(Runnable operation) {
             logger.trace("Submission of transaction {}", transactionId);
-            this.setOperation(new TransactionOperation(transactionId, operation));
-            this.parent.submit(this);
+            setOperation(new TransactionOperation(transactionId, operation));
+            parent.submit(this);
         }
 
         @Override
diff --git a/plc4j/tools/pom.xml b/plc4j/tools/pom.xml
index 94240bada..c3fbe156a 100644
--- a/plc4j/tools/pom.xml
+++ b/plc4j/tools/pom.xml
@@ -35,8 +35,8 @@
 
   <modules>
     <module>capture-replay</module>
-    <module>connection-pool</module>
     <module>connection-cache</module>
+    <module>connection-pool</module>
     <module>opm</module>
     <module>plc4x-server</module>
     <module>scraper</module>
diff --git a/plc4net/drivers/knxnetip/src/drivers/knxnetip/readwrite/model/KnxManufacturer.cs b/plc4net/drivers/knxnetip/src/drivers/knxnetip/readwrite/model/KnxManufacturer.cs
index 59306feb1..1893caca6 100644
--- a/plc4net/drivers/knxnetip/src/drivers/knxnetip/readwrite/model/KnxManufacturer.cs
+++ b/plc4net/drivers/knxnetip/src/drivers/knxnetip/readwrite/model/KnxManufacturer.cs
@@ -624,8 +624,12 @@ namespace org.apache.plc4net.drivers.knxnetip.readwrite.model
         M_PHOENIX_CONTACT_2 = 597,
         M_RAMIREZ_ENGINEERING_GMBH = 598,
         M_ZHONGSHAN_TAIYANG_IMPANDEXP__CO_LTD = 599,
-        M_ABB___RESERVED = 600,
-        M_BUSCH_JAEGER_ELEKTRO___RESERVED = 601,
+        M_VIHAN_ELECTRIC_PVT_LTD = 600,
+        M_SPLENDID_MINDS_GMBH = 601,
+        M_ESTADA = 602,
+        M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI = 603,
+        M_ABB___RESERVED = 604,
+        M_BUSCH_JAEGER_ELEKTRO___RESERVED = 605,
     }
 
     public static class KnxManufacturerInfo
@@ -2309,10 +2313,22 @@ namespace org.apache.plc4net.drivers.knxnetip.readwrite.model
                 case KnxManufacturer.M_TECHEM: { /* '60' */
                     return 99;
                 }
-                case KnxManufacturer.M_ABB___RESERVED: { /* '600' */
+                case KnxManufacturer.M_VIHAN_ELECTRIC_PVT_LTD: { /* '600' */
+                    return 658;
+                }
+                case KnxManufacturer.M_SPLENDID_MINDS_GMBH: { /* '601' */
+                    return 659;
+                }
+                case KnxManufacturer.M_ESTADA: { /* '602' */
+                    return 660;
+                }
+                case KnxManufacturer.M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI: { /* '603' */
+                    return 661;
+                }
+                case KnxManufacturer.M_ABB___RESERVED: { /* '604' */
                     return 43954;
                 }
-                case KnxManufacturer.M_BUSCH_JAEGER_ELEKTRO___RESERVED: { /* '601' */
+                case KnxManufacturer.M_BUSCH_JAEGER_ELEKTRO___RESERVED: { /* '605' */
                     return 43959;
                 }
                 case KnxManufacturer.M_SCHNEIDER_ELECTRIC_INDUSTRIES_SAS: { /* '61' */
@@ -4125,10 +4141,22 @@ namespace org.apache.plc4net.drivers.knxnetip.readwrite.model
                 case KnxManufacturer.M_TECHEM: { /* '60' */
                     return "Techem";
                 }
-                case KnxManufacturer.M_ABB___RESERVED: { /* '600' */
+                case KnxManufacturer.M_VIHAN_ELECTRIC_PVT_LTD: { /* '600' */
+                    return "Vihan electric pvt ltd";
+                }
+                case KnxManufacturer.M_SPLENDID_MINDS_GMBH: { /* '601' */
+                    return "Splendid Minds GmbH";
+                }
+                case KnxManufacturer.M_ESTADA: { /* '602' */
+                    return "Estada";
+                }
+                case KnxManufacturer.M_ZHONGYUNXINZHIKONGGUJITUANYOUXIANGONGSI: { /* '603' */
+                    return "zhongyunxinzhikonggujituanyouxiangongsi";
+                }
+                case KnxManufacturer.M_ABB___RESERVED: { /* '604' */
                     return "ABB - reserved";
                 }
-                case KnxManufacturer.M_BUSCH_JAEGER_ELEKTRO___RESERVED: { /* '601' */
+                case KnxManufacturer.M_BUSCH_JAEGER_ELEKTRO___RESERVED: { /* '605' */
                     return "Busch-Jaeger Elektro - reserved";
                 }
                 case KnxManufacturer.M_SCHNEIDER_ELECTRIC_INDUSTRIES_SAS: { /* '61' */