You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by hu...@apache.org on 2021/08/17 20:47:45 UTC

[plc4x] branch feature/modbusrtu created (now 27f6a9a)

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

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


      at 27f6a9a  Checked modbus-rtu package, updated mspec

This branch includes the following new commits:

     new c1a8bbd  Initial Modbus RTU Support.
     new 27f6a9a  Checked modbus-rtu package, updated mspec

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


[plc4x] 02/02: Checked modbus-rtu package, updated mspec

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

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

commit 27f6a9aa2e996ed263abf822e41dc2e7dfb01e46
Author: hutcheb <hu...@apache.org>
AuthorDate: Wed Aug 18 06:46:39 2021 +1000

    Checked modbus-rtu package, updated mspec
---
 plc4j/drivers/modbus/pom.xml                       | 16 ++++
 .../plc4x/java/modbus_rtu/ModbusRTUDriver.java     |  6 +-
 .../protocol/ModbusRTUProtocolLogic.java           |  6 +-
 .../plc4x/java/modbus_rtu/utils/StaticHelper.java  | 86 ++++++++++++++++++++++
 .../services/org.apache.plc4x.java.api.PlcDriver   |  1 +
 .../main/resources/protocols/modbus/modbus.mspec   |  3 +-
 6 files changed, 112 insertions(+), 6 deletions(-)

diff --git a/plc4j/drivers/modbus/pom.xml b/plc4j/drivers/modbus/pom.xml
index 57e007f..bab886e 100644
--- a/plc4j/drivers/modbus/pom.xml
+++ b/plc4j/drivers/modbus/pom.xml
@@ -98,6 +98,22 @@
         </configuration>
       </plugin>
       <plugin>
+        <groupId>org.apache.felix</groupId>
+        <artifactId>maven-bundle-plugin</artifactId>
+        <extensions>true</extensions>
+        <configuration>
+          <instructions>
+            <Bundle-SymbolicName>${project.groupId}.${project.artifactId}</Bundle-SymbolicName>
+            <Bundle-Activator>org.apache.plc4x.java.osgi.DriverActivator</Bundle-Activator>
+            <Export-Service>org.apache.plc4x.java.api.PlcDriver,org.apache.plc4x.java.modbus_rtu.ModbusRTUDriver</Export-Service>
+            <Import-Package>
+              com.fasterxml.jackson.annotation;resolution:=optional,
+              *
+            </Import-Package>
+          </instructions>
+        </configuration>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-dependency-plugin</artifactId>
         <configuration>
diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java
index a8ede08..16ba73f 100644
--- a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java
@@ -48,7 +48,7 @@ public class ModbusRTUDriver extends GeneratedDriverBase<ModbusSerialADU> {
 
     @Override
     public String getProtocolName() {
-        return "ModbusRTU";
+        return "Modbus RTU";
     }
 
     @Override
@@ -118,8 +118,8 @@ public class ModbusRTUDriver extends GeneratedDriverBase<ModbusSerialADU> {
     public static class ByteLengthEstimator implements ToIntFunction<ByteBuf> {
         @Override
         public int applyAsInt(ByteBuf byteBuf) {
-            if (byteBuf.readableBytes() >= 6) {
-                return byteBuf.getUnsignedShort(byteBuf.readerIndex() + 4) + 6;
+            if (byteBuf.readableBytes() >= 3) {
+                return byteBuf.getUnsignedByte(byteBuf.readerIndex() + 2) + 5;
             }
             return -1;
         }
diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java
index 204441c..457c699 100644
--- a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java
@@ -54,6 +54,8 @@ import java.util.*;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import static org.apache.plc4x.java.modbus_rtu.utils.StaticHelper.calculateChecksum;
+
 public class ModbusRTUProtocolLogic extends Plc4xProtocolBase<ModbusSerialADU> implements HasConfiguration<ModbusRTUConfiguration> {
 
     private Duration requestTimeout;
@@ -93,7 +95,7 @@ public class ModbusRTUProtocolLogic extends Plc4xProtocolBase<ModbusSerialADU> i
             String fieldName = request.getFieldNames().iterator().next();
             ModbusField field = (ModbusField) request.getField(fieldName);
             final ModbusPDU requestPdu = getReadRequestPdu(field);
-            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu);
+            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu, calculateChecksum(address, requestPdu));
 
             RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
             transaction.submit(() -> context.sendRequest(modbusSerialADU)
@@ -187,7 +189,7 @@ public class ModbusRTUProtocolLogic extends Plc4xProtocolBase<ModbusSerialADU> i
             PlcField field = request.getField(fieldName);
             final ModbusPDU requestPdu = getWriteRequestPdu(field, ((DefaultPlcWriteRequest) writeRequest).getPlcValue(fieldName));
 
-            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu);
+            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu, calculateChecksum(address, requestPdu));
             RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
             transaction.submit(() -> context.sendRequest(modbusSerialADU)
                 .expectResponse(ModbusSerialADU.class, requestTimeout)
diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/utils/StaticHelper.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/utils/StaticHelper.java
new file mode 100644
index 0000000..d4e7758
--- /dev/null
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/utils/StaticHelper.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.plc4x.java.modbus_rtu.utils;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
+import org.apache.plc4x.java.api.messages.PlcReadRequest;
+import org.apache.plc4x.java.api.messages.PlcReadResponse;
+import org.apache.plc4x.java.api.messages.PlcWriteRequest;
+import org.apache.plc4x.java.api.messages.PlcWriteResponse;
+import org.apache.plc4x.java.api.model.PlcField;
+import org.apache.plc4x.java.api.types.PlcResponseCode;
+import org.apache.plc4x.java.api.value.PlcValue;
+import org.apache.plc4x.java.modbus.field.*;
+import org.apache.plc4x.java.modbus.readwrite.*;
+import org.apache.plc4x.java.modbus.readwrite.io.DataItemIO;
+import org.apache.plc4x.java.modbus.readwrite.io.ModbusPDUIO;
+import org.apache.plc4x.java.modbus.readwrite.types.ModbusDataType;
+import org.apache.plc4x.java.modbus_rtu.config.ModbusRTUConfiguration;
+import org.apache.plc4x.java.spi.ConversationContext;
+import org.apache.plc4x.java.spi.Plc4xProtocolBase;
+import org.apache.plc4x.java.spi.configuration.HasConfiguration;
+import org.apache.plc4x.java.spi.generation.ParseException;
+import org.apache.plc4x.java.spi.generation.ReadBuffer;
+import org.apache.plc4x.java.spi.generation.ReadBufferByteBased;
+import org.apache.plc4x.java.spi.generation.WriteBufferByteBased;
+import org.apache.plc4x.java.spi.messages.DefaultPlcReadRequest;
+import org.apache.plc4x.java.spi.messages.DefaultPlcReadResponse;
+import org.apache.plc4x.java.spi.messages.DefaultPlcWriteRequest;
+import org.apache.plc4x.java.spi.messages.DefaultPlcWriteResponse;
+import org.apache.plc4x.java.spi.messages.utils.ResponseItem;
+import org.apache.plc4x.java.spi.transaction.RequestTransactionManager;
+import org.apache.plc4x.java.spi.values.PlcBOOL;
+import org.apache.plc4x.java.spi.values.PlcList;
+
+import java.time.Duration;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+
+public class StaticHelper {
+
+    public static Integer ModbusCRC(byte[] buf, int len)
+    {
+        Integer crc = 0xFFFF;
+        for (int pos = 0; pos < len; pos++) {
+            crc ^= (int)buf[pos];
+            for (int i = 8; i != 0; i--) {
+                if ((crc & 0x0001) != 0) {
+                    crc >>= 1;
+                    crc ^= 0xA001;
+                }
+                else
+                    crc >>= 1;
+            }
+        }
+        return ((crc >> 8) & 0xff) | ((crc & 0xff) << 8);
+    }
+
+    public static Integer calculateChecksum(short address, ModbusPDU pdu) {
+        WriteBufferByteBased buffer = new WriteBufferByteBased(pdu.getLengthInBytes() + 1);
+        try {
+            buffer.writeShort("test", 8, address, null);
+            ModbusPDUIO.staticSerialize(buffer, pdu);
+        } catch (ParseException e) {
+            return -1;
+        }
+        return ModbusCRC(buffer.getData(), buffer.getData().length);
+    }
+
+}
diff --git a/plc4j/drivers/modbus/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver b/plc4j/drivers/modbus/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver
index abed59a..818f994 100644
--- a/plc4j/drivers/modbus/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver
+++ b/plc4j/drivers/modbus/src/main/resources/META-INF/services/org.apache.plc4x.java.api.PlcDriver
@@ -17,3 +17,4 @@
 # under the License.
 #
 org.apache.plc4x.java.modbus.ModbusDriver
+org.apache.plc4x.java.modbus_rtu.ModbusRTUDriver
\ No newline at end of file
diff --git a/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec b/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
index 8995bcb..2d99ee1 100644
--- a/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
+++ b/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
@@ -52,7 +52,8 @@
     [simple         ModbusPDU   'pdu' ['response']]
 
     //This should be a checksum, not sure the status of the checksum field esp in C and Go
-    [checksum uint 16           'headerChecksum' '100']
+    //[checksum uint 16           'headerChecksum' 'STATIC_CALL("org.apache.plc4x.java.modbus_rtu.utils.StaticHelper.calculateChecksum", readBuffer, _type.encoding)']
+    [simple uint 16 'crc']
 ]
 
 [discriminatedType 'ModbusPDU' [bit 'response']

[plc4x] 01/02: Initial Modbus RTU Support.

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

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

commit c1a8bbdc0c834a61cdcf977ee0f03dc6e73892e7
Author: hutcheb <hu...@apache.org>
AuthorDate: Tue Aug 17 07:14:29 2021 +1000

    Initial Modbus RTU Support.
    
    Hasn't been tested,
    Checksum field still needs work.
    As well as check to see how multiple drivers in one module works
    Refactor PDU handling, it is the same for Modbus TCP
---
 .../plc4x/java/modbus_rtu/ModbusRTUDriver.java     | 133 ++++++
 .../modbus_rtu/config/ModbusRTUConfiguration.java  |  64 +++
 .../protocol/ModbusRTUProtocolLogic.java           | 473 +++++++++++++++++++++
 .../main/resources/protocols/modbus/modbus.mspec   |   6 +-
 4 files changed, 673 insertions(+), 3 deletions(-)

diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java
new file mode 100644
index 0000000..a8ede08
--- /dev/null
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/ModbusRTUDriver.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.plc4x.java.modbus_rtu;
+
+import io.netty.buffer.ByteBuf;
+import org.apache.plc4x.java.api.value.PlcValueHandler;
+import org.apache.plc4x.java.modbus.config.ModbusConfiguration;
+import org.apache.plc4x.java.modbus.field.ModbusField;
+import org.apache.plc4x.java.modbus.field.ModbusFieldHandler;
+import org.apache.plc4x.java.modbus.readwrite.ModbusSerialADU;
+import org.apache.plc4x.java.modbus.readwrite.ModbusTcpADU;
+import org.apache.plc4x.java.modbus.readwrite.io.ModbusSerialADUIO;
+import org.apache.plc4x.java.modbus.readwrite.io.ModbusTcpADUIO;
+import org.apache.plc4x.java.modbus_rtu.config.ModbusRTUConfiguration;
+import org.apache.plc4x.java.modbus_rtu.protocol.ModbusRTUProtocolLogic;
+import org.apache.plc4x.java.spi.configuration.Configuration;
+import org.apache.plc4x.java.spi.connection.GeneratedDriverBase;
+import org.apache.plc4x.java.spi.connection.ProtocolStackConfigurer;
+import org.apache.plc4x.java.spi.connection.SingleProtocolStackConfigurer;
+import org.apache.plc4x.java.spi.optimizer.BaseOptimizer;
+import org.apache.plc4x.java.spi.optimizer.SingleFieldOptimizer;
+import org.apache.plc4x.java.spi.values.IEC61131ValueHandler;
+
+import java.util.function.ToIntFunction;
+
+public class ModbusRTUDriver extends GeneratedDriverBase<ModbusSerialADU> {
+
+    @Override
+    public String getProtocolCode() {
+        return "modbus-rtu";
+    }
+
+    @Override
+    public String getProtocolName() {
+        return "ModbusRTU";
+    }
+
+    @Override
+    protected Class<? extends Configuration> getConfigurationType() {
+        return ModbusRTUConfiguration.class;
+    }
+
+    @Override
+    protected String getDefaultTransport() {
+        return "serial";
+    }
+
+    /**
+     * Modbus doesn't have a login procedure, so there is no need to wait for a login to finish.
+     * @return false
+     */
+    @Override
+    protected boolean awaitSetupComplete() {
+        return false;
+    }
+
+    /**
+     * This protocol doesn't have a disconnect procedure, so there is no need to wait for a login to finish.
+     * @return false
+     */
+    @Override
+    protected boolean awaitDisconnectComplete() {
+        return false;
+    }
+
+    @Override
+    protected boolean canRead() {
+        return true;
+    }
+
+    @Override
+    protected boolean canWrite() {
+        return true;
+    }
+
+    @Override
+    protected BaseOptimizer getOptimizer() {
+        return new SingleFieldOptimizer();
+    }
+
+    @Override
+    protected ModbusFieldHandler getFieldHandler() {
+        return new ModbusFieldHandler();
+    }
+
+    @Override
+    protected PlcValueHandler getValueHandler() {
+        return new IEC61131ValueHandler();
+    }
+
+    @Override
+    protected ProtocolStackConfigurer<ModbusSerialADU> getStackConfigurer() {
+        return SingleProtocolStackConfigurer.builder(ModbusSerialADU.class, ModbusSerialADUIO.class)
+            .withProtocol(ModbusRTUProtocolLogic.class)
+            .withPacketSizeEstimator(ByteLengthEstimator.class)
+            // Every incoming message is to be treated as a response.
+            .withParserArgs(true)
+            .build();
+    }
+
+    /** Estimate the Length of a Packet */
+    public static class ByteLengthEstimator implements ToIntFunction<ByteBuf> {
+        @Override
+        public int applyAsInt(ByteBuf byteBuf) {
+            if (byteBuf.readableBytes() >= 6) {
+                return byteBuf.getUnsignedShort(byteBuf.readerIndex() + 4) + 6;
+            }
+            return -1;
+        }
+    }
+
+    @Override
+    public ModbusField prepareField(String query){
+        return ModbusField.of(query);
+    }
+
+}
diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/config/ModbusRTUConfiguration.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/config/ModbusRTUConfiguration.java
new file mode 100644
index 0000000..350f1a2
--- /dev/null
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/config/ModbusRTUConfiguration.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.plc4x.java.modbus_rtu.config;
+
+import org.apache.plc4x.java.modbus.readwrite.ModbusConstants;
+import org.apache.plc4x.java.spi.configuration.Configuration;
+import org.apache.plc4x.java.spi.configuration.annotations.ConfigurationParameter;
+import org.apache.plc4x.java.spi.configuration.annotations.defaults.IntDefaultValue;
+import org.apache.plc4x.java.transport.tcp.TcpTransportConfiguration;
+
+public class ModbusRTUConfiguration implements Configuration, TcpTransportConfiguration {
+
+    @ConfigurationParameter("request-timeout")
+    @IntDefaultValue(5_000)
+    private int requestTimeout;
+
+    @ConfigurationParameter("address")
+    @IntDefaultValue(1)
+    private int address;
+
+    public int getRequestTimeout() {
+        return requestTimeout;
+    }
+
+    public void setRequestTimeout(int requestTimeout) {
+        this.requestTimeout = requestTimeout;
+    }
+
+    public int getAddress() {
+        return address;
+    }
+
+    public void setAddress(int address) {
+        this.address = address;
+    }
+
+    @Override
+    public int getDefaultPort() {
+        return ModbusConstants.MODBUSTCPDEFAULTPORT;
+    }
+
+    @Override
+    public String toString() {
+        return "Configuration{" +
+            '}';
+    }
+
+}
diff --git a/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java
new file mode 100644
index 0000000..204441c
--- /dev/null
+++ b/plc4j/drivers/modbus/src/main/java/org/apache/plc4x/java/modbus_rtu/protocol/ModbusRTUProtocolLogic.java
@@ -0,0 +1,473 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.plc4x.java.modbus_rtu.protocol;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
+import org.apache.plc4x.java.api.messages.PlcReadRequest;
+import org.apache.plc4x.java.api.messages.PlcReadResponse;
+import org.apache.plc4x.java.api.messages.PlcWriteRequest;
+import org.apache.plc4x.java.api.messages.PlcWriteResponse;
+import org.apache.plc4x.java.api.model.PlcField;
+import org.apache.plc4x.java.api.types.PlcResponseCode;
+import org.apache.plc4x.java.api.value.PlcValue;
+import org.apache.plc4x.java.modbus.config.ModbusConfiguration;
+import org.apache.plc4x.java.modbus.field.*;
+import org.apache.plc4x.java.modbus.readwrite.*;
+import org.apache.plc4x.java.modbus.readwrite.io.DataItemIO;
+import org.apache.plc4x.java.modbus.readwrite.types.ModbusDataType;
+import org.apache.plc4x.java.modbus_rtu.config.ModbusRTUConfiguration;
+import org.apache.plc4x.java.spi.ConversationContext;
+import org.apache.plc4x.java.spi.Plc4xProtocolBase;
+import org.apache.plc4x.java.spi.configuration.HasConfiguration;
+import org.apache.plc4x.java.spi.generation.ParseException;
+import org.apache.plc4x.java.spi.generation.ReadBuffer;
+import org.apache.plc4x.java.spi.generation.ReadBufferByteBased;
+import org.apache.plc4x.java.spi.generation.WriteBufferByteBased;
+import org.apache.plc4x.java.spi.messages.DefaultPlcReadRequest;
+import org.apache.plc4x.java.spi.messages.DefaultPlcReadResponse;
+import org.apache.plc4x.java.spi.messages.DefaultPlcWriteRequest;
+import org.apache.plc4x.java.spi.messages.DefaultPlcWriteResponse;
+import org.apache.plc4x.java.spi.messages.utils.ResponseItem;
+import org.apache.plc4x.java.spi.transaction.RequestTransactionManager;
+import org.apache.plc4x.java.spi.values.PlcBOOL;
+import org.apache.plc4x.java.spi.values.PlcList;
+
+import java.time.Duration;
+import java.util.*;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class ModbusRTUProtocolLogic extends Plc4xProtocolBase<ModbusSerialADU> implements HasConfiguration<ModbusRTUConfiguration> {
+
+    private Duration requestTimeout;
+    private short address;
+    private RequestTransactionManager tm;
+    private final static int FC_EXTENDED_REGISTERS_GROUP_HEADER_LENGTH = 2;
+    private final static int FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH = 10000;
+
+    @Override
+    public void setConfiguration(ModbusRTUConfiguration configuration) {
+        this.requestTimeout = Duration.ofMillis(configuration.getRequestTimeout());
+        this.address = (short) configuration.getAddress();
+        this.tm = new RequestTransactionManager(1);
+    }
+
+    @Override
+    public void close(ConversationContext<ModbusSerialADU> context) {
+        // Nothing to do here ...
+    }
+
+    @Override
+    public CompletableFuture<PlcReadResponse> read(PlcReadRequest readRequest) {
+        CompletableFuture<PlcReadResponse> future = new CompletableFuture<>();
+        DefaultPlcReadRequest request = (DefaultPlcReadRequest) readRequest;
+
+        // 1. Sort all items by type:
+        //      - DiscreteInput     (read-only)     --> ModbusPduReadDiscreteInputsRequest
+        //      - Coil              (read-write)    --> ModbusPduReadCoilsRequest
+        //      - InputRegister     (read-only)     --> ModbusPduReadInputRegistersRequest
+        //      - HoldingRegister   (read-write)    --> ModbusPduReadHoldingRegistersRequest
+        //      - FifoQueue         (read-only)     --> ModbusPduReadFifoQueueRequest
+        //      - FileRecord        (read-write)    --> ModbusPduReadFileRecordRequest
+        // 2. Split up into multiple sub-requests
+
+        // Example for sending a request ...
+        if(request.getFieldNames().size() == 1) {
+            String fieldName = request.getFieldNames().iterator().next();
+            ModbusField field = (ModbusField) request.getField(fieldName);
+            final ModbusPDU requestPdu = getReadRequestPdu(field);
+            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu);
+
+            RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
+            transaction.submit(() -> context.sendRequest(modbusSerialADU)
+                .expectResponse(ModbusSerialADU.class, requestTimeout)
+                .onTimeout(future::completeExceptionally)
+                .onError((p, e) -> future.completeExceptionally(e))
+                .check(p -> (p.getAddress() == address))
+                .unwrap(ModbusSerialADU::getPdu)
+                .handle(responsePdu -> {
+                    // Try to decode the response data based on the corresponding request.
+                    PlcValue plcValue = null;
+                    PlcResponseCode responseCode;
+                    // Check if the response was an error response.
+                    if (responsePdu instanceof ModbusPDUError) {
+                        ModbusPDUError errorResponse = (ModbusPDUError) responsePdu;
+                        responseCode = getErrorCode(errorResponse);
+                    } else {
+                        try {
+                            plcValue = toPlcValue(requestPdu, responsePdu, field.getDataType());
+                            responseCode = PlcResponseCode.OK;
+                        } catch (ParseException e) {
+                            // Add an error response code ...
+                            responseCode = PlcResponseCode.INTERNAL_ERROR;
+                        }
+                    }
+
+                    // Prepare the response.
+                    PlcReadResponse response = new DefaultPlcReadResponse(request,
+                        Collections.singletonMap(fieldName, new ResponseItem<>(responseCode, plcValue)));
+
+                    // Pass the response back to the application.
+                    future.complete(response);
+
+                    // Finish the request-transaction.
+                    transaction.endRequest();
+            }));
+        } else {
+            future.completeExceptionally(new PlcRuntimeException("Modbus only supports single filed requests"));
+        }
+        return future;
+    }
+
+    @Override
+    protected void decode(ConversationContext<ModbusSerialADU> context, ModbusSerialADU msg) throws Exception {
+        super.decode(context, msg);
+    }
+
+    private PlcResponseCode getErrorCode(ModbusPDUError errorResponse) {
+        switch (errorResponse.getExceptionCode()) {
+            case ILLEGAL_FUNCTION:
+                return PlcResponseCode.UNSUPPORTED;
+            case ILLEGAL_DATA_ADDRESS:
+                return PlcResponseCode.INVALID_ADDRESS;
+            case ILLEGAL_DATA_VALUE:
+                return PlcResponseCode.INVALID_DATA;
+            case SLAVE_DEVICE_FAILURE:
+                return PlcResponseCode.REMOTE_ERROR;
+            case ACKNOWLEDGE:
+                return PlcResponseCode.OK;
+            case SLAVE_DEVICE_BUSY:
+                return PlcResponseCode.REMOTE_BUSY;
+            case NEGATIVE_ACKNOWLEDGE:
+                return PlcResponseCode.REMOTE_ERROR;
+            case MEMORY_PARITY_ERROR:
+                return PlcResponseCode.INTERNAL_ERROR;
+            case GATEWAY_PATH_UNAVAILABLE:
+                return PlcResponseCode.INTERNAL_ERROR;
+            case GATEWAY_TARGET_DEVICE_FAILED_TO_RESPOND:
+                return PlcResponseCode.REMOTE_ERROR;
+            default:
+                // This generally implies that something went wrong which we didn't anticipate.
+                return PlcResponseCode.INTERNAL_ERROR;
+        }
+    }
+
+    @Override
+    public CompletableFuture<PlcWriteResponse> write(PlcWriteRequest writeRequest) {
+        CompletableFuture<PlcWriteResponse> future = new CompletableFuture<>();
+        DefaultPlcWriteRequest request = (DefaultPlcWriteRequest) writeRequest;
+
+        // 1. Sort all items by type:
+        //      - DiscreteInput     (read-only)     --> Error
+        //      - Coil              (read-write)    --> ModbusPduWriteSingleCoilRequest / ModbusPduWriteMultipleCoilsRequest
+        //      - InputRegister     (read-only)     --> Error
+        //      - HoldingRegister   (read-write)    --> ModbusPduWriteSingleRegisterRequest / ModbusPduWriteMultipleRegistersRequest
+        //      - FifoQueue         (read-only)     --> Error
+        //      - FileRecord        (read-write)    --> ModbusPduWriteFileRecordRequest
+        // 2. Split up into multiple sub-requests
+        if(request.getFieldNames().size() == 1) {
+            String fieldName = request.getFieldNames().iterator().next();
+            PlcField field = request.getField(fieldName);
+            final ModbusPDU requestPdu = getWriteRequestPdu(field, ((DefaultPlcWriteRequest) writeRequest).getPlcValue(fieldName));
+
+            ModbusSerialADU modbusSerialADU = new ModbusSerialADU(address, requestPdu);
+            RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
+            transaction.submit(() -> context.sendRequest(modbusSerialADU)
+                .expectResponse(ModbusSerialADU.class, requestTimeout)
+                .onTimeout(future::completeExceptionally)
+                .onError((p, e) -> future.completeExceptionally(e))
+                .check(p -> p.getAddress() == address)
+                .unwrap(ModbusSerialADU::getPdu)
+                .handle(responsePdu -> {
+                    // Try to decode the response data based on the corresponding request.
+                    PlcValue plcValue = null;
+                    PlcResponseCode responseCode;
+
+                    // Check if the response was an error response.
+                    if (responsePdu instanceof ModbusPDUError) {
+                        ModbusPDUError errorResponse = (ModbusPDUError) responsePdu;
+                        responseCode = getErrorCode(errorResponse);
+                    } else {
+                        // TODO: Check the correct number of elements were written.
+                        if (responsePdu instanceof ModbusPDUWriteSingleCoilResponse) {
+                            ModbusPDUWriteSingleCoilResponse response = (ModbusPDUWriteSingleCoilResponse) responsePdu;
+                            ModbusPDUWriteSingleCoilRequest requestSingleCoil = (ModbusPDUWriteSingleCoilRequest) requestPdu;
+                            if (!((response.getValue() == requestSingleCoil.getValue()) && (response.getAddress() == requestSingleCoil.getAddress()))) {
+                                responseCode = PlcResponseCode.REMOTE_ERROR;
+                            }
+                        }
+                        responseCode = PlcResponseCode.OK;
+                    }
+
+                    // Prepare the response.
+                    PlcWriteResponse response = new DefaultPlcWriteResponse(request,
+                        Collections.singletonMap(fieldName, responseCode));
+
+                    // Pass the response back to the application.
+                    future.complete(response);
+
+                    // Finish the request-transaction.
+                    transaction.endRequest();
+                }));
+
+        } else {
+            future.completeExceptionally(new PlcRuntimeException("Modbus only supports single filed requests"));
+        }
+        return future;
+    }
+
+    private ModbusPDU getReadRequestPdu(PlcField field) {
+        if(field instanceof ModbusFieldDiscreteInput) {
+            ModbusFieldDiscreteInput discreteInput = (ModbusFieldDiscreteInput) field;
+            return new ModbusPDUReadDiscreteInputsRequest(discreteInput.getAddress(), discreteInput.getNumberOfElements());
+        } else if(field instanceof ModbusFieldCoil) {
+            ModbusFieldCoil coil = (ModbusFieldCoil) field;
+            return new ModbusPDUReadCoilsRequest(coil.getAddress(), coil.getNumberOfElements());
+        } else if(field instanceof ModbusFieldInputRegister) {
+            ModbusFieldInputRegister inputRegister = (ModbusFieldInputRegister) field;
+            return new ModbusPDUReadInputRegistersRequest(inputRegister.getAddress(), Math.max(inputRegister.getLengthWords(), 1));
+        } else if(field instanceof ModbusFieldHoldingRegister) {
+            ModbusFieldHoldingRegister holdingRegister = (ModbusFieldHoldingRegister) field;
+            return new ModbusPDUReadHoldingRegistersRequest(holdingRegister.getAddress(), Math.max(holdingRegister.getLengthWords(), 1));
+        } else if(field instanceof ModbusExtendedRegister) {
+            ModbusExtendedRegister extendedRegister = (ModbusExtendedRegister) field;
+            int group1Address = extendedRegister.getAddress() % 10000;
+            int group2Address = 0;
+            int group1Quantity;
+            int group2Quantity;
+            short group1FileNumber = (short) (Math.floor(extendedRegister.getAddress() / 10000) + 1);
+            short group2FileNumber;
+            ModbusPDUReadFileRecordRequestItem[] itemArray;
+
+            if ((group1Address + extendedRegister.getLengthWords()) <= FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH) {
+              // If request doesn't span file records, use a single group
+              group1Quantity = extendedRegister.getLengthWords();
+              ModbusPDUReadFileRecordRequestItem group1 =
+                  new ModbusPDUReadFileRecordRequestItem((short) 6, group1FileNumber, group1Address, group1Quantity);
+              itemArray = new ModbusPDUReadFileRecordRequestItem[] {group1};
+            } else {
+              // If it doesn't span a file record. e.g. 609998[10] request 2 words in first group and 8 in second.
+              group1Quantity = FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH - group1Address;
+              group2Quantity = extendedRegister.getLengthWords() - group1Quantity;
+              group2FileNumber = (short) (group1FileNumber + 1);
+              ModbusPDUReadFileRecordRequestItem group1 =
+                  new ModbusPDUReadFileRecordRequestItem((short) 6, group1FileNumber, group1Address, group1Quantity);
+              ModbusPDUReadFileRecordRequestItem group2 =
+                  new ModbusPDUReadFileRecordRequestItem((short) 6, group2FileNumber, group2Address, group2Quantity);
+              itemArray = new ModbusPDUReadFileRecordRequestItem[] {group1, group2};
+            }
+            return new ModbusPDUReadFileRecordRequest(itemArray);
+        }
+        throw new PlcRuntimeException("Unsupported read field type " + field.getClass().getName());
+    }
+
+    private ModbusPDU getWriteRequestPdu(PlcField field, PlcValue plcValue) {
+        if(field instanceof ModbusFieldCoil) {
+            ModbusFieldCoil coil = (ModbusFieldCoil) field;
+            ModbusPDUWriteMultipleCoilsRequest request =
+                new ModbusPDUWriteMultipleCoilsRequest(coil.getAddress(), coil.getNumberOfElements(),
+                fromPlcValue(field, plcValue));
+            if (request.getQuantity() == coil.getNumberOfElements()) {
+                return request;
+            } else {
+                throw new PlcRuntimeException("Number of requested bytes (" + request.getQuantity() +
+                    ") doesn't match number of requested addresses (" + coil.getNumberOfElements() + ")");
+            }
+        } else if(field instanceof ModbusFieldHoldingRegister) {
+            ModbusFieldHoldingRegister holdingRegister = (ModbusFieldHoldingRegister) field;
+            ModbusPDUWriteMultipleHoldingRegistersRequest request =
+                new ModbusPDUWriteMultipleHoldingRegistersRequest(holdingRegister.getAddress(),
+                holdingRegister.getLengthWords(), fromPlcValue(field, plcValue));
+            if (request.getValue().length == holdingRegister.getLengthWords()*2) {
+                return request;
+            } else {
+                throw new PlcRuntimeException("Number of requested values (" + request.getValue().length/2 +
+                    ") doesn't match number of requested addresses (" + holdingRegister.getLengthWords() + ")");
+            }
+        } else if(field instanceof ModbusExtendedRegister) {
+            ModbusExtendedRegister extendedRegister = (ModbusExtendedRegister) field;
+            int group1Address = extendedRegister.getAddress() % FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH;
+            int group2Address = 0;
+            int group1Quantity;
+            int group2Quantity;
+            byte[] plcValue1, plcValue2;
+            short group1FileNumber = (short)
+                (Math.floor(extendedRegister.getAddress() / FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH) + 1);
+            short group2FileNumber;
+            ModbusPDUWriteFileRecordRequestItem[] itemArray;
+            if ((group1Address + extendedRegister.getLengthWords()) <= FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH) {
+              //If request doesn't span file records, use a single group
+              group1Quantity = extendedRegister.getLengthWords();
+              ModbusPDUWriteFileRecordRequestItem group1 = new ModbusPDUWriteFileRecordRequestItem(
+                  (short) 6, group1FileNumber, group1Address, fromPlcValue(field, plcValue));
+              itemArray = new ModbusPDUWriteFileRecordRequestItem[] {group1};
+            } else {
+              //If it doesn't span a file record. e.g. 609998[10] request 2 words in first group and 8 in second.
+              group1Quantity = FC_EXTENDED_REGISTERS_FILE_RECORD_LENGTH - group1Address;
+              group2Quantity = extendedRegister.getLengthWords() - group1Quantity;
+              group2FileNumber = (short) (group1FileNumber + 1);
+
+              plcValue1 = ArrayUtils.subarray(fromPlcValue(field, plcValue), 0, group1Quantity);
+              plcValue2 = ArrayUtils.subarray(
+                  fromPlcValue(field, plcValue), group1Quantity, fromPlcValue(field, plcValue).length);
+              ModbusPDUWriteFileRecordRequestItem group1 = new ModbusPDUWriteFileRecordRequestItem(
+                  (short) 6, group1FileNumber, group1Address, plcValue1);
+              ModbusPDUWriteFileRecordRequestItem group2 = new ModbusPDUWriteFileRecordRequestItem(
+                  (short) 6, group2FileNumber, group2Address, plcValue2);
+              itemArray = new ModbusPDUWriteFileRecordRequestItem[] {group1, group2};
+            }
+            return new ModbusPDUWriteFileRecordRequest(itemArray);
+        }
+        throw new PlcRuntimeException("Unsupported write field type " + field.getClass().getName());
+    }
+
+    private PlcValue toPlcValue(ModbusPDU request, ModbusPDU response, ModbusDataType dataType) throws ParseException {
+        Short fieldDataTypeSize = dataType.getDataTypeSize();
+
+        if (request instanceof ModbusPDUReadDiscreteInputsRequest) {
+            if (!(response instanceof ModbusPDUReadDiscreteInputsResponse)) {
+                throw new PlcRuntimeException("Unexpected response type. " +
+                    "Expected ModbusPDUReadDiscreteInputsResponse, but got " + response.getClass().getName());
+            }
+            ModbusPDUReadDiscreteInputsRequest req = (ModbusPDUReadDiscreteInputsRequest) request;
+            ModbusPDUReadDiscreteInputsResponse resp = (ModbusPDUReadDiscreteInputsResponse) response;
+            return readBooleanList(req.getQuantity(), resp.getValue());
+        } else if (request instanceof ModbusPDUReadCoilsRequest) {
+            if (!(response instanceof ModbusPDUReadCoilsResponse)) {
+                throw new PlcRuntimeException("Unexpected response type. " +
+                    "Expected ModbusPDUReadCoilsResponse, but got " + response.getClass().getName());
+            }
+            ModbusPDUReadCoilsRequest req = (ModbusPDUReadCoilsRequest) request;
+            ModbusPDUReadCoilsResponse resp = (ModbusPDUReadCoilsResponse) response;
+            return readBooleanList(req.getQuantity(), resp.getValue());
+        } else if (request instanceof ModbusPDUReadInputRegistersRequest) {
+            if (!(response instanceof ModbusPDUReadInputRegistersResponse)) {
+                throw new PlcRuntimeException("Unexpected response type. " +
+                    "Expected ModbusPDUReadInputRegistersResponse, but got " + response.getClass().getName());
+            }
+            ModbusPDUReadInputRegistersRequest req = (ModbusPDUReadInputRegistersRequest) request;
+            ModbusPDUReadInputRegistersResponse resp = (ModbusPDUReadInputRegistersResponse) response;
+            ReadBuffer io = new ReadBufferByteBased(resp.getValue());
+            if(fieldDataTypeSize < 2) {
+                io.readByte();
+            }
+            return DataItemIO.staticParse(io, dataType, Math.max(Math.round(req.getQuantity()/(fieldDataTypeSize/2.0f)), 1));
+        } else if (request instanceof ModbusPDUReadHoldingRegistersRequest) {
+            if (!(response instanceof ModbusPDUReadHoldingRegistersResponse)) {
+                throw new PlcRuntimeException("Unexpected response type. " +
+                    "Expected ModbusPDUReadHoldingRegistersResponse, but got " + response.getClass().getName());
+            }
+            ModbusPDUReadHoldingRegistersRequest req = (ModbusPDUReadHoldingRegistersRequest) request;
+            ModbusPDUReadHoldingRegistersResponse resp = (ModbusPDUReadHoldingRegistersResponse) response;
+            ReadBuffer io = new ReadBufferByteBased(resp.getValue());
+            if((dataType != ModbusDataType.STRING) && fieldDataTypeSize < 2) {
+                io.readByte();
+            }
+            return DataItemIO.staticParse(io, dataType, Math.max(Math.round(req.getQuantity()/(fieldDataTypeSize/2.0f)), 1));
+        } else if (request instanceof ModbusPDUReadFileRecordRequest) {
+            if (!(response instanceof ModbusPDUReadFileRecordResponse)) {
+                throw new PlcRuntimeException("Unexpected response type. " +
+                    "Expected ModbusPDUReadFileRecordResponse, but got " + response.getClass().getName());
+            }
+            ModbusPDUReadFileRecordRequest req = (ModbusPDUReadFileRecordRequest) request;
+            ModbusPDUReadFileRecordResponse resp = (ModbusPDUReadFileRecordResponse) response;
+            ReadBuffer io;
+            short dataLength;
+
+            if (resp.getItems().length == 2 && resp.getItems().length == req.getItems().length) {
+              //If request was split over file records, two groups in reponse should be received.
+              io = new ReadBufferByteBased(ArrayUtils.addAll(resp.getItems()[0].getData(), resp.getItems()[1].getData()));
+              dataLength = (short) (resp.getItems()[0].getLengthInBytes() + resp.getItems()[1].getLengthInBytes() - (2 * FC_EXTENDED_REGISTERS_GROUP_HEADER_LENGTH));
+            } else if (resp.getItems().length == 1 && resp.getItems().length == req.getItems().length) {
+              //If request was within a single file record, one group should be received.
+              io = new ReadBufferByteBased(resp.getItems()[0].getData());
+              dataLength = (short) (resp.getItems()[0].getLengthInBytes() - FC_EXTENDED_REGISTERS_GROUP_HEADER_LENGTH);
+            } else {
+              throw new PlcRuntimeException("Unexpected number of groups in response. " +
+                  "Expected " + req.getItems().length + ", but got " + resp.getItems().length);
+            }
+            if(fieldDataTypeSize < 2) {
+                io.readByte();
+            }
+            return DataItemIO.staticParse(io, dataType, Math.round(Math.max(dataLength/2.0f, 1)/Math.max(fieldDataTypeSize/2.0f, 1)));
+        }
+        return null;
+    }
+
+    private byte[] fromPlcValue(PlcField field, PlcValue plcValue) {
+        ModbusDataType fieldDataType = ((ModbusField) field).getDataType();
+        try {
+            WriteBufferByteBased buffer;
+            if(plcValue instanceof PlcList) {
+                buffer = DataItemIO.staticSerialize(plcValue, fieldDataType, plcValue.getLength(), false);
+                byte[] data = buffer.getData();
+                switch (((ModbusField) field).getDataType()) {
+                    case BOOL:
+                        //Reverse Bits in each byte as
+                        //they should ordered like this: 8 7 6 5 4 3 2 1 | 0 0 0 0 0 0 0 9
+                        byte[] bytes = new byte[data.length];
+                        for (int i = 0; i < data.length; i++) {
+                            bytes[i] = reverseBitsOfByte(data[i]);
+                        }
+                        return bytes;
+                    default:
+                        return data;
+                }
+            } else {
+                buffer = DataItemIO.staticSerialize(plcValue, fieldDataType, plcValue.getLength(), false);
+                if (buffer != null) {
+                    return buffer.getData();
+                } else {
+                    throw new PlcRuntimeException("Unable to parse PlcValue :- " + ((ModbusField) field).getPlcDataType());
+                }
+            }
+        } catch (ParseException e) {
+            throw new PlcRuntimeException("Unable to parse PlcValue :- " + e);
+        }
+
+    }
+
+    private byte reverseBitsOfByte(byte b) {
+        BitSet bits = BitSet.valueOf(new byte[] {b});
+        BitSet reverse = BitSet.valueOf(new byte[] {(byte) 0xFF});
+        for (int j = 0; j < 8; j++) {
+            reverse.set(j, bits.get(7-j));
+        }
+        //toByteArray returns an empty array if all the bits are set to 0.
+        return Arrays.copyOf(reverse.toByteArray(), 1)[0];
+    }
+
+    private PlcValue readBooleanList(int count, byte[] data) throws ParseException {
+        ReadBuffer io = new ReadBufferByteBased(data);
+        if(count == 1) {
+            return DataItemIO.staticParse(io, ModbusDataType.BOOL, 1);
+        }
+        // Make sure we read in all the bytes. Unfortunately when requesting 9 bytes
+        // they are ordered like this: 8 7 6 5 4 3 2 1 | 0 0 0 0 0 0 0 9
+        // Luckily it turns out that this is exactly how BitSet parses byte[]
+        BitSet bits = BitSet.valueOf(data);
+        List<PlcValue> result = new ArrayList<>(count);
+        for(int i = 0; i < count; i++) {
+            result.add(new PlcBOOL(bits.get(i)));
+        }
+        return new PlcList(result);
+    }
+
+}
diff --git a/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec b/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
index 844e007..8995bcb 100644
--- a/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
+++ b/protocols/modbus/src/main/resources/protocols/modbus/modbus.mspec
@@ -46,13 +46,13 @@
 ]
 
 [type 'ModbusSerialADU' [bit 'response']
-    [simple         uint 16     'transactionId']
-    [reserved       uint 16     '0x0000']
-    [simple         uint 16     'length']
     [simple         uint 8      'address']
 
     // The actual modbus payload
     [simple         ModbusPDU   'pdu' ['response']]
+
+    //This should be a checksum, not sure the status of the checksum field esp in C and Go
+    [checksum uint 16           'headerChecksum' '100']
 ]
 
 [discriminatedType 'ModbusPDU' [bit 'response']