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/05 12:32:46 UTC

[incubator-plc4x] branch master updated (5714e07 -> 036e394)

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 5714e07  Added some more information about the wago web-interface and the default login to this.
     add 369ee00  small fixes in the modbus protocol
     new f95d70f  Introduced TcpHexDumper to driver-bases/test
     new 492ed4a  added test for modbus PlcDriver
     new 1c65826  moved Assert and Junit5Backport to driver-bases/test
     new c326e36  added modbus connection tests
     new 036e394  fixed build problems

The 5 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:
 plc4j/protocols/ads/pom.xml                        |   6 +
 .../java/org/apache/plc4x/java/ads/AdsDumper.java  |   2 +-
 .../apache/plc4x/java/ads/AdsPlcDriverTest.java    |   2 +-
 .../commands/types/AdsNotificationSampleTest.java  |   2 +-
 .../ads/api/commands/types/AdsStampHeaderTest.java |   2 +-
 .../java/ads/api/commands/types/LengthTest.java    |   2 +-
 .../ads/api/commands/types/ReadLengthTest.java     |   2 +-
 .../ads/api/commands/types/SampleSizeTest.java     |   2 +-
 .../java/ads/api/commands/types/SamplesTest.java   |   2 +-
 .../ads/api/commands/types/WriteLengthTest.java    |   2 +-
 .../java/ads/api/generic/types/AmsPortTest.java    |   2 +-
 .../java/ads/api/generic/types/DataLengthTest.java |   2 +-
 .../java/ads/api/generic/types/TcpLengthTest.java  |   2 +-
 .../connection/AdsAbstractPlcConnectionTest.java   |   2 +-
 .../ads/connection/AdsConnectionFactoryTest.java   |   2 +-
 ...load2SerialProtocolExampleConversationTest.java |   2 +-
 .../java/ads/protocol/util/DigestUtilTest.java     |   2 +-
 .../ads/protocol/util/LittleEndianDecoderTest.java |   2 +-
 .../ads/protocol/util/LittleEndianEncoderTest.java |   4 +-
 plc4j/protocols/driver-bases/test/pom.xml          |  33 ++++
 .../java/base/connection/tcp}/TcpHexDumper.java    |   2 +-
 .../org/apache/plc4x/java/base}/util/Assert.java   |  22 ++-
 .../plc4x/java/base}/util/Junit5Backport.java      |   2 +-
 plc4j/protocols/modbus/pom.xml                     |  16 ++
 .../apache/plc4x/java/modbus/ModbusPlcDriver.java  |  34 +++--
 .../modbus/connection/BaseModbusPlcConnection.java |  24 +--
 .../connection/ModbusConnectionFactory.java}       |  32 ++--
 .../connection/ModbusSerialPlcConnection.java      |   7 +-
 .../modbus/connection/ModbusTcpPlcConnection.java  |  14 +-
 .../{CoilAddress.java => CoilModbusAddress.java}   |   8 +-
 .../java/modbus/netty/Plc4XModbusProtocol.java     |  22 +--
 .../plc4x/java/modbus/ModbusPlcDriverTest.java     | 125 ++++++++++++++++
 .../connection/BaseModbusPlcConnectionTest.java    | 166 +++++++++++++++++++++
 .../connection/ModbusConnectionFactoryTest.java    |  95 ++++++++++++
 .../connection/ModbusSerialPlcConnectionTest.java} | 121 ++++++++-------
 .../connection/ModbusTcpPlcConnectionTests.java    | 134 +++++++++++++++++
 36 files changed, 760 insertions(+), 141 deletions(-)
 rename plc4j/protocols/{ads/src/test/java/org/apache/plc4x/java/ads/util => driver-bases/test/src/main/java/org/apache/plc4x/java/base/connection/tcp}/TcpHexDumper.java (98%)
 rename plc4j/protocols/{ads/src/test/java/org/apache/plc4x/java/ads => driver-bases/test/src/main/java/org/apache/plc4x/java/base}/util/Assert.java (80%)
 copy plc4j/protocols/{ads/src/test/java/org/apache/plc4x/java/ads => driver-bases/test/src/main/java/org/apache/plc4x/java/base}/util/Junit5Backport.java (97%)
 rename plc4j/protocols/{ads/src/test/java/org/apache/plc4x/java/ads/util/Junit5Backport.java => modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactory.java} (53%)
 rename plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/{CoilAddress.java => CoilModbusAddress.java} (86%)
 create mode 100644 plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/ModbusPlcDriverTest.java
 create mode 100644 plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnectionTest.java
 create mode 100644 plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactoryTest.java
 copy plc4j/protocols/{ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsSerialPlcConnectionTest.java => modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnectionTest.java} (72%)
 create mode 100644 plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnectionTests.java


[incubator-plc4x] 04/05: added modbus connection tests

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 c326e36d5ca3b038226c7e924e1678ecf286eeb1
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 5 14:18:33 2018 +0200

    added modbus connection tests
---
 .../modbus/connection/BaseModbusPlcConnection.java |  24 +-
 .../modbus/connection/ModbusTcpPlcConnection.java  |   4 +
 .../{CoilAddress.java => CoilModbusAddress.java}   |   8 +-
 .../java/modbus/netty/Plc4XModbusProtocol.java     |  14 +-
 .../connection/BaseModbusPlcConnectionTest.java    | 166 ++++++++++++
 .../connection/ModbusConnectionFactoryTest.java    |  95 +++++++
 .../connection/ModbusSerialPlcConnectionTest.java  | 286 +++++++++++++++++++++
 .../connection/ModbusTcpPlcConnectionTests.java    | 134 ++++++++++
 8 files changed, 711 insertions(+), 20 deletions(-)

diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnection.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnection.java
index 3c65ee6..0056363 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnection.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnection.java
@@ -18,6 +18,7 @@ under the License.
 */
 package org.apache.plc4x.java.modbus.connection;
 
+import io.netty.channel.ChannelFuture;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.plc4x.java.api.connection.PlcReader;
 import org.apache.plc4x.java.api.connection.PlcWriter;
@@ -65,8 +66,8 @@ public abstract class BaseModbusPlcConnection extends AbstractPlcConnection impl
             return ReadHoldingRegistersModbusAddress.of(addressString);
         } else if (ReadInputRegistersModbusAddress.ADDRESS_PATTERN.matcher(addressString).matches()) {
             return ReadInputRegistersModbusAddress.of(addressString);
-        } else if (CoilAddress.ADDRESS_PATTERN.matcher(addressString).matches()) {
-            return CoilAddress.of(addressString);
+        } else if (CoilModbusAddress.ADDRESS_PATTERN.matcher(addressString).matches()) {
+            return CoilModbusAddress.of(addressString);
         } else if (RegisterAddress.ADDRESS_PATTERN.matcher(addressString).matches()) {
             return RegisterAddress.of(addressString);
         }
@@ -76,19 +77,24 @@ public abstract class BaseModbusPlcConnection extends AbstractPlcConnection impl
     @Override
     public CompletableFuture<PlcReadResponse> read(PlcReadRequest readRequest) {
         CompletableFuture<PlcReadResponse> readFuture = new CompletableFuture<>();
-        PlcRequestContainer<PlcReadRequest, PlcReadResponse> container =
-            new PlcRequestContainer<>(readRequest, readFuture);
-        channel.writeAndFlush(container);
+        ChannelFuture channelFuture = channel.writeAndFlush(new PlcRequestContainer<>(readRequest, readFuture));
+        channelFuture.addListener(future -> {
+            if (!future.isSuccess()) {
+                readFuture.completeExceptionally(future.cause());
+            }
+        });
         return readFuture;
     }
 
     @Override
     public CompletableFuture<PlcWriteResponse> write(PlcWriteRequest writeRequest) {
         CompletableFuture<PlcWriteResponse> writeFuture = new CompletableFuture<>();
-        PlcRequestContainer<PlcWriteRequest, PlcWriteResponse> container =
-            new PlcRequestContainer<>(writeRequest, writeFuture);
-        channel.writeAndFlush(container);
+        ChannelFuture channelFuture = channel.writeAndFlush(new PlcRequestContainer<>(writeRequest, writeFuture));
+        channelFuture.addListener(future -> {
+            if (!future.isSuccess()) {
+                writeFuture.completeExceptionally(future.cause());
+            }
+        });
         return writeFuture;
     }
-
 }
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
index c418031..92f5337 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
@@ -71,4 +71,8 @@ public class ModbusTcpPlcConnection extends BaseModbusPlcConnection {
             }
         };
     }
+
+    public InetAddress getRemoteAddress() {
+        return ((TcpSocketChannelFactory) channelFactory).getAddress();
+    }
 }
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilAddress.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilModbusAddress.java
similarity index 86%
rename from plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilAddress.java
rename to plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilModbusAddress.java
index fb83da9..414c968 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilAddress.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/model/CoilModbusAddress.java
@@ -23,20 +23,20 @@ import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-public class CoilAddress extends ModbusAddress {
+public class CoilModbusAddress extends ModbusAddress {
 
     public static final Pattern ADDRESS_PATTERN = Pattern.compile("coil:" + ModbusAddress.ADDRESS_PATTERN);
 
-    protected CoilAddress(int address) {
+    protected CoilModbusAddress(int address) {
         super(address);
     }
 
-    public static CoilAddress of(String addressString) {
+    public static CoilModbusAddress of(String addressString) {
         Matcher matcher = ADDRESS_PATTERN.matcher(addressString);
         if (!matcher.matches()) {
             throw new PlcRuntimeException(addressString + " doesn't match" + ADDRESS_PATTERN);
         }
         int address = Integer.valueOf(matcher.group("address"));
-        return new CoilAddress(address);
+        return new CoilModbusAddress(address);
     }
 }
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 288cda7..8f751fa 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
@@ -89,15 +89,15 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
                 int intToWrite = register[0] << 8 | register[1];
                 modbusRequest = new WriteSingleRegisterRequest(registerAddress.getAddress(), intToWrite);
             }
-        } else if (address instanceof CoilAddress) {
-            CoilAddress coilAddress = (CoilAddress) address;
+        } else if (address instanceof CoilModbusAddress) {
+            CoilModbusAddress coilModbusAddress = (CoilModbusAddress) address;
             if (quantity > 1) {
                 byte[] bytesToWrite = produceCoilValue(writeRequestItem.getValues());
-                modbusRequest = new WriteMultipleCoilsRequest(coilAddress.getAddress(), quantity, bytesToWrite);
+                modbusRequest = new WriteMultipleCoilsRequest(coilModbusAddress.getAddress(), quantity, bytesToWrite);
             } else {
                 byte[] coil = produceCoilValue(writeRequestItem.getValues());
                 boolean booleanToWrite = (coil[0] >> 8) == 1;
-                modbusRequest = new WriteSingleCoilRequest(coilAddress.getAddress(), booleanToWrite);
+                modbusRequest = new WriteSingleCoilRequest(coilModbusAddress.getAddress(), booleanToWrite);
             }
         } else {
             throw new PlcProtocolException("Unsupported address type" + address.getClass());
@@ -119,9 +119,9 @@ public class Plc4XModbusProtocol extends MessageToMessageCodec<ModbusTcpPayload,
 
         ModbusAddress address = (ModbusAddress) readRequestItem.getAddress();
         ModbusPdu modbusRequest;
-        if (address instanceof CoilAddress) {
-            CoilAddress coilAddress = (CoilAddress) address;
-            modbusRequest = new ReadCoilsRequest(coilAddress.getAddress(), quantity);
+        if (address instanceof CoilModbusAddress) {
+            CoilModbusAddress coilModbusAddress = (CoilModbusAddress) address;
+            modbusRequest = new ReadCoilsRequest(coilModbusAddress.getAddress(), quantity);
         } else if (address instanceof RegisterAddress) {
             RegisterAddress registerAddress = (RegisterAddress) address;
             modbusRequest = new ReadHoldingRegistersRequest(registerAddress.getAddress(), quantity);
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnectionTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnectionTest.java
new file mode 100644
index 0000000..9c00a4b
--- /dev/null
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/BaseModbusPlcConnectionTest.java
@@ -0,0 +1,166 @@
+/*
+ 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.connection;
+
+import io.netty.channel.Channel;
+import io.netty.channel.ChannelFuture;
+import io.netty.channel.ChannelHandler;
+import io.netty.util.concurrent.Future;
+import io.netty.util.concurrent.GenericFutureListener;
+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.messages.specific.TypeSafePlcReadRequest;
+import org.apache.plc4x.java.api.messages.specific.TypeSafePlcReadResponse;
+import org.apache.plc4x.java.api.messages.specific.TypeSafePlcWriteRequest;
+import org.apache.plc4x.java.api.messages.specific.TypeSafePlcWriteResponse;
+import org.apache.plc4x.java.base.connection.ChannelFactory;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.Answers;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+@RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("unchecked")
+public class BaseModbusPlcConnectionTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(BaseModbusPlcConnectionTest.class);
+
+    private BaseModbusPlcConnection SUT;
+
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private ChannelFactory channelFactory;
+
+    @Mock(answer = Answers.RETURNS_DEEP_STUBS)
+    private Channel channel;
+
+    @Before
+    public void setUp() throws Exception {
+        SUT = new BaseModbusPlcConnection(channelFactory, null) {
+            @Override
+            protected ChannelHandler getChannelHandler(CompletableFuture<Void> sessionSetupCompleteFuture) {
+                return null;
+            }
+        };
+
+        when(channelFactory.createChannel(any())).thenReturn(channel);
+
+        SUT.connect();
+    }
+
+    @Test
+    public void lazyConstructor() {
+        new BaseModbusPlcConnection(channelFactory, null) {
+            @Override
+            protected ChannelHandler getChannelHandler(CompletableFuture<Void> sessionSetupCompleteFuture) {
+                return null;
+            }
+        };
+    }
+
+    @Test
+    public void read() {
+        CompletableFuture<PlcReadResponse> read = SUT.read(mock(PlcReadRequest.class));
+        assertNotNull(read);
+        CompletableFuture<TypeSafePlcReadResponse<Object>> typeSafeRead = SUT.read(mock(TypeSafePlcReadRequest.class));
+        assertNotNull(typeSafeRead);
+
+        simulatePipelineError(() -> SUT.read(mock(PlcReadRequest.class)));
+        simulatePipelineError(() -> SUT.read(mock(TypeSafePlcReadRequest.class)));
+    }
+
+    @Test
+    public void write() {
+        CompletableFuture<PlcWriteResponse> write = SUT.write(mock(PlcWriteRequest.class));
+        assertNotNull(write);
+        CompletableFuture<TypeSafePlcWriteResponse<Object>> typeSafeWrite = SUT.write(mock(TypeSafePlcWriteRequest.class));
+        assertNotNull(typeSafeWrite);
+
+        simulatePipelineError(() -> SUT.write(mock(PlcWriteRequest.class)));
+        simulatePipelineError(() -> SUT.write(mock(TypeSafePlcWriteRequest.class)));
+    }
+
+    public void simulatePipelineError(FutureProducingTestRunnable futureProducingTestRunnable) {
+        ChannelFuture channelFuture = mock(ChannelFuture.class);
+        // Simulate error in the pipeline
+        when(channelFuture.addListener(any())).thenAnswer(invocation -> {
+            Future future = mock(Future.class);
+            when(future.isSuccess()).thenReturn(false);
+            when(future.cause()).thenReturn(new DummyException());
+            GenericFutureListener genericFutureListener = invocation.getArgument(0);
+            genericFutureListener.operationComplete(future);
+            return mock(ChannelFuture.class);
+        });
+        when(channel.writeAndFlush(any())).thenReturn(channelFuture);
+        assertThrows(DummyException.class, () -> {
+            CompletableFuture completableFuture = futureProducingTestRunnable.run();
+            try {
+                completableFuture.get(3, TimeUnit.SECONDS);
+                fail("Should have thrown a ExecutionException");
+            } catch (ExecutionException e) {
+                if (e.getCause() instanceof DummyException) {
+                    throw (DummyException) e.getCause();
+                }
+                throw e;
+            }
+        });
+    }
+
+    @Test
+    public void testToString() {
+        String s = SUT.toString();
+        assertNotNull(s);
+    }
+
+    /**
+     * Variant of {@link Runnable} which adds a {@code throws Exception} to the {@code run} signature.
+     */
+    private interface TestRunnable {
+        /**
+         * @throws Exception when the test throws a exception.
+         * @see Runnable#run()
+         */
+        void run() throws Exception;
+    }
+
+    private static class DummyException extends Exception {
+
+    }
+
+    @FunctionalInterface
+    private interface FutureProducingTestRunnable {
+        CompletableFuture run() throws Exception;
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactoryTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactoryTest.java
new file mode 100644
index 0000000..c4e3711
--- /dev/null
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactoryTest.java
@@ -0,0 +1,95 @@
+/*
+ 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.connection;
+
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.plc4x.java.base.connection.AbstractPlcConnection;
+import org.apache.plc4x.java.base.connection.SerialChannelFactory;
+import org.apache.plc4x.java.base.connection.TcpSocketChannelFactory;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.mockito.InjectMocks;
+import org.mockito.Mock;
+import org.mockito.junit.MockitoJUnitRunner;
+
+import java.net.InetAddress;
+
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
+import static org.junit.Assert.assertEquals;
+
+@RunWith(MockitoJUnitRunner.class)
+public class ModbusConnectionFactoryTest {
+
+    @InjectMocks
+    private ModbusConnectionFactory SUT;
+
+    @Mock
+    private InetAddress inetAddress;
+
+
+    @Test
+    public void modbusTcpPlcConnectionOf() throws Exception {
+        {
+            assertThrows(NullPointerException.class, () -> SUT.modbusTcpPlcConnectionOf(null, null, null));
+        }
+        {
+            ModbusTcpPlcConnection modbusTcpPlcConnection = SUT.modbusTcpPlcConnectionOf(inetAddress, null, null);
+            assertGeneratedPort(modbusTcpPlcConnection);
+        }
+        {
+            ModbusTcpPlcConnection modbusTcpPlcConnection = SUT.modbusTcpPlcConnectionOf(inetAddress, 13, null);
+            assertEquals(inetAddress, modbusTcpPlcConnection.getRemoteAddress());
+            assertPort(modbusTcpPlcConnection, 13);
+        }
+        {
+            ModbusTcpPlcConnection modbusTcpPlcConnection = SUT.modbusTcpPlcConnectionOf(inetAddress, null, "xyz");
+            assertEquals(inetAddress, modbusTcpPlcConnection.getRemoteAddress());
+            assertGeneratedPort(modbusTcpPlcConnection);
+        }
+    }
+
+    public void assertGeneratedPort(ModbusTcpPlcConnection modbusTcpPlcConnection) throws Exception {
+        assertPort(modbusTcpPlcConnection, 502);
+    }
+
+    public void assertPort(ModbusTcpPlcConnection modbusTcpPlcConnection, int port) throws Exception {
+        TcpSocketChannelFactory channelFactory = (TcpSocketChannelFactory) FieldUtils
+            .getDeclaredField(AbstractPlcConnection.class, "channelFactory", true)
+            .get(modbusTcpPlcConnection);
+        assertEquals(port, channelFactory.getPort());
+    }
+
+    @Test
+    public void modbusSerialPlcConnectionOf() throws Exception {
+        {
+            assertThrows(NullPointerException.class, () -> SUT.modbusSerialPlcConnectionOf(null, null));
+        }
+        {
+            ModbusSerialPlcConnection modbusSerialPlcConnection = SUT.modbusSerialPlcConnectionOf("/dev/ttyS01", null);
+            assertPort(modbusSerialPlcConnection, "/dev/ttyS01");
+        }
+    }
+
+    public void assertPort(ModbusSerialPlcConnection modbusSerialPlcConnection, String serialPort) throws Exception {
+        SerialChannelFactory channelFactory = (SerialChannelFactory) FieldUtils
+            .getDeclaredField(AbstractPlcConnection.class, "channelFactory", true)
+            .get(modbusSerialPlcConnection);
+        assertEquals(serialPort, channelFactory.getSerialPort());
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnectionTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnectionTest.java
new file mode 100644
index 0000000..4339c42
--- /dev/null
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnectionTest.java
@@ -0,0 +1,286 @@
+/*
+ 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.connection;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.channel.embedded.EmbeddedChannel;
+import io.netty.channel.jsc.JSerialCommDeviceAddress;
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.commons.lang3.reflect.MethodUtils;
+import org.apache.plc4x.java.api.messages.PlcReadRequest;
+import org.apache.plc4x.java.api.messages.PlcReadResponse;
+import org.apache.plc4x.java.base.connection.AbstractPlcConnection;
+import org.apache.plc4x.java.base.connection.SerialChannelFactory;
+import org.apache.plc4x.java.modbus.model.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.util.Arrays;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.spy;
+
+@Ignore("Not yet implemented in modbus")
+public class ModbusSerialPlcConnectionTest {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ModbusSerialPlcConnectionTest.class);
+
+    private ModbusSerialPlcConnection SUT;
+
+    @Before
+    public void setUp() {
+        SUT = ModbusSerialPlcConnection.of("/dev/tty0", null);
+    }
+
+    @After
+    public void tearDown() {
+        SUT = null;
+    }
+
+    @Test
+    public void emptyParseAddress() {
+        try {
+            SUT.parseAddress("");
+        } catch (IllegalArgumentException exception) {
+            assertTrue("Unexpected exception", exception.getMessage().startsWith("address  doesn't match "));
+        }
+    }
+
+    @Test
+    public void parseCoilModbusAddress() {
+        try {
+            CoilModbusAddress address = (CoilModbusAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseMaskWriteRegisterModbusAddress() {
+        try {
+            MaskWriteRegisterModbusAddress address = (MaskWriteRegisterModbusAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadDiscreteInputsModbusAddress() {
+        try {
+            ReadDiscreteInputsModbusAddress address = (ReadDiscreteInputsModbusAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadHoldingRegistersModbusAddress() {
+        try {
+            ReadHoldingRegistersModbusAddress address = (ReadHoldingRegistersModbusAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadInputRegistersModbusAddress() {
+        try {
+            ReadInputRegistersModbusAddress address = (ReadInputRegistersModbusAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseRegisterAddress() {
+        try {
+            RegisterAddress address = (RegisterAddress) SUT.parseAddress("0/1");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void testRead() throws Exception {
+        prepareSerialSimulator();
+        CompletableFuture<PlcReadResponse> read = SUT.read(new PlcReadRequest(String.class, SUT.parseAddress("0/0")));
+        PlcReadResponse plcReadResponse = read.get(30, TimeUnit.SECONDS);
+        assertNotNull(plcReadResponse);
+    }
+
+    private void prepareSerialSimulator() throws Exception {
+        Field channelFactoryField = FieldUtils.getField(AbstractPlcConnection.class, "channelFactory", true);
+        SerialChannelFactory serialChannelFactory = (SerialChannelFactory) channelFactoryField.get(SUT);
+        SerialChannelFactory serialChannelFactorySpied = spy(serialChannelFactory);
+        EmbeddedChannel embeddedChannel = new EmbeddedChannel(SUT.getChannelHandler(null));
+        embeddedChannel.connect(new JSerialCommDeviceAddress("/dev/tty0"));
+        doReturn(embeddedChannel).when(serialChannelFactorySpied).createChannel(any());
+        channelFactoryField.set(SUT, serialChannelFactorySpied);
+        SUT.connect();
+        new SerialSimulator(embeddedChannel).start();
+    }
+
+    private class SerialSimulator extends Thread {
+
+        private EmbeddedChannel embeddedChannel;
+
+        private SimulatorState state = SimulatorState.RECEIVE_REQUEST;
+
+        private byte[] currentInvokeId = new byte[0];
+
+        public SerialSimulator(EmbeddedChannel embeddedChannel) {
+            super("Serial Simulator");
+            this.embeddedChannel = embeddedChannel;
+        }
+
+        @Override
+        public void run() {
+            while (true) {
+                LOGGER.trace("in state {}. CurrentInvokeId: {}", state, currentInvokeId);
+                switch (state) {
+                    // Receiving state
+                    case RECEIVE_REQUEST: {
+                        LOGGER.info("Waiting for normal message");
+                        ByteBuf outputBuffer;
+                        while ((outputBuffer = embeddedChannel.readOutbound()) == null) {
+                            LOGGER.trace("No buffer available yet");
+                            if (!trySleep()) {
+                                return;
+                            }
+                        }
+                        // TODO
+                        int headerBytes = 4711;
+                        LOGGER.info("Skipping " + headerBytes + " bytes");
+                        outputBuffer.skipBytes(headerBytes);
+                        short dataLength = outputBuffer.readUnsignedByte();
+                        LOGGER.info("Expect at least " + dataLength + "bytes");
+                        while (outputBuffer.readableBytes() < dataLength) {
+                            if (!trySleep()) {
+                                return;
+                            }
+                        }
+                        byte[] bytes = new byte[dataLength];
+                        LOGGER.info("Read " + dataLength + "bytes. Having " + outputBuffer.readableBytes() + "bytes");
+                        outputBuffer.readBytes(bytes);
+                        currentInvokeId = Arrays.copyOfRange(bytes, 28, 32);
+                        // TODO
+                        outputBuffer.skipBytes(4711);
+                        LOGGER.info("Wrote Inbound");
+                        state = SimulatorState.ACK_MESSAGE;
+                        if (!trySleep()) {
+                            return;
+                        }
+                    }
+                    break;
+                    case ACK_MESSAGE: {
+                        // TODO
+                        ByteBuf byteBuf = Unpooled.buffer();
+                        try {
+                            MethodUtils.invokeMethod(byteBuf, true, "setRefCnt", 2);
+                        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+                            throw new RuntimeException(e);
+                        }
+                        embeddedChannel.writeOneInbound(byteBuf);
+                        LOGGER.info("Acked Message");
+                        state = SimulatorState.SEND_RESPONSE;
+                    }
+                    case SEND_RESPONSE: {
+                        LOGGER.info("Sending data message");
+                        //TODO:
+                        ByteBuf byteBuf = Unpooled.buffer();
+                        try {
+                            MethodUtils.invokeMethod(byteBuf, true, "setRefCnt", 2);
+                        } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+                            throw new RuntimeException(e);
+                        }
+                        embeddedChannel.writeOneInbound(byteBuf);
+                        LOGGER.info("Wrote Inbound");
+                        state = SimulatorState.WAIT_FOR_ACK;
+                        if (!trySleep()) {
+                            return;
+                        }
+                    }
+                    break;
+                    case WAIT_FOR_ACK: {
+                        LOGGER.info("Waiting for ack message");
+                        ByteBuf outputBuffer;
+                        while ((outputBuffer = embeddedChannel.readOutbound()) == null) {
+                            if (!trySleep()) {
+                                return;
+                            }
+                        }
+                        //TODO:
+                        int headerBytes = 4711;
+                        LOGGER.info("Skipping " + headerBytes + " bytes");
+                        outputBuffer.skipBytes(headerBytes);
+                        short dataLength = outputBuffer.readUnsignedByte();
+                        LOGGER.info("Expect " + dataLength + "bytes");
+                        state = SimulatorState.DONE;
+                        if (!trySleep()) {
+                            return;
+                        }
+                    }
+                    case DONE: {
+                        LOGGER.info("Plc is Done. Goodbye");
+                        return;
+                    }
+                    default:
+                        throw new IllegalStateException("Illegal state number" + state);
+                }
+            }
+
+        }
+
+        private boolean trySleep() {
+            try {
+                TimeUnit.MILLISECONDS.sleep(10);
+            } catch (InterruptedException e) {
+                e.printStackTrace();
+                Thread.currentThread().interrupt();
+                return false;
+            }
+            return true;
+        }
+    }
+
+    private enum SimulatorState {
+        RECEIVE_REQUEST,
+        ACK_MESSAGE,
+        SEND_RESPONSE,
+        WAIT_FOR_ACK,
+        DONE
+    }
+}
\ No newline at end of file
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnectionTests.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnectionTests.java
new file mode 100644
index 0000000..10293e3
--- /dev/null
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnectionTests.java
@@ -0,0 +1,134 @@
+/*
+ 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.connection;
+
+import io.netty.channel.Channel;
+import org.apache.commons.lang3.reflect.FieldUtils;
+import org.apache.plc4x.java.modbus.model.*;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.InetAddress;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.mock;
+
+public class ModbusTcpPlcConnectionTests {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(ModbusTcpPlcConnectionTests.class);
+
+    private ModbusTcpPlcConnection SUT;
+
+    private Channel channelMock;
+
+    private ExecutorService executorService;
+
+    @Before
+    public void setUp() throws Exception {
+        SUT = ModbusTcpPlcConnection.of(InetAddress.getByName("localhost"), null);
+        channelMock = mock(Channel.class, RETURNS_DEEP_STUBS);
+        FieldUtils.writeField(SUT, "channel", channelMock, true);
+        executorService = Executors.newFixedThreadPool(10);
+    }
+
+    @After
+    public void tearDown() {
+        executorService.shutdownNow();
+        SUT = null;
+    }
+
+    @Test
+    public void emptyParseAddress() {
+        try {
+            SUT.parseAddress("");
+        } catch (IllegalArgumentException exception) {
+            assertTrue("Unexpected exception", exception.getMessage().startsWith("address  doesn't match "));
+        }
+    }
+
+    @Test
+    public void parseCoilModbusAddress() {
+        try {
+            CoilModbusAddress address = (CoilModbusAddress) SUT.parseAddress("coil:0");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseMaskWriteRegisterModbusAddress() {
+        try {
+            MaskWriteRegisterModbusAddress address = (MaskWriteRegisterModbusAddress) SUT.parseAddress("maskwrite:1/2/3");
+            assertEquals(address.getAddress(), 1);
+            assertEquals(address.getAndMask(), 2);
+            assertEquals(address.getOrMask(), 3);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadDiscreteInputsModbusAddress() {
+        try {
+            ReadDiscreteInputsModbusAddress address = (ReadDiscreteInputsModbusAddress) SUT.parseAddress("readdiscreteinputs:0");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadHoldingRegistersModbusAddress() {
+        try {
+            ReadHoldingRegistersModbusAddress address = (ReadHoldingRegistersModbusAddress) SUT.parseAddress("readholdingregisters:0");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseReadInputRegistersModbusAddress() {
+        try {
+            ReadInputRegistersModbusAddress address = (ReadInputRegistersModbusAddress) SUT.parseAddress("readinputregisters:0");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+    @Test
+    public void parseRegisterAddress() {
+        try {
+            RegisterAddress address = (RegisterAddress) SUT.parseAddress("register:0");
+            assertEquals(address.getAddress(), 0);
+        } catch (IllegalArgumentException exception) {
+            fail("valid data block address");
+        }
+    }
+
+}
\ No newline at end of file


[incubator-plc4x] 01/05: Introduced TcpHexDumper to 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 f95d70fff7909d01447ac7434cb51be401633429
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 5 13:43:18 2018 +0200

    Introduced TcpHexDumper to driver-bases/test
---
 plc4j/protocols/ads/pom.xml                                   |  6 ++++++
 .../test/java/org/apache/plc4x/java/ads/AdsPlcDriverTest.java |  2 +-
 plc4j/protocols/driver-bases/test/pom.xml                     | 11 +++++++++++
 .../apache/plc4x/java/base/connection/tcp}/TcpHexDumper.java  |  2 +-
 4 files changed, 19 insertions(+), 2 deletions(-)

diff --git a/plc4j/protocols/ads/pom.xml b/plc4j/protocols/ads/pom.xml
index a1557f1..fe505a9 100644
--- a/plc4j/protocols/ads/pom.xml
+++ b/plc4j/protocols/ads/pom.xml
@@ -114,6 +114,12 @@
       <version>1.0.0</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.plc4x</groupId>
+      <artifactId>plc4j-protocol-driver-base-test</artifactId>
+      <version>0.0.1-SNAPSHOT</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
 </project>
\ No newline at end of file
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsPlcDriverTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsPlcDriverTest.java
index 932f790..06e0eae 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsPlcDriverTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsPlcDriverTest.java
@@ -22,11 +22,11 @@ package org.apache.plc4x.java.ads;
 import org.apache.plc4x.java.PlcDriverManager;
 import org.apache.plc4x.java.ads.connection.AdsConnectionFactory;
 import org.apache.plc4x.java.ads.connection.AdsTcpPlcConnection;
-import org.apache.plc4x.java.ads.util.TcpHexDumper;
 import org.apache.plc4x.java.api.authentication.PlcUsernamePasswordAuthentication;
 import org.apache.plc4x.java.api.exceptions.PlcConnectionException;
 import org.apache.plc4x.java.api.exceptions.PlcException;
 import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
+import org.apache.plc4x.java.base.connection.tcp.TcpHexDumper;
 import org.junit.Rule;
 import org.junit.Test;
 
diff --git a/plc4j/protocols/driver-bases/test/pom.xml b/plc4j/protocols/driver-bases/test/pom.xml
index 450c7fd..722f6d5 100644
--- a/plc4j/protocols/driver-bases/test/pom.xml
+++ b/plc4j/protocols/driver-bases/test/pom.xml
@@ -50,6 +50,17 @@
     </dependency>
 
     <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <!-- This is a compile dependency for this test artifact -->
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>commons-io</groupId>
+      <artifactId>commons-io</artifactId>
+    </dependency>
+
+    <dependency>
       <groupId>ch.qos.logback</groupId>
       <artifactId>logback-classic</artifactId>
       <scope>test</scope>
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/TcpHexDumper.java b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/connection/tcp/TcpHexDumper.java
similarity index 98%
rename from plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/TcpHexDumper.java
rename to plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/connection/tcp/TcpHexDumper.java
index 926047b..38d9909 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/TcpHexDumper.java
+++ b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/connection/tcp/TcpHexDumper.java
@@ -16,7 +16,7 @@
  specific language governing permissions and limitations
  under the License.
  */
-package org.apache.plc4x.java.ads.util;
+package org.apache.plc4x.java.base.connection.tcp;
 
 import org.apache.commons.io.HexDump;
 import org.apache.commons.io.IOUtils;


[incubator-plc4x] 05/05: fixed build problems

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 036e3942f599a5ab87210feffc2b9f8a46616d5d
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 5 14:32:39 2018 +0200

    fixed build problems
---
 plc4j/protocols/driver-bases/test/pom.xml |  4 ++++
 plc4j/protocols/modbus/pom.xml            | 10 ++++++++++
 2 files changed, 14 insertions(+)

diff --git a/plc4j/protocols/driver-bases/test/pom.xml b/plc4j/protocols/driver-bases/test/pom.xml
index 685567e..d5ff9a7 100644
--- a/plc4j/protocols/driver-bases/test/pom.xml
+++ b/plc4j/protocols/driver-bases/test/pom.xml
@@ -77,6 +77,10 @@
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>ch.qos.logback</groupId>
diff --git a/plc4j/protocols/modbus/pom.xml b/plc4j/protocols/modbus/pom.xml
index e11914e..9830816 100644
--- a/plc4j/protocols/modbus/pom.xml
+++ b/plc4j/protocols/modbus/pom.xml
@@ -91,6 +91,10 @@
       <groupId>io.netty</groupId>
       <artifactId>netty-transport</artifactId>
     </dependency>
+    <dependency>
+      <groupId>io.netty</groupId>
+      <artifactId>netty-common</artifactId>
+    </dependency>
 
     <dependency>
       <groupId>commons-io</groupId>
@@ -107,6 +111,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>se.koc</groupId>
+      <artifactId>netty-transport-jserialcomm</artifactId>
+      <version>1.0.0</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
       <groupId>org.apache.plc4x</groupId>
       <artifactId>plc4j-protocol-driver-base-test</artifactId>
       <version>0.0.1-SNAPSHOT</version>


[incubator-plc4x] 02/05: added test for modbus PlcDriver

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 492ed4a883ac278d1fe382e21e69cc2884956e5f
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 5 13:43:32 2018 +0200

    added test for modbus PlcDriver
---
 plc4j/protocols/modbus/pom.xml                     |   6 +
 .../apache/plc4x/java/modbus/ModbusPlcDriver.java  |  34 +++---
 .../modbus/connection/ModbusConnectionFactory.java |  40 +++++++
 .../connection/ModbusSerialPlcConnection.java      |   7 +-
 .../modbus/connection/ModbusTcpPlcConnection.java  |  10 +-
 .../plc4x/java/modbus/ModbusPlcDriverTest.java     | 125 +++++++++++++++++++++
 6 files changed, 203 insertions(+), 19 deletions(-)

diff --git a/plc4j/protocols/modbus/pom.xml b/plc4j/protocols/modbus/pom.xml
index 17a57b9..e11914e 100644
--- a/plc4j/protocols/modbus/pom.xml
+++ b/plc4j/protocols/modbus/pom.xml
@@ -106,6 +106,12 @@
       <artifactId>logback-classic</artifactId>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>org.apache.plc4x</groupId>
+      <artifactId>plc4j-protocol-driver-base-test</artifactId>
+      <version>0.0.1-SNAPSHOT</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/ModbusPlcDriver.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/ModbusPlcDriver.java
index 0ceaf4d..0aad72e 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/ModbusPlcDriver.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/ModbusPlcDriver.java
@@ -18,12 +18,12 @@ under the License.
 */
 package org.apache.plc4x.java.modbus;
 
+import org.apache.commons.lang3.StringUtils;
 import org.apache.plc4x.java.api.PlcDriver;
 import org.apache.plc4x.java.api.authentication.PlcAuthentication;
 import org.apache.plc4x.java.api.connection.PlcConnection;
 import org.apache.plc4x.java.api.exceptions.PlcConnectionException;
-import org.apache.plc4x.java.modbus.connection.ModbusSerialPlcConnection;
-import org.apache.plc4x.java.modbus.connection.ModbusTcpPlcConnection;
+import org.apache.plc4x.java.modbus.connection.ModbusConnectionFactory;
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
@@ -40,6 +40,16 @@ public class ModbusPlcDriver implements PlcDriver {
     public static final Pattern SERIAL_PATTERN = Pattern.compile("serial://(?<serialDefinition>((?!/\\d).)*)");
     public static final Pattern MODBUS_URI_PATTERN = Pattern.compile("^modbus:(" + INET_ADDRESS_PATTERN + "|" + SERIAL_PATTERN + ")/?" + "(?<params>\\?.*)?");
 
+    private ModbusConnectionFactory modbusConnectionFactory;
+
+    public ModbusPlcDriver() {
+        this.modbusConnectionFactory = new ModbusConnectionFactory();
+    }
+
+    public ModbusPlcDriver(ModbusConnectionFactory modbusConnectionFactory) {
+        this.modbusConnectionFactory = modbusConnectionFactory;
+    }
+
     @Override
     public String getProtocolCode() {
         return "modbus";
@@ -60,22 +70,18 @@ public class ModbusPlcDriver implements PlcDriver {
 
         String host = matcher.group("host");
         String serialDefinition = matcher.group("serialDefinition");
-        String port = matcher.group("port");
+        String portString = matcher.group("port");
+        Integer port = StringUtils.isNotBlank(portString) ? Integer.parseInt(portString) : null;
         String params = matcher.group("params") != null ? matcher.group("params").substring(1) : null;
-        if (serialDefinition == null) {
-            String hostName = matcher.group("host");
+
+        if (serialDefinition != null) {
+            return modbusConnectionFactory.modbusSerialPlcConnectionOf(serialDefinition, params);
+        } else {
             try {
-                InetAddress inetAddress = InetAddress.getByName(host);
-                if (port == null) {
-                    return new ModbusTcpPlcConnection(inetAddress, params);
-                } else {
-                    return new ModbusTcpPlcConnection(inetAddress, Integer.valueOf(port), params);
-                }
+                return modbusConnectionFactory.modbusTcpPlcConnectionOf(InetAddress.getByName(host), port, params);
             } catch (UnknownHostException e) {
-                throw new PlcConnectionException("Unknown host" + hostName, e);
+                throw new PlcConnectionException(e);
             }
-        } else {
-            return new ModbusSerialPlcConnection(serialDefinition, params);
         }
     }
 
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactory.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactory.java
new file mode 100644
index 0000000..bf23c5d
--- /dev/null
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusConnectionFactory.java
@@ -0,0 +1,40 @@
+/*
+ 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.connection;
+
+import java.net.InetAddress;
+import java.util.Objects;
+
+public class ModbusConnectionFactory {
+
+    public ModbusTcpPlcConnection modbusTcpPlcConnectionOf(InetAddress address, Integer port, String params) {
+        Objects.requireNonNull(address);
+
+        if (port == null) {
+            return ModbusTcpPlcConnection.of(address, params);
+        } else {
+            return ModbusTcpPlcConnection.of(address, port, params);
+        }
+    }
+
+    public ModbusSerialPlcConnection modbusSerialPlcConnectionOf(String serialPort, String params) {
+        Objects.requireNonNull(serialPort);
+        return ModbusSerialPlcConnection.of(serialPort, params);
+    }
+}
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnection.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnection.java
index 214772b..5b5fa31 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnection.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusSerialPlcConnection.java
@@ -20,7 +20,6 @@ package org.apache.plc4x.java.modbus.connection;
 
 import io.netty.channel.ChannelHandler;
 import org.apache.commons.lang3.NotImplementedException;
-import org.apache.plc4x.java.base.connection.ChannelFactory;
 import org.apache.plc4x.java.base.connection.SerialChannelFactory;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -31,13 +30,13 @@ public class ModbusSerialPlcConnection extends BaseModbusPlcConnection {
 
     private static final Logger logger = LoggerFactory.getLogger(ModbusSerialPlcConnection.class);
 
-    public ModbusSerialPlcConnection(String port, String params) {
+    private ModbusSerialPlcConnection(String port, String params) {
         super(new SerialChannelFactory(port), params);
         logger.info("Configured ModbusSerialPlcConnection with: serial-port {}", port);
     }
 
-    ModbusSerialPlcConnection(ChannelFactory channelFactory, String params) {
-        super(channelFactory, params);
+    public static ModbusSerialPlcConnection of(String serialPort, String params) {
+        return new ModbusSerialPlcConnection(serialPort, params);
     }
 
     @Override
diff --git a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
index d5c912d..c418031 100644
--- a/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
+++ b/plc4j/protocols/modbus/src/main/java/org/apache/plc4x/java/modbus/connection/ModbusTcpPlcConnection.java
@@ -39,7 +39,7 @@ public class ModbusTcpPlcConnection extends BaseModbusPlcConnection {
 
     private static final Logger logger = LoggerFactory.getLogger(ModbusTcpPlcConnection.class);
 
-    public ModbusTcpPlcConnection(InetAddress address, String params) {
+    private ModbusTcpPlcConnection(InetAddress address, String params) {
         this(new TcpSocketChannelFactory(address, MODBUS_TCP_PORT), params);
         logger.info("Configured ModbusTcpPlcConnection with: host-name {}", address.getHostAddress());
     }
@@ -53,6 +53,14 @@ public class ModbusTcpPlcConnection extends BaseModbusPlcConnection {
         super(channelFactory, params);
     }
 
+    public static ModbusTcpPlcConnection of(InetAddress address, String params) {
+        return new ModbusTcpPlcConnection(address, params);
+    }
+
+    public static ModbusTcpPlcConnection of(InetAddress address, int port, String params) {
+        return new ModbusTcpPlcConnection(address, port, params);
+    }
+
     @Override
     protected ChannelHandler getChannelHandler(CompletableFuture<Void> sessionSetupCompleteFuture) {
         return new ChannelInitializer() {
diff --git a/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/ModbusPlcDriverTest.java b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/ModbusPlcDriverTest.java
new file mode 100644
index 0000000..f951877
--- /dev/null
+++ b/plc4j/protocols/modbus/src/test/java/org/apache/plc4x/java/modbus/ModbusPlcDriverTest.java
@@ -0,0 +1,125 @@
+/*
+ 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;
+
+
+import org.apache.plc4x.java.PlcDriverManager;
+import org.apache.plc4x.java.api.authentication.PlcUsernamePasswordAuthentication;
+import org.apache.plc4x.java.api.exceptions.PlcConnectionException;
+import org.apache.plc4x.java.api.exceptions.PlcException;
+import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
+import org.apache.plc4x.java.base.connection.tcp.TcpHexDumper;
+import org.apache.plc4x.java.modbus.connection.ModbusConnectionFactory;
+import org.apache.plc4x.java.modbus.connection.ModbusTcpPlcConnection;
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.util.regex.Pattern;
+import java.util.stream.Stream;
+
+import static org.apache.plc4x.java.modbus.ModbusPlcDriver.INET_ADDRESS_PATTERN;
+import static org.apache.plc4x.java.modbus.ModbusPlcDriver.SERIAL_PATTERN;
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
+
+public class ModbusPlcDriverTest {
+
+    @Rule
+    public TcpHexDumper tcpHexDumper = new TcpHexDumper(0, 2);
+
+    @Test
+    public void testModbusAddressPattern() {
+        assertMatching(INET_ADDRESS_PATTERN, "tcp://localhost");
+        assertMatching(INET_ADDRESS_PATTERN, "tcp://localhost:3131");
+        assertMatching(INET_ADDRESS_PATTERN, "tcp://www.google.de");
+        assertMatching(INET_ADDRESS_PATTERN, "tcp://www.google.de:443");
+
+        assertMatching(SERIAL_PATTERN, "serial:///dev/com1");
+        assertMatching(SERIAL_PATTERN, "serial://COM1");
+        assertMatching(SERIAL_PATTERN, "serial:///dev/ttyUSB0");
+    }
+
+    @Test
+    public void testDriverWithCompleteUrls() {
+        ModbusPlcDriver SUT = new ModbusPlcDriver(mock(ModbusConnectionFactory.class));
+        Stream.of(
+            "modbus:tcp://www.google.de",
+            "modbus:tcp://www.google.de:443",
+            "modbus:serial:///dev/com1",
+            "modbus:serial://COM1",
+            "modbus:serial:///dev/ttyUSB0"
+        ).forEach(url -> {
+            try {
+                SUT.connect(url);
+            } catch (PlcConnectionException e) {
+                throw new PlcRuntimeException(e);
+            }
+        });
+    }
+
+    private void assertMatching(Pattern pattern, String match) {
+        if (!pattern.matcher(match).matches()) {
+            fail(pattern + "doesn't match " + match);
+        }
+    }
+
+    @Test
+    public void getConnection() throws Exception {
+        ModbusTcpPlcConnection modbusConnection = (ModbusTcpPlcConnection)
+            new PlcDriverManager().getConnection("modbus:tcp://localhost:" + tcpHexDumper.getPort());
+        modbusConnection.close();
+    }
+
+    @Test(expected = PlcConnectionException.class)
+    public void getConnectionNoAuthSupported() throws Exception {
+        new PlcDriverManager().getConnection("modbus:tcp://localhost:" + tcpHexDumper.getPort(),
+            new PlcUsernamePasswordAuthentication("admin", "admin"));
+    }
+
+    @Test(expected = PlcConnectionException.class)
+    public void getConnectionUnknownHost() throws Exception {
+        new PlcDriverManager().getConnection("modbus:tcp://nowhere:8080");
+    }
+
+    @Test(expected = PlcConnectionException.class)
+    public void getConnectionUnknownPort() throws Exception {
+        new PlcDriverManager().getConnection("modbus:tcp://nowhere:unknown");
+    }
+
+    /**
+     * In this test case the 'modbus' driver should report an invalid url format.
+     *
+     * @throws PlcException something went wrong
+     */
+    @Test(expected = PlcConnectionException.class)
+    public void getConnectionInvalidUrl() throws PlcException {
+        new PlcDriverManager().getConnection("modbus:tcp://localhost/hurz/2");
+    }
+
+    @Test
+    public void getProtocol() {
+        ModbusPlcDriver driver = new ModbusPlcDriver();
+        assertThat(driver.getProtocolCode(), is("modbus"));
+        assertThat(driver.getProtocolName(), is("Modbus (TCP / Serial)"));
+    }
+
+
+}


[incubator-plc4x] 03/05: moved Assert and Junit5Backport to 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 1c65826cd66fbea4f1bc71f3f1f9e368004be17e
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Thu Jul 5 13:53:12 2018 +0200

    moved Assert and Junit5Backport to driver-bases/test
---
 .../java/org/apache/plc4x/java/ads/AdsDumper.java  |  2 +-
 .../commands/types/AdsNotificationSampleTest.java  |  2 +-
 .../ads/api/commands/types/AdsStampHeaderTest.java |  2 +-
 .../java/ads/api/commands/types/LengthTest.java    |  2 +-
 .../ads/api/commands/types/ReadLengthTest.java     |  2 +-
 .../ads/api/commands/types/SampleSizeTest.java     |  2 +-
 .../java/ads/api/commands/types/SamplesTest.java   |  2 +-
 .../ads/api/commands/types/WriteLengthTest.java    |  2 +-
 .../java/ads/api/generic/types/AmsPortTest.java    |  2 +-
 .../java/ads/api/generic/types/DataLengthTest.java |  2 +-
 .../java/ads/api/generic/types/TcpLengthTest.java  |  2 +-
 .../connection/AdsAbstractPlcConnectionTest.java   |  2 +-
 .../ads/connection/AdsConnectionFactoryTest.java   |  2 +-
 ...load2SerialProtocolExampleConversationTest.java |  2 +-
 .../java/ads/protocol/util/DigestUtilTest.java     |  2 +-
 .../ads/protocol/util/LittleEndianDecoderTest.java |  2 +-
 .../ads/protocol/util/LittleEndianEncoderTest.java |  4 ++--
 plc4j/protocols/driver-bases/test/pom.xml          | 18 ++++++++++++++++++
 .../org/apache/plc4x/java/base}/util/Assert.java   | 22 +++++++++++++++-------
 .../plc4x/java/base}/util/Junit5Backport.java      |  2 +-
 20 files changed, 52 insertions(+), 26 deletions(-)

diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsDumper.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsDumper.java
index 8cb1890..52a4b22 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsDumper.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/AdsDumper.java
@@ -28,7 +28,7 @@ import org.apache.plc4x.java.ads.api.generic.types.AmsPort;
 import org.apache.plc4x.java.ads.api.generic.types.Invoke;
 import org.apache.plc4x.java.ads.api.tcp.AmsTCPPacket;
 import org.apache.plc4x.java.ads.api.tcp.types.UserData;
-import org.apache.plc4x.java.ads.util.TcpHexDumper;
+import org.apache.plc4x.java.base.connection.tcp.TcpHexDumper;
 import org.pcap4j.core.PcapDumper;
 import org.pcap4j.core.PcapHandle;
 import org.pcap4j.core.Pcaps;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsNotificationSampleTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsNotificationSampleTest.java
index 3d5f947..1627685 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsNotificationSampleTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsNotificationSampleTest.java
@@ -20,7 +20,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Assert.assertByteEquals;
+import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsStampHeaderTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsStampHeaderTest.java
index 86a8e01..2cc1e89 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsStampHeaderTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/AdsStampHeaderTest.java
@@ -22,7 +22,7 @@ import org.junit.Test;
 
 import java.util.Arrays;
 
-import static org.apache.plc4x.java.ads.util.Assert.assertByteEquals;
+import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/LengthTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/LengthTest.java
index 717d4d8..a982c35 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/LengthTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/LengthTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/ReadLengthTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/ReadLengthTest.java
index 587f4c1..746b303 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/ReadLengthTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/ReadLengthTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SampleSizeTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SampleSizeTest.java
index 1be24b2..64d193e 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SampleSizeTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SampleSizeTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SamplesTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SamplesTest.java
index a4c3d57..10633f2 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SamplesTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/SamplesTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/WriteLengthTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/WriteLengthTest.java
index 50d604f..b4b28c1 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/WriteLengthTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/commands/types/WriteLengthTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.commands.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/AmsPortTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/AmsPortTest.java
index 8f61aa7..f37be58 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/AmsPortTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/AmsPortTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.generic.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/DataLengthTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/DataLengthTest.java
index bdf7bc6..fac0fca 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/DataLengthTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/DataLengthTest.java
@@ -21,7 +21,7 @@ package org.apache.plc4x.java.ads.api.generic.types;
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/TcpLengthTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/TcpLengthTest.java
index e697112..4aa2cbd 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/TcpLengthTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/api/generic/types/TcpLengthTest.java
@@ -23,7 +23,7 @@ import org.apache.commons.codec.binary.Hex;
 import org.apache.plc4x.java.ads.api.tcp.types.TcpLength;
 import org.junit.Test;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsAbstractPlcConnectionTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsAbstractPlcConnectionTest.java
index d8e6f5b..4044858 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsAbstractPlcConnectionTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsAbstractPlcConnectionTest.java
@@ -52,7 +52,7 @@ import java.util.Map;
 import java.util.Queue;
 import java.util.concurrent.*;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.junit.Assert.*;
 import static org.mockito.ArgumentMatchers.any;
 import static org.mockito.Mockito.*;
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsConnectionFactoryTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsConnectionFactoryTest.java
index 7df1c01..edd3850 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsConnectionFactoryTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/connection/AdsConnectionFactoryTest.java
@@ -32,7 +32,7 @@ import org.mockito.junit.MockitoJUnitRunner;
 
 import java.net.InetAddress;
 
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolExampleConversationTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolExampleConversationTest.java
index 1f58276..2bcd132 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolExampleConversationTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/Payload2SerialProtocolExampleConversationTest.java
@@ -51,7 +51,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.atomic.AtomicInteger;
 
-import static org.apache.plc4x.java.ads.util.Assert.byteArrayEqualsTo;
+import static org.apache.plc4x.java.base.util.Assert.byteArrayEqualsTo;
 import static org.mockito.Mockito.*;
 
 /**
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/DigestUtilTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/DigestUtilTest.java
index e595779..5271462 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/DigestUtilTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/DigestUtilTest.java
@@ -30,7 +30,7 @@ import java.util.Arrays;
 import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
-import static org.apache.plc4x.java.ads.util.Assert.assertByteEquals;
+import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
 import static org.junit.Assert.assertEquals;
 
 public class DigestUtilTest {
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianDecoderTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianDecoderTest.java
index 9410038..65e7883 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianDecoderTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianDecoderTest.java
@@ -27,7 +27,7 @@ import java.util.Date;
 
 import static java.util.Arrays.asList;
 import static java.util.Collections.singletonList;
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 import static org.junit.Assert.assertEquals;
 
 public class LittleEndianDecoderTest {
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianEncoderTest.java b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianEncoderTest.java
index c7d471a..40f02da 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianEncoderTest.java
+++ b/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/protocol/util/LittleEndianEncoderTest.java
@@ -24,8 +24,8 @@ import org.junit.Test;
 import java.util.Calendar;
 import java.util.Date;
 
-import static org.apache.plc4x.java.ads.util.Assert.assertByteEquals;
-import static org.apache.plc4x.java.ads.util.Junit5Backport.assertThrows;
+import static org.apache.plc4x.java.base.util.Assert.assertByteEquals;
+import static org.apache.plc4x.java.base.util.Junit5Backport.assertThrows;
 
 public class LittleEndianEncoderTest {
 
diff --git a/plc4j/protocols/driver-bases/test/pom.xml b/plc4j/protocols/driver-bases/test/pom.xml
index 722f6d5..685567e 100644
--- a/plc4j/protocols/driver-bases/test/pom.xml
+++ b/plc4j/protocols/driver-bases/test/pom.xml
@@ -56,6 +56,24 @@
       <scope>compile</scope>
     </dependency>
     <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-core</artifactId>
+      <!-- This is a compile dependency for this test artifact -->
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-library</artifactId>
+      <!-- This is a compile dependency for this test artifact -->
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <!-- This is a compile dependency for this test artifact -->
+      <scope>compile</scope>
+    </dependency>
+    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Assert.java b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Assert.java
similarity index 80%
rename from plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Assert.java
rename to plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Assert.java
index 86c10cc..2898d13 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Assert.java
+++ b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Assert.java
@@ -16,13 +16,14 @@
  specific language governing permissions and limitations
  under the License.
  */
-package org.apache.plc4x.java.ads.util;
+package org.apache.plc4x.java.base.util;
 
-import org.apache.plc4x.java.ads.api.util.ByteValue;
+import org.apache.commons.io.HexDump;
 import org.hamcrest.Description;
 import org.hamcrest.Matcher;
 import org.hamcrest.core.IsEqual;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
 import static java.lang.Integer.toHexString;
@@ -31,15 +32,15 @@ import static org.junit.Assert.assertEquals;
 
 public class Assert {
 
-    public static void assertByteEquals(byte expected, byte actual) throws Exception {
+    public static void assertByteEquals(byte expected, byte actual) {
         String expectedHex = "0x" + leftPad(toHexString(expected), 2, '0');
         String actualHex = "0x" + leftPad(toHexString(actual), 2, '0');
         assertEquals(expectedHex, actualHex);
     }
 
     public static void assertByteEquals(byte[] expected, byte[] actual) throws Exception {
-        String expectedHex = cleanHexDump(ByteValue.of(expected).dump());
-        String actualHex = cleanHexDump(ByteValue.of(actual).dump());
+        String expectedHex = cleanHexDump(dump(expected));
+        String actualHex = cleanHexDump(dump(actual));
         assertEquals(expectedHex, actualHex);
     }
 
@@ -48,7 +49,7 @@ public class Assert {
             @Override
             public void describeTo(Description description) {
                 try {
-                    String dump = ByteValue.of(expected).dump();
+                    String dump = dump(expected);
                     description.appendText("\n").appendText(cleanHexDump(dump));
                 } catch (IOException e) {
                     throw new RuntimeException(e);
@@ -62,7 +63,7 @@ public class Assert {
                     return;
                 }
                 try {
-                    String dump = ByteValue.of((byte[]) item).dump();
+                    String dump = dump((byte[]) item);
                     description.appendText("was ").appendText("\n").appendText(cleanHexDump(dump));
                 } catch (IOException e) {
                     throw new RuntimeException(e);
@@ -74,4 +75,11 @@ public class Assert {
     private static String cleanHexDump(String in) {
         return in.replaceAll("@.*\\{", "@XXXXXXXX{");
     }
+
+    private static String dump(byte[] bytes) throws IOException {
+        try (ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream()) {
+            HexDump.dump(bytes, 0, byteArrayOutputStream, 0);
+            return HexDump.EOL + byteArrayOutputStream.toString();
+        }
+    }
 }
diff --git a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Junit5Backport.java b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Junit5Backport.java
similarity index 97%
rename from plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Junit5Backport.java
rename to plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Junit5Backport.java
index 66c2347..5b82934 100644
--- a/plc4j/protocols/ads/src/test/java/org/apache/plc4x/java/ads/util/Junit5Backport.java
+++ b/plc4j/protocols/driver-bases/test/src/main/java/org/apache/plc4x/java/base/util/Junit5Backport.java
@@ -16,7 +16,7 @@
  specific language governing permissions and limitations
  under the License.
  */
-package org.apache.plc4x.java.ads.util;
+package org.apache.plc4x.java.base.util;
 
 import static org.junit.Assert.fail;