You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by sr...@apache.org on 2018/07/19 13:23:20 UTC

[incubator-plc4x] branch master updated (e413887 -> 234c121)

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

sruehl pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git.


    from e413887  refined test for modbus protocol
     new 917c7fb  added sanity checks for produced values and cleaned up type checks
     new c4a35b3  added tests for double data type in ads
     new be1703c  create a blacklist of not yet supported types on modbus
     new 234c121  refactored common used test code into driver-bases-test

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


Summary of changes:
 .../java/ads/protocol/Plc4x2AdsProtocolTest.java   |  38 ++-----
 .../base/protocol/Plc4XSupportedDataTypes.java     | 118 +++++++++++++++++++++
 .../java/modbus/netty/Plc4XModbusProtocol.java     |  91 ++++++++++------
 .../java/modbus/netty/Plc4XModbusProtocolTest.java |  79 ++++++--------
 4 files changed, 216 insertions(+), 110 deletions(-)
 create mode 100644 plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/protocol/Plc4XSupportedDataTypes.java


[incubator-plc4x] 04/04: refactored common used test code into driver-bases-test

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

sruehl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git

commit 234c1211e54ebdbf3dda83847b0146ead42da209
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 19 15:23:15 2018 +0200

    refactored common used test code into driver-bases-test
---
 .../java/ads/protocol/Plc4x2AdsProtocolTest.java   |  37 +------
 .../base/protocol/Plc4XSupportedDataTypes.java     | 118 +++++++++++++++++++++
 .../java/modbus/netty/Plc4XModbusProtocolTest.java |  63 +++--------
 3 files changed, 140 insertions(+), 78 deletions(-)

diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
index 4d456bd..6a921c4 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
@@ -51,6 +51,7 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.plc4x.java.base.protocol.Plc4XSupportedDataTypes.streamOfLittleEndianDataTypePairs;
 import static org.hamcrest.Matchers.*;
 import static org.junit.Assert.assertThat;
 
@@ -87,41 +88,13 @@ public class Plc4x2AdsProtocolTest {
         AmsNetId sourceAmsNetId = AmsNetId.of("8.9.10.11.12.13");
         AmsPort sourceAmsPort = AmsPort.of(14);
         Invoke invokeId = Invoke.of(2);
-        return Stream.of(Boolean.class,
-            Byte.class,
-            Short.class,
-            Calendar.class,
-            Float.class,
-            Double.class,
-            Integer.class,
-            String.class)
-            .map(clazz -> {
-                if (clazz == Boolean.class) {
-                    return ImmutablePair.of(Boolean.TRUE, new byte[]{0x01});
-                } else if (clazz == Byte.class) {
-                    return ImmutablePair.of(Byte.valueOf("1"), new byte[]{0x1});
-                } else if (clazz == Short.class) {
-                    return ImmutablePair.of(Short.valueOf("1"), new byte[]{0x1, 0x0});
-                } else if (clazz == Calendar.class) {
-                    return ImmutablePair.of(calenderInstance, new byte[]{0x0, 0x0, 0x0, 0x0, 0x4, 0x3, 0x2, 0x1});
-                } else if (clazz == Float.class) {
-                    return ImmutablePair.of(Float.valueOf("1"), new byte[]{0x0, 0x0, (byte) 0x80, 0x3F});
-                } else if (clazz == Double.class) {
-                    return ImmutablePair.of(Double.valueOf("1"), new byte[]{0x0, 0x0,0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F});
-                } else if (clazz == Integer.class) {
-                    return ImmutablePair.of(Integer.valueOf("1"), new byte[]{0x1, 0x0, 0x0, 0x0});
-                } else if (clazz == String.class) {
-                    return ImmutablePair.of(String.valueOf("Hello World!"), new byte[]{0x48, 0x65, 0x6c, 0x6c, 0x6f, 0x20, 0x57, 0x6f, 0x72, 0x6c, 0x64, 0x21, 0x00});
-                } else {
-                    throw new IllegalArgumentException("Unmapped type " + clazz);
-                }
-            })
+        return streamOfLittleEndianDataTypePairs()
             .map(pair -> Stream.of(
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem(AdsAddress.of(1, 2), pair.left)
+                            .addItem(AdsAddress.of(1, 2), pair.getLeft())
                             .build(), new CompletableFuture<>()),
                     AdsWriteResponse.of(targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId, Result.of(0))
                 ),
@@ -129,9 +102,9 @@ public class Plc4x2AdsProtocolTest {
                     new PlcRequestContainer<>(
                         PlcReadRequest
                             .builder()
-                            .addItem(pair.left.getClass(), AdsAddress.of(1, 2))
+                            .addItem(pair.getLeft().getClass(), AdsAddress.of(1, 2))
                             .build(), new CompletableFuture<>()),
-                    AdsReadResponse.of(targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId, Result.of(0), Data.of(pair.right))
+                    AdsReadResponse.of(targetAmsNetId, targetAmsPort, sourceAmsNetId, sourceAmsPort, invokeId, Result.of(0), Data.of(pair.getRight()))
                 )
             ))
             .flatMap(stream -> stream)
diff --git a/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/protocol/Plc4XSupportedDataTypes.java b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/protocol/Plc4XSupportedDataTypes.java
new file mode 100644
index 0000000..b7be8cf
--- /dev/null
+++ b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/protocol/Plc4XSupportedDataTypes.java
@@ -0,0 +1,118 @@
+/*
+ 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.base.protocol;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.io.Serializable;
+import java.util.*;
+import java.util.stream.Stream;
+
+@SuppressWarnings("unused")
+public class Plc4XSupportedDataTypes {
+
+    private final static Map<Class, Pair<? extends Serializable, byte[]>> littleEndianMap;
+    private final static Map<Class, Pair<? extends Serializable, byte[]>> bigEndianMap;
+
+    static {
+        Calendar calenderInstance = Calendar.getInstance();
+        calenderInstance.setTime(new Date(283686951976960L));
+        littleEndianMap = new HashMap<>();
+        littleEndianMap.put(Boolean.class, ImmutablePair.of(Boolean.TRUE, new byte[]{0x01}));
+        littleEndianMap.put(Byte.class, ImmutablePair.of(Byte.valueOf("1"), new byte[]{0x1}));
+        littleEndianMap.put(Short.class, ImmutablePair.of(Short.valueOf("1"), new byte[]{0x1, 0x0}));
+        littleEndianMap.put(Calendar.class, ImmutablePair.of(calenderInstance, new byte[]{0x0, 0x0, 0x0, 0x0, 0x4, 0x3, 0x2, 0x1}));
+        littleEndianMap.put(Float.class, ImmutablePair.of(Float.valueOf("1"), new byte[]{0x0, 0x0, (byte) 0x80, 0x3F}));
+        littleEndianMap.put(Double.class, ImmutablePair.of(Double.valueOf("1"), new byte[]{0x0, 0x0, 0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F}));
+        littleEndianMap.put(Integer.class, ImmutablePair.of(Integer.valueOf("1"), new byte[]{0x1, 0x0, 0x0, 0x0}));
+        littleEndianMap.put(String.class, ImmutablePair.of(String.valueOf("Hello World!"), new byte[]{0x48, 0x65, 0x6c, 0x6c, 0x6f, 0x20, 0x57, 0x6f, 0x72, 0x6c, 0x64, 0x21, 0x00}));
+        bigEndianMap = new HashMap<>();
+        littleEndianMap.forEach((clazz, pair) -> {
+            Serializable serializable = pair.getLeft();
+            byte[] littleEndianBytes = pair.getRight();
+            byte[] bigEndianBytes = ArrayUtils.clone(littleEndianBytes);
+            ArrayUtils.reverse(bigEndianBytes);
+            bigEndianMap.put(clazz, ImmutablePair.of(serializable, bigEndianBytes));
+        });
+    }
+
+    /**
+     * A {@link Stream} of {@link Class}es plc4x can currently support.
+     *
+     * @return a stream of supported data types.
+     */
+    public static Stream<Class<? extends Serializable>> streamOfPlc4XSupportedDataTypes() {
+        return Stream.of(
+            Boolean.class,
+            Byte.class,
+            Short.class,
+            Calendar.class,
+            Float.class,
+            Double.class,
+            Integer.class,
+            String.class
+        );
+    }
+
+    /**
+     * A {@link Stream} of instances of {@link Class}es plc4x can currently support with their according byte representation.
+     *
+     * @return a stream of {@link org.apache.commons.lang3.tuple.Pair}s of instances and their byte values.
+     * @see #streamOfPlc4XSupportedDataTypes
+     */
+    public static Stream<? extends Pair<? extends Serializable, byte[]>> streamOfLittleEndianDataTypePairs() {
+        return streamOfLittleEndianDataTypePairs(streamOfPlc4XSupportedDataTypes());
+    }
+
+    /**
+     * A {@link Stream} of instances of {@link Class}es which are defined by {@code inputStream} can currently support with their according byte representation.
+     *
+     * @param inputStream a stream of {@link org.apache.commons.lang3.tuple.Pair}s of instances and their byte values.
+     * @see #streamOfPlc4XSupportedDataTypes
+     */
+    public static Stream<? extends Pair<? extends Serializable, byte[]>> streamOfLittleEndianDataTypePairs(Stream<Class<? extends Serializable>> inputStream) {
+        return inputStream
+            .map(littleEndianMap::get)
+            .peek(Objects::requireNonNull);
+    }
+
+    /**
+     * A {@link Stream} of instances of {@link Class}es plc4x can currently support with their according little endian byte representation.
+     *
+     * @return a stream of {@link org.apache.commons.lang3.tuple.Pair}s of instances and their byte values.
+     * @see #streamOfPlc4XSupportedDataTypes
+     */
+    public static Stream<? extends Pair<? extends Serializable, byte[]>> streamOfBigEndianDataTypePairs() {
+        return streamOfBigEndianDataTypePairs(streamOfPlc4XSupportedDataTypes());
+    }
+
+    /**
+     * A {@link Stream} of instances of {@link Class}es which are defined by {@code inputStream} can currently support with their according big endian byte representation.
+     *
+     * @param inputStream a stream of {@link org.apache.commons.lang3.tuple.Pair}s of instances and their byte values.
+     * @see #streamOfPlc4XSupportedDataTypes
+     */
+    public static Stream<? extends Pair<? extends Serializable, byte[]>> streamOfBigEndianDataTypePairs(Stream<Class<? extends Serializable>> inputStream) {
+        return inputStream
+            .map(bigEndianMap::get)
+            .peek(Objects::requireNonNull);
+    }
+}
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
index 6aa5863..544baa5 100644
--- a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
@@ -49,6 +49,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import static org.apache.plc4x.java.base.protocol.Plc4XSupportedDataTypes.streamOfLittleEndianDataTypePairs;
 import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
 import static org.hamcrest.Matchers.*;
 import static org.junit.Assert.assertEquals;
@@ -94,53 +95,23 @@ public class Plc4XModbusProtocolTest {
 
     @Parameterized.Parameters(name = "{index} Type:{0} {3} {5}")
     public static Collection<Object[]> data() {
-        return Stream.of(
-            Boolean.class,
-            Byte.class,
-            Short.class,
-            Calendar.class,
-            Float.class,
-            Double.class,
-            Integer.class,
-            String.class
-        )
-            .map(clazz -> {
-                if (clazz == Boolean.class) {
-                    return ImmutablePair.of(Boolean.TRUE, new byte[]{0x01});
-                } else if (clazz == Byte.class) {
-                    return ImmutablePair.of(Byte.valueOf("1"), new byte[]{0x1});
-                } else if (clazz == Short.class) {
-                    return ImmutablePair.of(Short.valueOf("1"), new byte[]{0x1, 0x0});
-                } else if (clazz == Calendar.class) {
-                    return ImmutablePair.of(calenderInstance, new byte[]{0x0, 0x0, 0x0, 0x0, 0x4, 0x3, 0x2, 0x1});
-                } else if (clazz == Float.class) {
-                    return ImmutablePair.of(Float.valueOf("1"), new byte[]{0x0, 0x0, (byte) 0x80, 0x3F});
-                } else if (clazz == Double.class) {
-                    return ImmutablePair.of(Double.valueOf("1"), new byte[]{0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F});
-                } else if (clazz == Integer.class) {
-                    return ImmutablePair.of(Integer.valueOf("1"), new byte[]{0x1, 0x0, 0x0, 0x0});
-                } else if (clazz == String.class) {
-                    return ImmutablePair.of(String.valueOf("Hello World!"), new byte[]{0x48, 0x65, 0x6c, 0x6c, 0x6f, 0x20, 0x57, 0x6f, 0x72, 0x6c, 0x64, 0x21, 0x00});
-                } else {
-                    throw new IllegalArgumentException("Unmapped type " + clazz);
-                }
-            })
+        return streamOfLittleEndianDataTypePairs()
             .map(pair -> Stream.of(
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcReadRequest
                             .builder()
-                            .addItem(pair.left.getClass(), CoilModbusAddress.of("coil:1"))
+                            .addItem(pair.getLeft().getClass(), CoilModbusAddress.of("coil:1"))
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new ReadCoilsResponse(Unpooled.wrappedBuffer(pair.right)))
+                    new ModbusTcpPayload((short) 0, (short) 0, new ReadCoilsResponse(Unpooled.wrappedBuffer(pair.getRight())))
                 ),
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem(CoilModbusAddress.of("coil:1"), pair.left)
+                            .addItem(CoilModbusAddress.of("coil:1"), pair.getLeft())
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new WriteSingleCoilResponse(1, pair.right[0]))
+                    new ModbusTcpPayload((short) 0, (short) 0, new WriteSingleCoilResponse(1, pair.getRight()[0]))
                 ),
                 /* Read request no supported on maskwrite so how to handle?
                 ImmutablePair.of(
@@ -155,7 +126,7 @@ public class Plc4XModbusProtocolTest {
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem(MaskWriteRegisterModbusAddress.of("maskwrite:1/1/2"), pair.left)
+                            .addItem(MaskWriteRegisterModbusAddress.of("maskwrite:1/1/2"), pair.getLeft())
                             .build(), new CompletableFuture<>()),
                     new ModbusTcpPayload((short) 0, (short) 0, new MaskWriteRegisterResponse(1, 1, 2))
                 ),
@@ -163,31 +134,31 @@ public class Plc4XModbusProtocolTest {
                     new PlcRequestContainer<>(
                         PlcReadRequest
                             .builder()
-                            .addItem(pair.left.getClass(), ReadDiscreteInputsModbusAddress.of("readdiscreteinputs:1"))
+                            .addItem(pair.getLeft().getClass(), ReadDiscreteInputsModbusAddress.of("readdiscreteinputs:1"))
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new ReadDiscreteInputsResponse(Unpooled.wrappedBuffer(pair.right)))
+                    new ModbusTcpPayload((short) 0, (short) 0, new ReadDiscreteInputsResponse(Unpooled.wrappedBuffer(pair.getRight())))
                 ),
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcReadRequest
                             .builder()
-                            .addItem(pair.left.getClass(), ReadHoldingRegistersModbusAddress.of("readholdingregisters:1"))
+                            .addItem(pair.getLeft().getClass(), ReadHoldingRegistersModbusAddress.of("readholdingregisters:1"))
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new ReadHoldingRegistersResponse(Unpooled.wrappedBuffer(evenUp(pair.right))))
+                    new ModbusTcpPayload((short) 0, (short) 0, new ReadHoldingRegistersResponse(Unpooled.wrappedBuffer(evenUp(pair.getRight()))))
                 ),
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcReadRequest
                             .builder()
-                            .addItem(pair.left.getClass(), ReadInputRegistersModbusAddress.of("readinputregisters:1"))
+                            .addItem(pair.getLeft().getClass(), ReadInputRegistersModbusAddress.of("readinputregisters:1"))
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new ReadInputRegistersResponse(Unpooled.wrappedBuffer(evenUp(pair.right))))
+                    new ModbusTcpPayload((short) 0, (short) 0, new ReadInputRegistersResponse(Unpooled.wrappedBuffer(evenUp(pair.getRight()))))
                 ),
                 ImmutablePair.of(
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem((Class) pair.left.getClass(), CoilModbusAddress.of("coil:1"), pair.left, pair.left, pair.left)
+                            .addItem((Class) pair.getLeft().getClass(), CoilModbusAddress.of("coil:1"), pair.getLeft(), pair.getLeft(), pair.getLeft())
                             .build(), new CompletableFuture<>()),
                     new ModbusTcpPayload((short) 0, (short) 0, new WriteMultipleCoilsResponse(1, 3))
                 ),
@@ -195,7 +166,7 @@ public class Plc4XModbusProtocolTest {
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem((Class) pair.left.getClass(), RegisterModbusAddress.of("register:1"), pair.left, pair.left, pair.left)
+                            .addItem((Class) pair.getLeft().getClass(), RegisterModbusAddress.of("register:1"), pair.getLeft(), pair.getLeft(), pair.getLeft())
                             .build(), new CompletableFuture<>()),
                     new ModbusTcpPayload((short) 0, (short) 0, new WriteMultipleCoilsResponse(1, 3))
                 ),
@@ -203,9 +174,9 @@ public class Plc4XModbusProtocolTest {
                     new PlcRequestContainer<>(
                         PlcWriteRequest
                             .builder()
-                            .addItem(RegisterModbusAddress.of("register:1"), pair.left)
+                            .addItem(RegisterModbusAddress.of("register:1"), pair.getLeft())
                             .build(), new CompletableFuture<>()),
-                    new ModbusTcpPayload((short) 0, (short) 0, new WriteSingleCoilResponse(1, pair.right[0]))
+                    new ModbusTcpPayload((short) 0, (short) 0, new WriteSingleCoilResponse(1, pair.getRight()[0]))
                 )
             ))
             .flatMap(stream -> stream)


[incubator-plc4x] 01/04: added sanity checks for produced values and cleaned up type checks

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

sruehl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git

commit 917c7fbb479c5f57ce75cc37bf4b5ddaa65fd850
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 19 11:14:45 2018 +0200

    added sanity checks for produced values and cleaned up type checks
---
 .../java/modbus/netty/Plc4XModbusProtocol.java     | 91 +++++++++++++---------
 1 file changed, 56 insertions(+), 35 deletions(-)

diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocol.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocol.java
index a12aca4..9a050f4 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocol.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocol.java
@@ -84,6 +84,10 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             RegisterModbusAddress registerModbusAddress = (RegisterModbusAddress) address;
             if (quantity > 1) {
                 byte[] bytesToWrite = produceRegisterValue(writeRequestItem.getValues());
+                int requiredLength = 2 * quantity;
+                if (bytesToWrite.length != requiredLength) {
+                    throw new PlcProtocolException("Invalid register values created. Should be at least quantity * 2 = N bytes. Was " + bytesToWrite.length + ", expected " + requiredLength);
+                }
                 modbusRequest = new WriteMultipleRegistersRequest(registerModbusAddress.getAddress(), quantity, bytesToWrite);
             } else {
                 byte[] register = produceRegisterValue(writeRequestItem.getValues());
@@ -94,6 +98,10 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             CoilModbusAddress coilModbusAddress = (CoilModbusAddress) address;
             if (quantity > 1) {
                 byte[] bytesToWrite = produceCoilValues(writeRequestItem.getValues());
+                int requiredLength = (quantity + 7) / 8;
+                if (bytesToWrite.length != requiredLength) {
+                    throw new PlcProtocolException("Invalid coil values created. Should be at least (quantity + 7) / 8 = N bytes. Was " + bytesToWrite.length + ", expected " + requiredLength);
+                }
                 modbusRequest = new WriteMultipleCoilsRequest(coilModbusAddress.getAddress(), quantity, bytesToWrite);
             } else {
                 boolean booleanToWrite = produceCoilValue(writeRequestItem.getValues());
@@ -167,7 +175,7 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
         // TODO: only single Item supported for now
         PlcRequest<?> request = plcRequestContainer.getRequest();
         RequestItem requestItem = request.getRequestItem().orElseThrow(() -> new PlcProtocolException("Only single message supported for now"));
-        Class datatype = requestItem.getDatatype();
+        Class<?> dataType = requestItem.getDatatype();
 
         ModbusPdu modbusPdu = msg.getModbusPdu();
         short unitId = msg.getUnitId();
@@ -197,14 +205,14 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             ReadCoilsResponse readCoilsResponse = (ReadCoilsResponse) modbusPdu;
             LOGGER.debug("{}: Nothing", readCoilsResponse);
             ByteBuf byteBuf = readCoilsResponse.getCoilStatus();
-            List data = produceCoilValueList(requestItem, datatype, byteBuf);
+            List<?> data = produceCoilValueList(requestItem, dataType, byteBuf);
             plcRequestContainer.getResponseFuture().complete(new PlcReadResponse((PlcReadRequest) request, new ReadResponseItem((ReadRequestItem) requestItem, ResponseCode.OK, data)));
         } else if (modbusPdu instanceof ReadDiscreteInputsResponse) {
             // TODO: finish implementation
             ReadDiscreteInputsResponse readDiscreteInputsResponse = (ReadDiscreteInputsResponse) modbusPdu;
             LOGGER.debug("{}: Nothing", readDiscreteInputsResponse);
             ByteBuf byteBuf = readDiscreteInputsResponse.getInputStatus();
-            List data = produceCoilValueList(requestItem, datatype, byteBuf);
+            List<?> data = produceCoilValueList(requestItem, dataType, byteBuf);
             plcRequestContainer.getResponseFuture().complete(new PlcReadResponse((PlcReadRequest) request, new ReadResponseItem((ReadRequestItem) requestItem, ResponseCode.OK, data)));
         } else if (modbusPdu instanceof ReadHoldingRegistersResponse) {
             // TODO: finish implementation
@@ -212,7 +220,7 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             LOGGER.debug("{}: Nothing", readHoldingRegistersResponse);
             ByteBuf byteBuf = readHoldingRegistersResponse.getRegisters();
             // TODO: use register method
-            List data = produceRegisterValueList(requestItem, datatype, byteBuf);
+            List<?> data = produceRegisterValueList(requestItem, dataType, byteBuf);
             plcRequestContainer.getResponseFuture().complete(new PlcReadResponse((PlcReadRequest) request, new ReadResponseItem((ReadRequestItem) requestItem, ResponseCode.OK, data)));
         } else if (modbusPdu instanceof ReadInputRegistersResponse) {
             // TODO: finish implementation
@@ -220,7 +228,7 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             LOGGER.debug("{}: Nothing", readInputRegistersResponse);
             ByteBuf byteBuf = readInputRegistersResponse.getRegisters();
             // TODO: use register method
-            List data = produceRegisterValueList(requestItem, datatype, byteBuf);
+            List<?> data = produceRegisterValueList(requestItem, dataType, byteBuf);
             plcRequestContainer.getResponseFuture().complete(new PlcReadResponse((PlcReadRequest) request, new ReadResponseItem((ReadRequestItem) requestItem, ResponseCode.OK, data)));
         } else if (modbusPdu instanceof MaskWriteRegisterResponse) {
             // TODO: finish implementation
@@ -317,7 +325,6 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
             // We only have one coil
             return new byte[]{actualCoil};
         }
-        // TODO: ensure we have a least (quantity + 7) / 8 = N bytes
         return ArrayUtils.toPrimitive(coils.toArray(new Byte[0]));
     }
 
@@ -345,7 +352,6 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
                 buffer.writeShort((int) value);
             }
         }
-        // TODO: ensure we have a least quantity * 2 = N bytes
         byte[] result = new byte[buffer.writerIndex()];
         buffer.readBytes(result);
         return result;
@@ -354,65 +360,80 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
     ////////////////////////////////////////////////////////////////////////////////
     // Decoding helpers.
     ////////////////////////////////////////////////////////////////////////////////
-
-    @SuppressWarnings("unchecked")
-    private List produceCoilValueList(RequestItem requestItem, Class datatype, ByteBuf byteBuf) {
+    private <T> List<T> produceCoilValueList(RequestItem requestItem, Class<T> dataType, ByteBuf byteBuf) {
         ReadRequestItem readRequestItem = (ReadRequestItem) requestItem;
         byte[] bytes = new byte[byteBuf.readableBytes()];
         if (bytes.length < 1) {
             return Collections.emptyList();
         }
         byteBuf.readBytes(bytes);
-        List data = new LinkedList();
+        List<T> data = new LinkedList<>();
         for (int i = 0, j = 0; i < readRequestItem.getSize(); i++) {
             if (i != 0 && i % 8 == 0) {
                 // Every 8 Coils we need to increase the access
                 j++;
             }
-            Boolean coilValue = (1 << i & bytes[j]) == 1;
-            if (datatype == Boolean.class) {
-                data.add(coilValue);
-            } else if (datatype == Byte.class) {
-                data.add((byte) (coilValue ? 1 : 0));
-            } else if (datatype == byte[].class) {
-                data.add(new byte[]{(byte) (coilValue ? 1 : 0)});
-            } else if (datatype == Short.class) {
-                data.add((short) (coilValue ? 1 : 0));
-            } else if (datatype == Integer.class) {
-                data.add(coilValue ? 1 : 0);
+            boolean coilSet = (1 << i & bytes[j]) == 1;
+            byte coilFlag = coilSet ? (byte) 1 : (byte) 0;
+            if (dataType == Boolean.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Boolean.valueOf(coilSet);
+                data.add(itemToBeAdded);
+            } else if (dataType == Byte.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Byte.valueOf(coilFlag);
+                data.add(itemToBeAdded);
+            } else if (dataType == byte[].class) {
+                data.add((T) new byte[]{coilFlag});
+            } else if (dataType == Short.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Short.valueOf(coilFlag);
+                data.add(itemToBeAdded);
+            } else if (dataType == Integer.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Integer.valueOf(coilFlag);
+                data.add(itemToBeAdded);
             }
         }
         return data;
     }
 
-    @SuppressWarnings("unchecked")
-    private List produceRegisterValueList(RequestItem requestItem, Class datatype, ByteBuf byteBuf) throws PlcProtocolException {
+    private <T> List<T> produceRegisterValueList(RequestItem requestItem, Class<T> dataType, ByteBuf byteBuf) throws PlcProtocolException {
         ReadRequestItem readRequestItem = (ReadRequestItem) requestItem;
         int readableBytes = byteBuf.readableBytes();
         if (readableBytes % 2 != 0) {
             throw new PlcProtocolException("Readables bytes should even: " + readableBytes);
         }
-        List data = new LinkedList();
+        List<T> data = new LinkedList<>();
         for (int i = 0; i < readRequestItem.getSize(); i++) {
             byte[] register = new byte[2];
             byteBuf.readBytes(register);
             int intValue = register[0] << 8 | register[1];
-            if (datatype == Boolean.class) {
-                data.add(intValue == 1);
-            } else if (datatype == Byte.class) {
+            if (dataType == Boolean.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Boolean.valueOf(intValue == 1);
+                data.add(itemToBeAdded);
+            } else if (dataType == Byte.class) {
                 if (intValue > Byte.MAX_VALUE) {
                     throw new PlcProtocolException("Value to high to fit into Byte: " + intValue);
                 }
-                data.add((byte) intValue);
-            } else if (datatype == byte[].class) {
-                data.add(register);
-            } else if (datatype == Short.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeADded = (T) Byte.valueOf((byte) intValue);
+                data.add(itemToBeADded);
+            } else if (dataType == byte[].class) {
+                T itemToBeAdded = (T) register;
+                data.add(itemToBeAdded);
+            } else if (dataType == Short.class) {
                 if (intValue > Short.MAX_VALUE) {
                     throw new PlcProtocolException("Value to high to fit into Short: " + intValue);
                 }
-                data.add((short) intValue);
-            } else if (datatype == Integer.class) {
-                data.add(intValue);
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Short.valueOf((short) intValue);
+                data.add(itemToBeAdded);
+            } else if (dataType == Integer.class) {
+                @SuppressWarnings("unchecked")
+                T itemToBeAdded = (T) Integer.valueOf(intValue);
+                data.add(itemToBeAdded);
             }
         }
         return data;


[incubator-plc4x] 03/04: create a blacklist of not yet supported types on modbus

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

sruehl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git

commit be1703c75252853e825e7ea60a9badce87a130a3
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 19 11:46:40 2018 +0200

    create a blacklist of not yet supported types on modbus
---
 .../java/modbus/netty/Plc4XModbusProtocolTest.java | 30 +++++++++++++++++-----
 1 file changed, 23 insertions(+), 7 deletions(-)

diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
index 5381702..6aa5863 100644
--- a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/netty/Plc4XModbusProtocolTest.java
@@ -33,7 +33,9 @@ import org.apache.plc4x.java.api.messages.items.WriteResponseItem;
 import org.apache.plc4x.java.api.types.ResponseCode;
 import org.apache.plc4x.java.modbus.model.*;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.slf4j.Logger;
@@ -41,9 +43,7 @@ import org.slf4j.LoggerFactory;
 
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.Calendar;
-import java.util.Collection;
+import java.util.*;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.Collectors;
@@ -53,6 +53,7 @@ import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
 import static org.hamcrest.Matchers.*;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
+import static org.junit.Assume.assumeThat;
 
 @RunWith(Parameterized.class)
 public class Plc4XModbusProtocolTest {
@@ -63,6 +64,16 @@ public class Plc4XModbusProtocolTest {
 
     private Plc4XModbusProtocol SUT;
 
+    @Rule
+    public ExpectedException expectedException = ExpectedException.none();
+    // TODO: implement these types
+    private List<String> notYetSupportedDataType = Stream.of(
+        GregorianCalendar.class,
+        Float.class,
+        Double.class,
+        String.class
+    ).map(Class::getSimpleName).collect(Collectors.toList());
+
     @Parameterized.Parameter
     public String payloadClazzName;
 
@@ -87,10 +98,11 @@ public class Plc4XModbusProtocolTest {
             Boolean.class,
             Byte.class,
             Short.class,
-            //Calendar.class,
-            //Float.class,
-            Integer.class //,
-            //String.class
+            Calendar.class,
+            Float.class,
+            Double.class,
+            Integer.class,
+            String.class
         )
             .map(clazz -> {
                 if (clazz == Boolean.class) {
@@ -103,6 +115,8 @@ public class Plc4XModbusProtocolTest {
                     return ImmutablePair.of(calenderInstance, new byte[]{0x0, 0x0, 0x0, 0x0, 0x4, 0x3, 0x2, 0x1});
                 } else if (clazz == Float.class) {
                     return ImmutablePair.of(Float.valueOf("1"), new byte[]{0x0, 0x0, (byte) 0x80, 0x3F});
+                } else if (clazz == Double.class) {
+                    return ImmutablePair.of(Double.valueOf("1"), new byte[]{0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F});
                 } else if (clazz == Integer.class) {
                     return ImmutablePair.of(Integer.valueOf("1"), new byte[]{0x1, 0x0, 0x0, 0x0});
                 } else if (clazz == String.class) {
@@ -213,6 +227,7 @@ public class Plc4XModbusProtocolTest {
 
     @Test
     public void encode() throws Exception {
+        assumeThat(payloadClazzName + " not yet implemented", notYetSupportedDataType, not(hasItem(payloadClazzName)));
         ArrayList<Object> out = new ArrayList<>();
         SUT.encode(null, plcRequestContainer, out);
         assertThat(out, hasSize(1));
@@ -361,6 +376,7 @@ public class Plc4XModbusProtocolTest {
 
     @Test
     public void decode() throws Exception {
+        assumeThat(payloadClazzName + " not yet implemented", notYetSupportedDataType, not(hasItem(payloadClazzName)));
         ArrayList<Object> in = new ArrayList<>();
         SUT.encode(null, plcRequestContainer, in);
         assertThat(in, hasSize(1));


[incubator-plc4x] 02/04: added tests for double data type in ads

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

sruehl pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-plc4x.git

commit c4a35b3cdb0b4e23795f8881ed64a632b08194d6
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 19 11:32:49 2018 +0200

    added tests for double data type in ads
---
 .../org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java  | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
index 226bbde..4d456bd 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Plc4x2AdsProtocolTest.java
@@ -92,6 +92,7 @@ public class Plc4x2AdsProtocolTest {
             Short.class,
             Calendar.class,
             Float.class,
+            Double.class,
             Integer.class,
             String.class)
             .map(clazz -> {
@@ -105,6 +106,8 @@ public class Plc4x2AdsProtocolTest {
                     return ImmutablePair.of(calenderInstance, new byte[]{0x0, 0x0, 0x0, 0x0, 0x4, 0x3, 0x2, 0x1});
                 } else if (clazz == Float.class) {
                     return ImmutablePair.of(Float.valueOf("1"), new byte[]{0x0, 0x0, (byte) 0x80, 0x3F});
+                } else if (clazz == Double.class) {
+                    return ImmutablePair.of(Double.valueOf("1"), new byte[]{0x0, 0x0,0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F});
                 } else if (clazz == Integer.class) {
                     return ImmutablePair.of(Integer.valueOf("1"), new byte[]{0x1, 0x0, 0x0, 0x0});
                 } else if (clazz == String.class) {
@@ -165,6 +168,8 @@ public class Plc4x2AdsProtocolTest {
                 assertThat(value, equalTo(new byte[]{0x0}));
             } else if (payloadClazzName.equals(Float.class.getSimpleName())) {
                 assertThat(value, equalTo(new byte[]{0x0, 0x0, (byte) 0x80, 0x3F}));
+            } else if (payloadClazzName.equals(Double.class.getSimpleName())) {
+                assertThat(value, equalTo(new byte[]{0x0, 0x0, 0x0, 0x0, 0x0, 0x0, (byte) 0xF0, 0x3F}));
             } else if (payloadClazzName.equals(Integer.class.getSimpleName())) {
                 assertThat(value, equalTo(new byte[]{0x1, 0x0, 0x0, 0x0}));
             } else if (payloadClazzName.equals(String.class.getSimpleName())) {
@@ -200,6 +205,8 @@ public class Plc4x2AdsProtocolTest {
                 assertThat(value, equalTo(calenderInstance));
             } else if (payloadClazzName.equals(Float.class.getSimpleName())) {
                 assertThat(value, equalTo(Float.valueOf("1")));
+            } else if (payloadClazzName.equals(Double.class.getSimpleName())) {
+                assertThat(value, equalTo(Double.valueOf("1")));
             } else if (payloadClazzName.equals(Integer.class.getSimpleName())) {
                 assertThat(value, equalTo(Integer.valueOf("1")));
             } else if (payloadClazzName.equals(String.class.getSimpleName())) {