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/11/16 16:25:54 UTC

[plc4x] branch feature/plc4go updated: - Updated the ADS mspec to use the IEC type names for DataIo - Added definitions for processing the temporal values in ADS DataIo - Fixed problems in the ADS driver causing issues (Replaced stream and lambda expression with for loop) - Added implementation for little-endian reading of BigInteger in ReadBuffer - Added implementation for readDouble in ReadBuffer - Fixed the implementation for readFloat in ReadBuffer - Added some null-checks in DefaultPlcReadResponse - Worked on fine-tuning t [...]

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

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


The following commit(s) were added to refs/heads/feature/plc4go by this push:
     new fe44231  - Updated the ADS mspec to use the IEC type names for DataIo - Added definitions for processing the temporal values in ADS DataIo - Fixed problems in the ADS driver causing issues (Replaced stream and lambda expression with for loop) - Added implementation for little-endian reading of BigInteger in ReadBuffer - Added implementation for readDouble in ReadBuffer - Fixed the implementation for readFloat in ReadBuffer - Added some null-checks in DefaultPlcReadResponse - Work [...]
fe44231 is described below

commit fe4423129ba6559d6e8c1b5331ca933f25714fbb
Author: Christofer Dutz <ch...@c-ware.de>
AuthorDate: Mon Nov 16 17:25:47 2020 +0100

    - Updated the ADS mspec to use the IEC type names for DataIo
    - Added definitions for processing the temporal values in ADS DataIo
    - Fixed problems in the ADS driver causing issues (Replaced stream and lambda expression with for loop)
    - Added implementation for little-endian reading of BigInteger in ReadBuffer
    - Added implementation for readDouble in ReadBuffer
    - Fixed the implementation for readFloat in ReadBuffer
    - Added some null-checks in DefaultPlcReadResponse
    - Worked on fine-tuning the IEC Time and Date PlcValues
---
 .../plc4x/java/ads/protocol/AdsProtocolLogic.java  |  26 +--
 .../plc4x/java/spi/generation/ReadBuffer.java      |  18 +-
 .../plc4x/java/spi/generation/StaticHelper.java    |   5 +
 .../java/spi/messages/DefaultPlcReadResponse.java  |  10 +-
 .../org/apache/plc4x/java/spi/values/PlcDATE.java  |  12 ++
 .../plc4x/java/spi/values/PlcDATE_AND_TIME.java    |  13 +-
 .../spi/values/{PlcTIME.java => PlcLTIME.java}     |  23 ++-
 .../org/apache/plc4x/java/spi/values/PlcTIME.java  |  14 ++
 .../plc4x/java/spi/values/PlcTIME_OF_DAY.java      |   7 +
 .../ads/src/main/resources/protocols/ads/ads.mspec | 189 ++++++++++-----------
 10 files changed, 193 insertions(+), 124 deletions(-)

diff --git a/plc4j/drivers/ads/src/main/java/org/apache/plc4x/java/ads/protocol/AdsProtocolLogic.java b/plc4j/drivers/ads/src/main/java/org/apache/plc4x/java/ads/protocol/AdsProtocolLogic.java
index 45e6b24..817215f 100644
--- a/plc4j/drivers/ads/src/main/java/org/apache/plc4x/java/ads/protocol/AdsProtocolLogic.java
+++ b/plc4j/drivers/ads/src/main/java/org/apache/plc4x/java/ads/protocol/AdsProtocolLogic.java
@@ -302,12 +302,12 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsTCPPacket> implements
         try {
             if (field.getNumberOfElements() == 1) {
                 return new ResponseItem<>(PlcResponseCode.OK,
-                    DataItemIO.staticParse(readBuffer, field.getAdsDataType()));
+                    DataItemIO.staticParse(readBuffer, field.getAdsDataType().getDataFormatName()));
             } else {
                 // Fetch all
                 final PlcValue[] resultItems = IntStream.range(0, field.getNumberOfElements()).mapToObj(i -> {
                     try {
-                        return DataItemIO.staticParse(readBuffer, field.getAdsDataType());
+                        return DataItemIO.staticParse(readBuffer, field.getAdsDataType().getDataFormatName());
                     } catch (ParseException e) {
                         LOGGER.warn("Error parsing field item of type: '{}' (at position {}})", field.getAdsDataType(), i, e);
                     }
@@ -388,7 +388,7 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsTCPPacket> implements
         final AdsField plcField = (AdsField) writeRequest.getField(fieldName);
         final PlcValue plcValue = writeRequest.getPlcValue(fieldName);
         try {
-            WriteBuffer writeBuffer = DataItemIO.staticSerialize(plcValue, plcField.getAdsDataType(), true);
+            WriteBuffer writeBuffer = DataItemIO.staticSerialize(plcValue, plcField.getAdsDataType().getDataFormatName(), true);
             AdsData adsData = new AdsWriteRequest(
                 directAdsField.getIndexGroup(), directAdsField.getIndexOffset(), writeBuffer.getData());
             AmsPacket amsPacket = new AmsPacket(configuration.getTargetAmsNetId(), configuration.getTargetAmsPort(),
@@ -435,7 +435,7 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsTCPPacket> implements
             final AdsField field = (AdsField) writeRequest.getField(fieldName);
             final PlcValue plcValue = writeRequest.getPlcValue(fieldName);
             try {
-                final WriteBuffer itemWriteBuffer = DataItemIO.staticSerialize(plcValue, field.getAdsDataType(), true);
+                final WriteBuffer itemWriteBuffer = DataItemIO.staticSerialize(plcValue, field.getAdsDataType().getDataFormatName(), true);
                 int numBytes = itemWriteBuffer.getPos();
                 System.arraycopy(itemWriteBuffer.getData(), 0, writeBuffer, pos, numBytes);
                 pos += numBytes;
@@ -557,13 +557,19 @@ public class AdsProtocolLogic extends Plc4xProtocolBase<AmsTCPPacket> implements
 
             // 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;
+                if(throwable != null) {
+                    return future.completeExceptionally(throwable.getCause());
+                } else {
+                    List<DirectAdsField> directAdsFields = new ArrayList<>(fields.size());
+                    for (PlcField field : fields) {
+                        if (field instanceof SymbolicAdsField) {
+                            directAdsFields.add(symbolicFieldMapping.get(field));
+                        } else {
+                            directAdsFields.add((DirectAdsField) field);
+                        }
                     }
-                }).collect(Collectors.toList()));
+                    return future.complete(directAdsFields);
+                }
             });
         } else {
             // If all fields were resolved, we can continue instantly.
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/ReadBuffer.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/ReadBuffer.java
index d890f7b..b4cc44e 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/ReadBuffer.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/ReadBuffer.java
@@ -160,9 +160,10 @@ public class ReadBuffer {
             throw new ParseException("unsigned long can only contain max 64 bits");
         }
         try {
+            // Read as signed value
             Long val = bi.readLong(false, bitLength);
             if (littleEndian) {
-                throw new UnsupportedOperationException("not implemented yet");
+                val = Long.reverseBytes(val);
             }
             if (val >= 0) {
                 return BigInteger.valueOf(val);
@@ -269,12 +270,8 @@ public class ReadBuffer {
                     throw new NumberFormatException();
                 }
             } else if (bitLength == 32) {
-                byte[] buffer = new byte[4];
-                buffer[0] = bi.readByte(false, 8);
-                buffer[1] = bi.readByte(false, 8);
-                buffer[2] = bi.readByte(false, 8);
-                buffer[3] = bi.readByte(false, 8);
-                return Float.intBitsToFloat((buffer[0] & 0xff) ^ buffer[1] << 8 ^ buffer[2] << 16 ^ buffer[3] << 24);
+                int intValue = readInt(32);
+                return Float.intBitsToFloat(intValue);
             } else {
                 throw new UnsupportedOperationException("unsupported bit length (only 16 and 32 supported)");
             }
@@ -284,7 +281,12 @@ public class ReadBuffer {
     }
 
     public double readDouble(int bitLength) throws ParseException {
-        throw new UnsupportedOperationException("not implemented yet");
+        if(bitLength == 64) {
+            long longValue = readLong(64);
+            return Double.longBitsToDouble(longValue);
+        } else {
+            throw new UnsupportedOperationException("unsupported bit length (only 64 supported)");
+        }
     }
 
     public BigDecimal readBigDecimal(int bitLength) throws ParseException {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/StaticHelper.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/StaticHelper.java
index 5f927dd..26a37fa 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/StaticHelper.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/generation/StaticHelper.java
@@ -100,6 +100,11 @@ public class StaticHelper {
 
     public static double toFloat(ReadBuffer io, boolean signed, int bitsExponent, int bitsMantissa) {
         try {
+            if (signed && bitsExponent == 8 && bitsMantissa == 23) {
+                return io.readFloat(32);
+            } else if (signed && bitsExponent == 11 & bitsMantissa == 52) {
+                return io.readDouble(64);
+            }
             boolean negative = (signed) && io.readBit();
             long exponent = io.readUnsignedLong(bitsExponent);
             exponent = exponent - (((long) Math.pow(2, bitsExponent) / 2) - 1);
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/messages/DefaultPlcReadResponse.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/messages/DefaultPlcReadResponse.java
index a59d985..6eb5250 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/messages/DefaultPlcReadResponse.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/messages/DefaultPlcReadResponse.java
@@ -114,14 +114,20 @@ public class DefaultPlcReadResponse implements PlcReadResponse, XmlSerializable
     @Override
     @JsonIgnore
     public Object getObject(String name) {
-        return getFieldInternal(name).getObject();
+        if(getFieldInternal(name) != null) {
+            return getFieldInternal(name).getObject();
+        }
+        return null;
     }
 
     @Override
     @JsonIgnore
     public Object getObject(String name, int index) {
         PlcValue fieldInternal = getFieldIndexInternal(name, index);
-        return fieldInternal.getObject();
+        if(fieldInternal != null) {
+            return fieldInternal.getObject();
+        }
+        return null;
     }
 
     @Override
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE.java
index fffa5c9..e87fb37 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE.java
@@ -26,7 +26,11 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
 import org.w3c.dom.Element;
 
+import java.sql.Timestamp;
+import java.time.Instant;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneId;
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, property = "className")
 public class PlcDATE extends PlcSimpleValue<LocalDate> {
@@ -34,6 +38,9 @@ public class PlcDATE extends PlcSimpleValue<LocalDate> {
     public static PlcDATE of(Object value) {
         if (value instanceof LocalDate) {
             return new PlcDATE((LocalDate) value);
+        } else if(value instanceof Long) {
+            return new PlcDATE(LocalDateTime.ofInstant(
+                Instant.ofEpochSecond((long) value), ZoneId.systemDefault()).toLocalDate());
         }
         throw new PlcRuntimeException("Invalid value type");
     }
@@ -43,6 +50,11 @@ public class PlcDATE extends PlcSimpleValue<LocalDate> {
         super(value, true);
     }
 
+    @JsonCreator(mode = JsonCreator.Mode.PROPERTIES)
+    public PlcDATE(@JsonProperty("value") Long value) {
+        super(LocalDateTime.ofInstant(Instant.ofEpochSecond(value), ZoneId.systemDefault()).toLocalDate(), true);
+    }
+
     @Override
     @JsonIgnore
     public boolean isString() {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE_AND_TIME.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE_AND_TIME.java
index 1bc4acf..9a846ff 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE_AND_TIME.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcDATE_AND_TIME.java
@@ -26,9 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
 import org.w3c.dom.Element;
 
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
+import java.time.*;
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, property = "className")
 public class PlcDATE_AND_TIME extends PlcSimpleValue<LocalDateTime> {
@@ -36,6 +34,9 @@ public class PlcDATE_AND_TIME extends PlcSimpleValue<LocalDateTime> {
     public static PlcDATE_AND_TIME of(Object value) {
         if (value instanceof LocalDateTime) {
             return new PlcDATE_AND_TIME((LocalDateTime) value);
+        } else if (value instanceof Long) {
+            return new PlcDATE_AND_TIME(LocalDateTime.ofInstant(
+                Instant.ofEpochSecond((long) value), ZoneId.of("UTC")));
         }
         throw new PlcRuntimeException("Invalid value type");
     }
@@ -45,6 +46,12 @@ public class PlcDATE_AND_TIME extends PlcSimpleValue<LocalDateTime> {
         super(value, true);
     }
 
+    @JsonCreator(mode = JsonCreator.Mode.PROPERTIES)
+    public PlcDATE_AND_TIME(@JsonProperty("value") Long value) {
+        super(LocalDateTime.ofInstant(
+            Instant.ofEpochSecond(value), ZoneId.of("UTC")), true);
+    }
+
     @Override
     @JsonIgnore
     public boolean isString() {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcLTIME.java
similarity index 72%
copy from plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java
copy to plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcLTIME.java
index 3de1ce7..206cc72 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcLTIME.java
@@ -26,24 +26,35 @@ import org.apache.plc4x.java.api.exceptions.PlcRuntimeException;
 import org.w3c.dom.Element;
 
 import java.time.Duration;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, property = "className")
-public class PlcTIME extends PlcSimpleValue<Duration> {
+public class PlcLTIME extends PlcSimpleValue<Duration> {
 
-    public static PlcTIME of(Object value) {
+    public static PlcLTIME of(Object value) {
         if (value instanceof Duration) {
-            return new PlcTIME((Duration) value);
+            return new PlcLTIME((Duration) value);
+        } else if(value instanceof Integer) {
+            return new PlcLTIME(Duration.of((long) value, ChronoUnit.MILLIS));
+        } else if(value instanceof Long) {
+            return new PlcLTIME(Duration.of((long) value, ChronoUnit.NANOS));
         }
         throw new PlcRuntimeException("Invalid value type");
     }
 
     @JsonCreator(mode = JsonCreator.Mode.PROPERTIES)
-    public PlcTIME(@JsonProperty("value") Duration value) {
+    public PlcLTIME(@JsonProperty("value") Duration value) {
         super(value, true);
     }
 
+    public PlcLTIME(@JsonProperty("value") Integer value) {
+        super(Duration.of((long) value, ChronoUnit.NANOS), true);
+    }
+
+    public PlcLTIME(@JsonProperty("value") Long value) {
+        super(Duration.of(value, ChronoUnit.NANOS), true);
+    }
+
     @Override
     @JsonIgnore
     public boolean isString() {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java
index 3de1ce7..ee7bea0 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME.java
@@ -28,6 +28,8 @@ import org.w3c.dom.Element;
 import java.time.Duration;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
+import java.time.temporal.ChronoUnit;
+import java.time.temporal.TemporalUnit;
 
 @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, property = "className")
 public class PlcTIME extends PlcSimpleValue<Duration> {
@@ -35,6 +37,10 @@ public class PlcTIME extends PlcSimpleValue<Duration> {
     public static PlcTIME of(Object value) {
         if (value instanceof Duration) {
             return new PlcTIME((Duration) value);
+        } else if(value instanceof Integer) {
+            return new PlcTIME(Duration.of((long) value, ChronoUnit.MILLIS));
+        } else if(value instanceof Long) {
+            return new PlcTIME(Duration.of((long) value, ChronoUnit.MILLIS));
         }
         throw new PlcRuntimeException("Invalid value type");
     }
@@ -44,6 +50,14 @@ public class PlcTIME extends PlcSimpleValue<Duration> {
         super(value, true);
     }
 
+    public PlcTIME(@JsonProperty("value") Integer value) {
+        super(Duration.of((long) value, ChronoUnit.MILLIS), true);
+    }
+
+    public PlcTIME(@JsonProperty("value") Long value) {
+        super(Duration.of(value, ChronoUnit.MILLIS), true);
+    }
+
     @Override
     @JsonIgnore
     public boolean isString() {
diff --git a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME_OF_DAY.java b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME_OF_DAY.java
index 5b763c5..f415ad8 100644
--- a/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME_OF_DAY.java
+++ b/plc4j/spi/src/main/java/org/apache/plc4x/java/spi/values/PlcTIME_OF_DAY.java
@@ -35,6 +35,8 @@ public class PlcTIME_OF_DAY extends PlcSimpleValue<LocalTime> {
     public static PlcTIME_OF_DAY of(Object value) {
         if (value instanceof LocalTime) {
             return new PlcTIME_OF_DAY((LocalTime) value);
+        } else if (value instanceof Long) {
+            return new PlcTIME_OF_DAY(LocalTime.ofSecondOfDay(((long) value) / 1000));
         }
         throw new PlcRuntimeException("Invalid value type");
     }
@@ -44,6 +46,11 @@ public class PlcTIME_OF_DAY extends PlcSimpleValue<LocalTime> {
         super(value, true);
     }
 
+    @JsonCreator(mode = JsonCreator.Mode.PROPERTIES)
+    public PlcTIME_OF_DAY(@JsonProperty("value") Long value) {
+        super(LocalTime.ofSecondOfDay(value / 1000), true);
+    }
+
     @Override
     @JsonIgnore
     public boolean isString() {
diff --git a/protocols/ads/src/main/resources/protocols/ads/ads.mspec b/protocols/ads/src/main/resources/protocols/ads/ads.mspec
index beab13c..adcc785 100644
--- a/protocols/ads/src/main/resources/protocols/ads/ads.mspec
+++ b/protocols/ads/src/main/resources/protocols/ads/ads.mspec
@@ -386,20 +386,12 @@
     [array int 8 'data' count 'sampleSize']
 ]
 
-[dataIo 'DataItem' [AdsDataType 'adsDataType']
-    [typeSwitch 'adsDataType'
+[dataIo 'DataItem' [string 'dataFormatName']
+    [typeSwitch 'dataFormatName'
         // -----------------------------------------
         // Bit
         // -----------------------------------------
-        ['AdsDataType.BOOL' BOOL
-            [reserved uint 7 '0x00']
-            [simple   bit    'value']
-        ]
-        ['AdsDataType.BIT' BOOL
-            [reserved uint 7 '0x00']
-            [simple   bit    'value']
-        ]
-        ['AdsDataType.BIT8' BOOL
+        ['IEC61131_BOOL' BOOL
             [reserved uint 7 '0x00']
             [simple   bit    'value']
         ]
@@ -408,24 +400,15 @@
         // Bit-strings
         // -----------------------------------------
         // 1 byte
-        ['AdsDataType.BYTE' List
-            [array bit 'value' count '8']
-        ]
-        ['AdsDataType.BITARR8' List
+        ['IEC61131_BYTE' List
             [array bit 'value' count '8']
         ]
         // 2 byte (16 bit)
-        ['AdsDataType.WORD' List
-            [array bit 'value' count '16']
-        ]
-        ['AdsDataType.BITARR16' List
+        ['IEC61131_WORD' List
             [array bit 'value' count '16']
         ]
         // 4 byte (32 bit)
-        ['AdsDataType.DWORD' List
-            [array bit 'value' count '32']
-        ]
-        ['AdsDataType.BITARR32' List
+        ['IEC61131_DWORD' List
             [array bit 'value' count '32']
         ]
 
@@ -433,133 +416,149 @@
         // Integers
         // -----------------------------------------
         // 8 bit:
-        ['AdsDataType.SINT' SINT
+        ['IEC61131_SINT' SINT
             [simple int 8 'value']
         ]
-        ['AdsDataType.INT8' SINT
-            [simple int 8 'value']
-        ]
-        ['AdsDataType.USINT' USINT
-            [simple uint 8 'value']
-        ]
-        ['AdsDataType.UINT8' USINT
+        ['IEC61131_USINT' USINT
             [simple uint 8 'value']
         ]
         // 16 bit:
-        ['AdsDataType.INT' INT
+        ['IEC61131_INT' INT
             [simple int 16 'value']
         ]
-        ['AdsDataType.INT16' INT
-            [simple int 16 'value']
-        ]
-        ['AdsDataType.UINT' UINT
-            [simple uint 16 'value']
-        ]
-        ['AdsDataType.UINT16' UINT
+        ['IEC61131_UINT' UINT
             [simple uint 16 'value']
         ]
         // 32 bit:
-        ['AdsDataType.DINT' DINT
-            [simple int 32 'value']
-        ]
-        ['AdsDataType.INT32' DINT
+        ['IEC61131_DINT' DINT
             [simple int 32 'value']
         ]
-        ['AdsDataType.UDINT' UDINT
-            [simple uint 32 'value']
-        ]
-        ['AdsDataType.UINT32' UDINT
+        ['IEC61131_UDINT' UDINT
             [simple uint 32 'value']
         ]
         // 64 bit:
-        ['AdsDataType.LINT' LINT
-            [simple int 64 'value']
-        ]
-        ['AdsDataType.INT64' LINT
+        ['IEC61131_LINT' LINT
             [simple int 64 'value']
         ]
-        ['AdsDataType.ULINT' ULINT
-            [simple uint 64 'value']
-        ]
-        ['AdsDataType.UINT64' ULINT
+        ['IEC61131_ULINT' ULINT
             [simple uint 64 'value']
         ]
 
         // -----------------------------------------
         // Floating point values
         // -----------------------------------------
-        ['AdsDataType.REAL' REAL
+        ['IEC61131_REAL' REAL
             [simple float 8.23  'value']
         ]
-        ['AdsDataType.FLOAT' REAL
-            [simple float 8.23  'value']
-        ]
-        ['AdsDataType.LREAL' LREAL
-            [simple float 11.52 'value']
-        ]
-        ['AdsDataType.DOUBLE' LREAL
+        ['IEC61131_LREAL' LREAL
             [simple float 11.52 'value']
         ]
 
         // -----------------------------------------
         // Characters & Strings
         // -----------------------------------------
-        ['AdsDataType.STRING' STRING
+        ['IEC61131_CHAR' STRING
+//            [simple string 8 'UTF-8' 'value']
+        ]
+        ['IEC61131_WCHAR' STRING
+//            [simple string 16 'UTF-16' 'value']
+        ]
+        ['IEC61131_STRING' STRING
 //            [manual string 'UTF-8' 'value' 'STATIC_CALL("org.apache.plc4x.java.ads.utils.StaticHelper.parseAmsString", io, _type.encoding)' 'STATIC_CALL("org.apache.plc4x.java.ads.utils.StaticHelper.serializeAmsString", io, _value, _type.encoding)' '_value.length + 2']
         ]
+        ['IEC61131_WSTRING' STRING
+//            [manual string 'UTF-16' 'value' 'STATIC_CALL("org.apache.plc4x.java.ads.utils.StaticHelper.parseAmsString", io, _type.encoding)' 'STATIC_CALL("org.apache.plc4x.java.ads.utils.StaticHelper.serializeAmsString", io, _value, _type.encoding)' '_value.length + 2']
+        ]
+
+        // -----------------------------------------
+        // Date & Times
+        // -----------------------------------------
+        // Interpreted as "milliseconds"
+        ['IEC61131_TIME' TIME
+            [simple uint 32 'value']
+        ]
+        // Interpreted as "nanoseconds"
+        ['IEC61131_LTIME' LTIME
+            [simple int 64 'value']
+        ]
+        // Interpreted as "seconds since epoch"
+        ['IEC61131_DATE' DATE
+            [simple uint 32 'value']
+        ]
+        // Interpreted as "milliseconds since midnight"
+        ['IEC61131_TIME_OF_DAY' TIME_OF_DAY
+            [simple uint 32 'value']
+        ]
+        // Interpreted as "seconds since epoch"
+        ['IEC61131_DATE_AND_TIME' DATE_AND_TIME
+            [simple uint 32 'value']
+        ]
     ]
 ]
 
-[enum 'AdsDataType' [uint 8 'numBytes']
-    [BOOL       ['1']]
-    [BIT        ['1']]
-    [BIT8       ['1']]
+[enum 'AdsDataType' [uint 8 'numBytes', string 'dataFormatName']
+    [BOOL       ['1', 'IEC61131_BOOL']]
+    [BIT        ['1', 'IEC61131_BOOL']]
+    [BIT8       ['1', 'IEC61131_BOOL']]
     // -----------------------------------------
     // Bit-strings
     // -----------------------------------------
     // 1 byte
-    [BYTE       ['1']]
-    [BITARR8    ['1']]
+    [BYTE       ['1', 'IEC61131_BYTE']]
+    [BITARR8    ['1', 'IEC61131_BYTE']]
     // 2 byte (16 bit)
-    [WORD       ['2']]
-    [BITARR16   ['2']]
+    [WORD       ['2', 'IEC61131_WORD']]
+    [BITARR16   ['2', 'IEC61131_WORD']]
     // 4 byte (32 bit)
-    [DWORD      ['4']]
-    [BITARR32   ['4']]
+    [DWORD      ['4', 'IEC61131_DWORD']]
+    [BITARR32   ['4', 'IEC61131_DWORD']]
     // -----------------------------------------
     // Integers
     // -----------------------------------------
     // 8 bit:
-    [SINT       ['1']]
-    [INT8       ['1']]
-    [USINT      ['1']]
-    [UINT8      ['1']]
+    [SINT       ['1', 'IEC61131_SINT']]
+    [INT8       ['1', 'IEC61131_SINT']]
+    [USINT      ['1', 'IEC61131_USINT']]
+    [UINT8      ['1', 'IEC61131_USINT']]
     // 16 bit:
-    [INT        ['2']]
-    [INT16      ['2']]
-    [UINT       ['2']]
-    [UINT16     ['2']]
+    [INT        ['2', 'IEC61131_INT']]
+    [INT16      ['2', 'IEC61131_INT']]
+    [UINT       ['2', 'IEC61131_UINT']]
+    [UINT16     ['2', 'IEC61131_UINT']]
     // 32 bit:
-    [DINT       ['4']]
-    [INT32      ['4']]
-    [UDINT      ['4']]
-    [UINT32     ['4']]
+    [DINT       ['4', 'IEC61131_DINT']]
+    [INT32      ['4', 'IEC61131_DINT']]
+    [UDINT      ['4', 'IEC61131_UDINT']]
+    [UINT32     ['4', 'IEC61131_UDINT']]
     // 64 bit:
-    [LINT       ['8']]
-    [INT64      ['8']]
-    [ULINT      ['8']]
-    [UINT64     ['8']]
+    [LINT       ['8', 'IEC61131_LINT']]
+    [INT64      ['8', 'IEC61131_LINT']]
+    [ULINT      ['8', 'IEC61131_ULINT']]
+    [UINT64     ['8', 'IEC61131_ULINT']]
     // -----------------------------------------
     // Floating point values
     // -----------------------------------------
-    [REAL       ['4']]
-    [FLOAT      ['4']]
-    [LREAL      ['8']]
-    [DOUBLE     ['8']]
+    [REAL       ['4', 'IEC61131_REAL']]
+    [FLOAT      ['4', 'IEC61131_REAL']]
+    [LREAL      ['8', 'IEC61131_LREAL']]
+    [DOUBLE     ['8', 'IEC61131_LREAL']]
     // -----------------------------------------
     // Characters & Strings
     // -----------------------------------------
-    [STRING     ['9']]
+    [CHAR       ['1',  'IEC61131_CHAR']]
+    [WCHAR      ['2',  'IEC61131_WCHAR']]
+    [STRING     ['9',  'IEC61131_STRING']]
+    [WSTRING    ['18', 'IEC61131_WSTRING']]
+    // -----------------------------------------
+    // Dates & Times
+    // -----------------------------------------
+    [TIME           ['4', 'IEC61131_TIME']]
+    [LTIME          ['8', 'IEC61131_LTIME']]
+    [DATE           ['4', 'IEC61131_DATE']]
+    [TIME_OF_DAY    ['4', 'IEC61131_TIME_OF_DAY']]
+    [TOD            ['4', 'IEC61131_TIME_OF_DAY']]
+    [DATE_AND_TIME  ['4', 'IEC61131_DATE_AND_TIME']]
+    [DT             ['4', 'IEC61131_DATE_AND_TIME']]
 ]
 
 [enum uint 32 'ReservedIndexGroups'