You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@plc4x.apache.org by cd...@apache.org on 2020/08/11 13:59:06 UTC

[plc4x] 05/07: - Continued working on the new AMS/ADS driver

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

cdutz pushed a commit to branch develop
in repository https://gitbox.apache.org/repos/asf/plc4x.git

commit 8bbb7be2108d1454f241fc05f9bc76898a58ec6b
Author: Christofer Dutz <ch...@c-ware.de>
AuthorDate: Tue Aug 11 14:28:59 2020 +0200

    - Continued working on the new AMS/ADS driver
---
 .../org/apache/plc4x/java/api/model/PlcField.java  |   2 +
 .../apache/plc4x/java/amsads/field/AdsField.java   |   5 +-
 .../java/amsads/protocol/AdsProtocolLogic.java     | 157 +++++++-----
 .../plc4x/protocol/amsads/HexBase64Helper.java     |  31 +++
 .../src/test/resources/testsuite/AdsDriverIT.xml   | 270 +++++++++++++++++----
 5 files changed, 363 insertions(+), 102 deletions(-)

diff --git a/plc4j/api/src/main/java/org/apache/plc4x/java/api/model/PlcField.java b/plc4j/api/src/main/java/org/apache/plc4x/java/api/model/PlcField.java
index bad44c6..e19ad75 100644
--- a/plc4j/api/src/main/java/org/apache/plc4x/java/api/model/PlcField.java
+++ b/plc4j/api/src/main/java/org/apache/plc4x/java/api/model/PlcField.java
@@ -18,6 +18,7 @@
  */
 package org.apache.plc4x.java.api.model;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 
 /**
@@ -47,6 +48,7 @@ public interface PlcField {
      * <b>This method should always return the BOXED type for primitives. E.g. not bool.class but Boolean.class</b>
      * @return Either specific type or Object.class if no specific type is known.
      */
+    @JsonIgnore
     default Class<?> getDefaultJavaType() {
         return Object.class;
     }
diff --git a/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/field/AdsField.java b/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/field/AdsField.java
index 9c16181..88d7408 100644
--- a/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/field/AdsField.java
+++ b/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/field/AdsField.java
@@ -21,7 +21,10 @@ package org.apache.plc4x.java.amsads.field;
 import org.apache.plc4x.java.amsads.readwrite.types.AdsDataType;
 import org.apache.plc4x.java.api.model.PlcField;
 
-@FunctionalInterface
 public interface AdsField extends PlcField {
+
     AdsDataType getAdsDataType();
+
+    int getNumberOfElements();
+
 }
diff --git a/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/protocol/AdsProtocolLogic.java b/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/protocol/AdsProtocolLogic.java
index ad67193..2998533 100644
--- a/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/protocol/AdsProtocolLogic.java
+++ b/sandbox/test-java-amsads-driver/src/main/java/org/apache/plc4x/java/amsads/protocol/AdsProtocolLogic.java
@@ -19,12 +19,14 @@ under the License.
 package org.apache.plc4x.java.amsads.protocol;
 
 import org.apache.plc4x.java.amsads.configuration.AdsConfiguration;
+import org.apache.plc4x.java.amsads.field.AdsField;
 import org.apache.plc4x.java.amsads.field.DirectAdsField;
 import org.apache.plc4x.java.amsads.field.SymbolicAdsField;
 import org.apache.plc4x.java.amsads.readwrite.*;
 import org.apache.plc4x.java.amsads.readwrite.io.DataItemIO;
 import org.apache.plc4x.java.amsads.readwrite.types.CommandId;
 import org.apache.plc4x.java.amsads.readwrite.types.ReservedIndexGroups;
+import org.apache.plc4x.java.api.exceptions.PlcException;
 import org.apache.plc4x.java.api.messages.PlcReadRequest;
 import org.apache.plc4x.java.api.messages.PlcReadResponse;
 import org.apache.plc4x.java.api.messages.PlcWriteRequest;
@@ -37,7 +39,6 @@ import org.apache.plc4x.java.spi.Plc4xProtocolBase;
 import org.apache.plc4x.java.spi.configuration.HasConfiguration;
 import org.apache.plc4x.java.spi.generation.ParseException;
 import org.apache.plc4x.java.spi.generation.ReadBuffer;
-import org.apache.plc4x.java.spi.messages.DefaultPlcReadRequest;
 import org.apache.plc4x.java.spi.messages.DefaultPlcReadResponse;
 import org.apache.plc4x.java.spi.messages.InternalPlcReadRequest;
 import org.apache.plc4x.java.spi.messages.utils.ResponseItem;
@@ -109,11 +110,53 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
 
     @Override
     public CompletableFuture<PlcReadResponse> read(PlcReadRequest readRequest) {
-        DefaultPlcReadRequest request = (DefaultPlcReadRequest) readRequest;
-
         // Get all ADS addresses in their resolved state.
-        final List<DirectAdsField> directAdsFields = getDirectAddresses(readRequest.getFields());
+        final CompletableFuture<List<DirectAdsField>> directAdsFieldsFuture =
+            getDirectAddresses(readRequest.getFields());
+
+        // If all addresses were already resolved we can send the request immediately.
+        if(directAdsFieldsFuture.isDone()) {
+            final List<DirectAdsField> fields = directAdsFieldsFuture.getNow(null);
+            if(fields != null) {
+                return executeRead(readRequest, fields);
+            } else {
+                final CompletableFuture<PlcReadResponse> errorFuture = new CompletableFuture<>();
+                errorFuture.completeExceptionally(new PlcException("Error"));
+                return errorFuture;
+            }
+        }
+        // If there are still symbolic addresses that have to be resolved, send the
+        // request as soon as the resolution is done.
+        // In order to instantly be able to return a future, for the final result we have to
+        // create a new one which is then completed later on. Unfortunately as soon as the
+        // directAdsFieldsFuture is completed we still don't have the end result, but we can
+        // now actually send the delayed read request ... as soon as that future completes
+        // we can complete the initial one.
+        else {
+            CompletableFuture<PlcReadResponse> delayedRead = new CompletableFuture<>();
+            directAdsFieldsFuture.handle((directAdsFields, throwable) -> {
+                if(directAdsFields != null) {
+                    final CompletableFuture<PlcReadResponse> delayedResponse =
+                        executeRead(readRequest, directAdsFields);
+                    delayedResponse.handle((plcReadResponse, throwable1) -> {
+                        if (plcReadResponse != null) {
+                            delayedRead.complete(plcReadResponse);
+                        } else {
+                            delayedRead.completeExceptionally(throwable1);
+                        }
+                        return this;
+                    });
+                } else {
+                    delayedRead.completeExceptionally(throwable);
+                }
+                return this;
+            });
+            return delayedRead;
+        }
+    }
 
+    protected CompletableFuture<PlcReadResponse> executeRead(PlcReadRequest readRequest,
+                                                             List<DirectAdsField> directAdsFields) {
         // Depending on the number of fields, use a single item request or a sum-request
         if (directAdsFields.size() == 1) {
             // Do a normal (single item) ADS Read Request
@@ -191,12 +234,12 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
         Map<String, PlcResponseCode> responseCodes = new HashMap<>();
         if (adsData instanceof AdsReadResponse) {
             AdsReadResponse adsReadResponse = (AdsReadResponse) adsData;
-            readBuffer = new ReadBuffer(adsReadResponse.getData());
+            readBuffer = new ReadBuffer(adsReadResponse.getData(), true);
             responseCodes.put(readRequest.getFieldNames().stream().findFirst().orElse(""),
                 parsePlcResponseCode(adsReadResponse.getResult()));
         } else if (adsData instanceof AdsReadWriteResponse) {
             AdsReadWriteResponse adsReadWriteResponse = (AdsReadWriteResponse) adsData;
-            readBuffer = new ReadBuffer(adsReadWriteResponse.getData());
+            readBuffer = new ReadBuffer(adsReadWriteResponse.getData(), true);
             // When parsing a multi-item response, the error codes of each items come
             // in sequence and then come the values.
             for (String fieldName : readRequest.getFieldNames()) {
@@ -211,14 +254,14 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
         if(readBuffer != null) {
             Map<String, ResponseItem<PlcValue>> values = new HashMap<>();
             for (String fieldName : readRequest.getFieldNames()) {
-                DirectAdsField directAdsField = (DirectAdsField) readRequest.getField(fieldName);
+                AdsField field = (AdsField) readRequest.getField(fieldName);
                 // If the response-code was anything but OK, we don't need to parse the payload.
                 if(responseCodes.get(fieldName) != PlcResponseCode.OK) {
                     values.put(fieldName, new ResponseItem<>(responseCodes.get(fieldName), null));
                 }
                 // If the response-code was ok, parse the data returned.
                 else {
-                    values.put(fieldName, parsePlcValue(directAdsField, readBuffer));
+                    values.put(fieldName, parsePlcValue(field, readBuffer));
                 }
             }
             return new DefaultPlcReadResponse((InternalPlcReadRequest) readRequest, values);
@@ -227,15 +270,15 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
     }
 
     private PlcResponseCode parsePlcResponseCode(long adsResult) {
-            if (adsResult == 0L) {
-                return PlcResponseCode.OK;
-            } else {
-                // TODO: Implement this a little more ...
-                return PlcResponseCode.INTERNAL_ERROR;
-            }
+        if (adsResult == 0L) {
+            return PlcResponseCode.OK;
+        } else {
+            // TODO: Implement this a little more ...
+            return PlcResponseCode.INTERNAL_ERROR;
+        }
     }
 
-    private ResponseItem<PlcValue> parsePlcValue(DirectAdsField field, ReadBuffer readBuffer) {
+    private ResponseItem<PlcValue> parsePlcValue(AdsField field, ReadBuffer readBuffer) {
         try {
             if (field.getNumberOfElements() == 1) {
                 return new ResponseItem<>(PlcResponseCode.OK,
@@ -269,7 +312,9 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
         super.decode(context, msg);
     }
 
-    protected List<DirectAdsField> getDirectAddresses(List<PlcField> fields) {
+    protected CompletableFuture<List<DirectAdsField>> getDirectAddresses(List<PlcField> fields) {
+        CompletableFuture<List<DirectAdsField>> future = new CompletableFuture<>();
+
         // Get all symbolic fields from the current request.
         // These potentially need to be resolved to direct addresses, if this has not been done before.
         final List<SymbolicAdsField> referencedSymbolicFields = fields.stream()
@@ -297,10 +342,10 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
                 // resolution result has been added to the map.
                 if (requiredResolutionFields.size() == 1) {
                     SymbolicAdsField symbolicAdsField = requiredResolutionFields.get(0);
-                    resolutionFuture = resolveSymbolicAddress(requiredResolutionFields.get(0));
+                    resolutionFuture = resolveSingleSymbolicAddress(requiredResolutionFields.get(0));
                     pendingResolutionRequests.put(symbolicAdsField, resolutionFuture);
                 } else {
-                    resolutionFuture = resolveSymbolicAddresses(requiredResolutionFields);
+                    resolutionFuture = resolveMultipleSymbolicAddresses(requiredResolutionFields);
                     for (SymbolicAdsField symbolicAdsField : requiredResolutionFields) {
                         pendingResolutionRequests.put(symbolicAdsField, resolutionFuture);
                     }
@@ -313,28 +358,30 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
                     .map(symbolicAdsField -> pendingResolutionRequests.get(symbolicAdsField))
                     .toArray(CompletableFuture[]::new));
 
-            // BLOCKING: Wait for the resolution to finish.
-            // TODO: Make this asynchronous ...
-            try {
-                resolutionComplete.get(configuration.getTimeoutSymbolicAddressResolution(), TimeUnit.MILLISECONDS);
-            } catch (TimeoutException e) {
-                // TODO: Return an error indicating a timeout during resolution.
-            } catch (InterruptedException | ExecutionException e) {
-                // TODO: Return an error indicating a timeout an internal server error.
-            }
+            // Complete the future asynchronously as soon as all fields are resolved.
+            resolutionComplete.handleAsync((unused, throwable) -> {
+                return future.complete(fields.stream().map(plcField -> {
+                    if (plcField instanceof SymbolicAdsField) {
+                        return symbolicFieldMapping.get(plcField);
+                    } else {
+                        return (DirectAdsField) plcField;
+                    }
+                }).collect(Collectors.toList()));
+            });
+        } else {
+            // If all fields were resolved, we can continue instantly.
+            future.complete(fields.stream().map(plcField -> {
+                if (plcField instanceof SymbolicAdsField) {
+                    return symbolicFieldMapping.get(plcField);
+                } else {
+                    return (DirectAdsField) plcField;
+                }
+            }).collect(Collectors.toList()));
         }
-
-        // So here all fields should be resolved so we can continue normally.
-        return fields.stream().map(plcField -> {
-            if (plcField instanceof SymbolicAdsField) {
-                return symbolicFieldMapping.get(plcField);
-            } else {
-                return (DirectAdsField) plcField;
-            }
-        }).collect(Collectors.toList());
+        return future;
     }
 
-    protected CompletableFuture<Void> resolveSymbolicAddress(SymbolicAdsField symbolicAdsField) {
+    protected CompletableFuture<Void> resolveSingleSymbolicAddress(SymbolicAdsField symbolicAdsField) {
         CompletableFuture<Void> future = new CompletableFuture<>();
 
         // TODO: Instead of using 4 we need the size of the expected response
@@ -342,7 +389,7 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
             getNullByteTerminatedArray(symbolicAdsField.getSymbolicField()));
         AmsPacket amsPacket = new AmsPacket(configuration.getTargetAmsNetId(), configuration.getTargetAmsPort(),
             configuration.getSourceAmsNetId(), configuration.getSourceAmsPort(),
-            CommandId.ADS_READ, DEFAULT_COMMAND_STATE, 0, invokeIdGenerator.getAndIncrement(), adsData);
+            CommandId.ADS_READ_WRITE, DEFAULT_COMMAND_STATE, 0, invokeIdGenerator.getAndIncrement(), adsData);
 
         // Start a new request-transaction (Is ended in the response-handler)
         RequestTransactionManager.RequestTransaction transaction = tm.startRequest();
@@ -355,30 +402,25 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
             .check(adsDataResponse -> adsDataResponse instanceof AdsReadWriteResponse)
             .unwrap(adsDataResponse -> (AdsReadWriteResponse) adsDataResponse)
             .handle(responseAdsData -> {
-                ReadBuffer readBuffer = new ReadBuffer(responseAdsData.getData());
+                ReadBuffer readBuffer = new ReadBuffer(responseAdsData.getData(), true);
                 try {
-                    // This should be 0 in the success case.
-                    long returnCode = readBuffer.readLong(32);
-                    // This is always 4
-                    long itemLength = readBuffer.readLong(32);
-                    // Get the handle from the response.
-                    long handle = readBuffer.readLong(32);
-                    if (returnCode == 0) {
-                        DirectAdsField directAdsField = new DirectAdsField(ReservedIndexGroups.ADSIGRP_SYM_VALBYHND.getValue(),
-                            handle, symbolicAdsField.getAdsDataType(), symbolicAdsField.getNumberOfElements());
-                        symbolicFieldMapping.put(symbolicAdsField, directAdsField);
-                        future.complete(null);
-                    } else {
-                        // TODO: Handle the case of unsuccessful resolution ..
-                    }
+                    // Read the handle.
+                    long handle = readBuffer.readUnsignedLong(32);
+
+                    DirectAdsField directAdsField = new DirectAdsField(
+                        ReservedIndexGroups.ADSIGRP_SYM_VALBYHND.getValue(), handle,
+                        symbolicAdsField.getAdsDataType(), symbolicAdsField.getNumberOfElements());
+                    symbolicFieldMapping.put(symbolicAdsField, directAdsField);
+                    future.complete(null);
                 } catch (ParseException e) {
-                    e.printStackTrace();
+                    future.completeExceptionally(e);
                 }
+                transaction.endRequest();
             }));
         return future;
     }
 
-    protected CompletableFuture<Void> resolveSymbolicAddresses(List<SymbolicAdsField> symbolicAdsFields) {
+    protected CompletableFuture<Void> resolveMultipleSymbolicAddresses(List<SymbolicAdsField> symbolicAdsFields) {
         CompletableFuture<Void> future = new CompletableFuture<>();
 
         // TODO: Instead of using 4 we need the size of the expected response
@@ -401,8 +443,9 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
             .check(adsDataResponse -> adsDataResponse instanceof AdsReadWriteResponse)
             .unwrap(adsDataResponse -> (AdsReadWriteResponse) adsDataResponse)
             .handle(responseAdsData -> {
-                ReadBuffer readBuffer = new ReadBuffer(responseAdsData.getData());
+                ReadBuffer readBuffer = new ReadBuffer(responseAdsData.getData(), true);
                 Map<SymbolicAdsField, Long> returnCodes = new HashMap<>();
+                // In the response first come the return codes and the data-lengths for each item.
                 symbolicAdsFields.forEach(symbolicAdsField -> {
                     try {
                         // This should be 0 in the success case.
@@ -415,6 +458,7 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
                         e.printStackTrace();
                     }
                 });
+                // After reading the header-information, comes the data itself.
                 symbolicAdsFields.forEach(symbolicAdsField -> {
                     try {
                         if (returnCodes.get(symbolicAdsField) == 0) {
@@ -433,6 +477,7 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsPacket> implements Ha
                     }
                 });
                 future.complete(null);
+                transaction.endRequest();
             }));
         return future;
     }
diff --git a/sandbox/test-java-amsads-driver/src/test/java/org/apache/plc4x/protocol/amsads/HexBase64Helper.java b/sandbox/test-java-amsads-driver/src/test/java/org/apache/plc4x/protocol/amsads/HexBase64Helper.java
new file mode 100644
index 0000000..4985f0c
--- /dev/null
+++ b/sandbox/test-java-amsads-driver/src/test/java/org/apache/plc4x/protocol/amsads/HexBase64Helper.java
@@ -0,0 +1,31 @@
+/*
+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.protocol.amsads;
+
+import org.apache.commons.codec.binary.Hex;
+
+import java.util.Base64;
+
+public class HexBase64Helper {
+    public static void main(String[] args) throws Exception {
+        final byte[] bytes = Hex.decodeHex("0100801b");
+        final String s = Base64.getEncoder().encodeToString(bytes);
+        System.out.println(s);
+    }
+}
diff --git a/sandbox/test-java-amsads-driver/src/test/resources/testsuite/AdsDriverIT.xml b/sandbox/test-java-amsads-driver/src/test/resources/testsuite/AdsDriverIT.xml
index 2f21aee..83a2486 100644
--- a/sandbox/test-java-amsads-driver/src/test/resources/testsuite/AdsDriverIT.xml
+++ b/sandbox/test-java-amsads-driver/src/test/resources/testsuite/AdsDriverIT.xml
@@ -44,6 +44,10 @@
 
   <testcase>
     <name>Single element direct read request</name>
+    <description>
+      When doing a simple read request with only a single direct address, the
+      request should be answered directly.
+    </description>
     <steps>
       <api-request name="Receive Read Request from application">
         <TestReadRequest className="org.apache.plc4x.test.driver.model.api.TestReadRequest">
@@ -145,7 +149,6 @@
               <indexOffset>8</indexOffset>
               <adsDataType>BOOL</adsDataType>
               <numberOfElements>1</numberOfElements>
-              <defaultJavaType>java.lang.Object</defaultJavaType>
             </hurz>
           </request>
           <hurz>
@@ -162,6 +165,10 @@
 
   <testcase>
     <name>Multi element direct read request</name>
+    <description>
+      When doing a simple read request with only direct addresses, but multiple
+      ones, the unofficial multi-read method should be used to read all in one go.
+    </description>
     <steps>
       <api-request name="Receive Read Request from application">
         <TestReadRequest className="org.apache.plc4x.test.driver.model.api.TestReadRequest">
@@ -284,14 +291,12 @@
               <indexOffset>8</indexOffset>
               <adsDataType>BOOL</adsDataType>
               <numberOfElements>1</numberOfElements>
-              <defaultJavaType>java.lang.Object</defaultJavaType>
             </hurz1>
             <hurz2 className="org.apache.plc4x.java.amsads.field.DirectAdsField">
               <indexGroup>4040</indexGroup>
               <indexOffset>12</indexOffset>
               <adsDataType>BOOL</adsDataType>
               <numberOfElements>1</numberOfElements>
-              <defaultJavaType>java.lang.Object</defaultJavaType>
             </hurz2>
           </request>
           <hurz1>
@@ -312,8 +317,13 @@
     </steps>
   </testcase>
 
-  <!--testcase>
+  <testcase>
     <name>Single element symbolic read request</name>
+    <description>
+      When doing a simple read request with only a single symbolic address, which has not
+      been resolved previously, first a resolution request has to be issued and the data
+      from the response should be used in a second request to actually read the data.
+    </description>
     <steps>
       <api-request name="Receive Read Request from application">
         <TestReadRequest className="org.apache.plc4x.test.driver.model.api.TestReadRequest">
@@ -325,8 +335,7 @@
           </fields>
         </TestReadRequest>
       </api-request>
-      <outgoing-plc-message name="Send Ads Read Request">
-        <!- TODO: Should be AmsTCPPacket ->
+      <outgoing-plc-message name="Send Resolve Symbolic Address Request">
         <AmsPacket className="org.apache.plc4x.java.amsads.readwrite.AmsPacket">
           <targetAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
             <octet1>192</octet1>
@@ -359,28 +368,95 @@
             <broadcast>false</broadcast>
           </state>
           <errorCode>0</errorCode>
-          <invokeId>2</invokeId>
+          <invokeId>3</invokeId>
           <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteRequest">
-            <indexGroup>61568</indexGroup>
-            <indexOffset>2</indexOffset>
-            <readLength>0</readLength>
-            <items>
-              <items className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteRequest">
-                <indexGroup>4040</indexGroup>
-                <indexOffset>8</indexOffset>
-                <readLength>1</readLength>
-                <items/>
-                <data></data>
-              </items>
-              <items className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteRequest">
-                <indexGroup>4040</indexGroup>
-                <indexOffset>12</indexOffset>
-                <readLength>1</readLength>
-                <items/>
-                <data></data>
-              </items>
-            </items>
-            <data></data>
+            <indexGroup>61443</indexGroup>
+            <indexOffset>0</indexOffset>
+            <readLength>4</readLength>
+            <items/>
+            <data>bWFpbi5mX3RyaWdEYXRlaUdlbGVzZW4uTQA=</data>
+          </data>
+        </AmsPacket>
+      </outgoing-plc-message>
+      <incoming-plc-message name="Receive Resolve Symbolic Address Response">
+        <AmsPacket className="org.apache.plc4x.java.amsads.readwrite.AmsPacket">
+          <targetAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>200</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </targetAmsNetId>
+          <targetAmsPort>48898</targetAmsPort>
+          <sourceAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>20</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </sourceAmsNetId>
+          <sourceAmsPort>48898</sourceAmsPort>
+          <commandId>ADS_READ_WRITE</commandId>
+          <state className="org.apache.plc4x.java.amsads.readwrite.State">
+            <initCommand>false</initCommand>
+            <updCommand>false</updCommand>
+            <timestampAdded>false</timestampAdded>
+            <highPriorityCommand>false</highPriorityCommand>
+            <systemCommand>false</systemCommand>
+            <adsCommand>true</adsCommand>
+            <noReturn>false</noReturn>
+            <response>true</response>
+            <broadcast>false</broadcast>
+          </state>
+          <errorCode>0</errorCode>
+          <invokeId>3</invokeId>
+          <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteResponse">
+            <result>0</result>
+            <data>AQCAGw==</data>
+          </data>
+        </AmsPacket>
+      </incoming-plc-message>
+      <outgoing-plc-message name="Send Ads Read Request">
+        <!-- TODO: Should be AmsTCPPacket -->
+        <AmsPacket className="org.apache.plc4x.java.amsads.readwrite.AmsPacket">
+          <targetAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>20</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </targetAmsNetId>
+          <targetAmsPort>48898</targetAmsPort>
+          <sourceAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>200</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </sourceAmsNetId>
+          <sourceAmsPort>48898</sourceAmsPort>
+          <commandId>ADS_READ</commandId>
+          <state className="org.apache.plc4x.java.amsads.readwrite.State">
+            <initCommand>false</initCommand>
+            <updCommand>false</updCommand>
+            <timestampAdded>false</timestampAdded>
+            <highPriorityCommand>false</highPriorityCommand>
+            <systemCommand>false</systemCommand>
+            <adsCommand>true</adsCommand>
+            <noReturn>false</noReturn>
+            <response>false</response>
+            <broadcast>false</broadcast>
+          </state>
+          <errorCode>0</errorCode>
+          <invokeId>4</invokeId>
+          <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadRequest">
+            <indexGroup>61445</indexGroup>
+            <indexOffset>461373441</indexOffset>
+            <length>1</length>
           </data>
         </AmsPacket>
       </outgoing-plc-message>
@@ -417,7 +493,7 @@
             <broadcast>false</broadcast>
           </state>
           <errorCode>0</errorCode>
-          <invokeId>2</invokeId>
+          <invokeId>4</invokeId>
           <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteResponse">
             <result>0</result>
             <data>AAAAAAAAAAABAQ==</data>
@@ -427,37 +503,141 @@
       <api-response name="Report Read Response to application">
         <DefaultPlcReadResponse className="org.apache.plc4x.java.spi.messages.DefaultPlcReadResponse">
           <request className="org.apache.plc4x.java.spi.messages.DefaultPlcReadRequest">
-            <hurz1 className="org.apache.plc4x.java.amsads.field.DirectAdsField">
-              <indexGroup>4040</indexGroup>
-              <indexOffset>8</indexOffset>
+            <hurz1 className="org.apache.plc4x.java.amsads.field.SymbolicAdsField">
               <adsDataType>BOOL</adsDataType>
               <numberOfElements>1</numberOfElements>
-              <defaultJavaType>java.lang.Object</defaultJavaType>
+              <symbolicField>main.f_trigDateiGelesen.M</symbolicField>
             </hurz1>
-            <hurz2 className="org.apache.plc4x.java.amsads.field.DirectAdsField">
-              <indexGroup>4040</indexGroup>
-              <indexOffset>12</indexOffset>
-              <adsDataType>BOOL</adsDataType>
-              <numberOfElements>1</numberOfElements>
-              <defaultJavaType>java.lang.Object</defaultJavaType>
-            </hurz2>
           </request>
           <hurz1>
             <code>OK</code>
             <value className="org.apache.plc4x.java.api.value.PlcBoolean">
-              <object>true</object>
+              <object>false</object>
             </value>
           </hurz1>
-          <hurz2>
+        </DefaultPlcReadResponse>
+      </api-response>
+    </steps>
+  </testcase>
+
+  <testcase>
+    <name>Single element symbolic read request (Address previously resolved)</name>
+    <description>
+      When doing a simple read request with only a single symbolic address, which has
+      been resolved previously, the data from the previous request should be used directly
+      without re-resolving it.
+    </description>
+    <steps>
+      <api-request name="Receive Read Request from application">
+        <TestReadRequest className="org.apache.plc4x.test.driver.model.api.TestReadRequest">
+          <fields>
+            <field className="org.apache.plc4x.test.driver.model.api.TestField">
+              <name>hurz1</name>
+              <address>main.f_trigDateiGelesen.M:BOOL</address>
+            </field>
+          </fields>
+        </TestReadRequest>
+      </api-request>
+      <outgoing-plc-message name="Send Ads Read Request">
+        <!-- TODO: Should be AmsTCPPacket -->
+        <AmsPacket className="org.apache.plc4x.java.amsads.readwrite.AmsPacket">
+          <targetAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>20</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </targetAmsNetId>
+          <targetAmsPort>48898</targetAmsPort>
+          <sourceAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>200</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </sourceAmsNetId>
+          <sourceAmsPort>48898</sourceAmsPort>
+          <commandId>ADS_READ</commandId>
+          <state className="org.apache.plc4x.java.amsads.readwrite.State">
+            <initCommand>false</initCommand>
+            <updCommand>false</updCommand>
+            <timestampAdded>false</timestampAdded>
+            <highPriorityCommand>false</highPriorityCommand>
+            <systemCommand>false</systemCommand>
+            <adsCommand>true</adsCommand>
+            <noReturn>false</noReturn>
+            <response>false</response>
+            <broadcast>false</broadcast>
+          </state>
+          <errorCode>0</errorCode>
+          <invokeId>5</invokeId>
+          <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadRequest">
+            <indexGroup>61445</indexGroup>
+            <indexOffset>461373441</indexOffset>
+            <length>1</length>
+          </data>
+        </AmsPacket>
+      </outgoing-plc-message>
+      <incoming-plc-message name="Receive Ads Read Response">
+        <AmsPacket className="org.apache.plc4x.java.amsads.readwrite.AmsPacket">
+          <targetAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>200</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </targetAmsNetId>
+          <targetAmsPort>48898</targetAmsPort>
+          <sourceAmsNetId className="org.apache.plc4x.java.amsads.readwrite.AmsNetId">
+            <octet1>192</octet1>
+            <octet2>168</octet2>
+            <octet3>23</octet3>
+            <octet4>20</octet4>
+            <octet5>1</octet5>
+            <octet6>1</octet6>
+          </sourceAmsNetId>
+          <sourceAmsPort>48898</sourceAmsPort>
+          <commandId>ADS_READ_WRITE</commandId>
+          <state className="org.apache.plc4x.java.amsads.readwrite.State">
+            <initCommand>false</initCommand>
+            <updCommand>false</updCommand>
+            <timestampAdded>false</timestampAdded>
+            <highPriorityCommand>false</highPriorityCommand>
+            <systemCommand>false</systemCommand>
+            <adsCommand>true</adsCommand>
+            <noReturn>false</noReturn>
+            <response>true</response>
+            <broadcast>false</broadcast>
+          </state>
+          <errorCode>0</errorCode>
+          <invokeId>5</invokeId>
+          <data className="org.apache.plc4x.java.amsads.readwrite.AdsReadWriteResponse">
+            <result>0</result>
+            <data>AAAAAAAAAAABAQ==</data>
+          </data>
+        </AmsPacket>
+      </incoming-plc-message>
+      <api-response name="Report Read Response to application">
+        <DefaultPlcReadResponse className="org.apache.plc4x.java.spi.messages.DefaultPlcReadResponse">
+          <request className="org.apache.plc4x.java.spi.messages.DefaultPlcReadRequest">
+            <hurz1 className="org.apache.plc4x.java.amsads.field.SymbolicAdsField">
+              <adsDataType>BOOL</adsDataType>
+              <numberOfElements>1</numberOfElements>
+              <symbolicField>main.f_trigDateiGelesen.M</symbolicField>
+            </hurz1>
+          </request>
+          <hurz1>
             <code>OK</code>
             <value className="org.apache.plc4x.java.api.value.PlcBoolean">
-              <object>true</object>
+              <object>false</object>
             </value>
-          </hurz2>
+          </hurz1>
         </DefaultPlcReadResponse>
       </api-response>
-      <delay>1000</delay>
     </steps>
-  </testcase-->
+  </testcase>
 
 </test:driver-testsuite>
\ No newline at end of file