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 2022/08/16 10:53:08 UTC

[plc4x] 02/02: feat(plc4go/cbus): implemented proper plc-value mapping for identify calls

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

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

commit 8a208fa9952a8279543d8117df3041e8a7a10fc9
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 16 12:52:59 2022 +0200

    feat(plc4go/cbus): implemented proper plc-value mapping for identify calls
---
 plc4go/internal/cbus/Reader.go                     | 141 ++++++++++++++++++++-
 .../tests/drivers/tests/manual_cbus_driver_test.go |  31 +++++
 2 files changed, 170 insertions(+), 2 deletions(-)

diff --git a/plc4go/internal/cbus/Reader.go b/plc4go/internal/cbus/Reader.go
index 59f4f1511..69e0b2cad 100644
--- a/plc4go/internal/cbus/Reader.go
+++ b/plc4go/internal/cbus/Reader.go
@@ -22,6 +22,7 @@ package cbus
 import (
 	"context"
 	"fmt"
+	"strconv"
 	"sync"
 	"time"
 
@@ -164,6 +165,7 @@ func (m *Reader) Read(ctx context.Context, readRequest model.PlcReadRequest) <-c
 					}
 
 					log.Trace().Msg("Handling confirmed data")
+					// TODO: check if we can use a plcValueSerializer
 					switch reply := embeddedReply.GetReply().(readWriteModel.ReplyEncodedReply).GetEncodedReply().(type) {
 					case readWriteModel.EncodedReplyCALReplyExactly:
 						calData := reply.GetCalReply().GetCalData()
@@ -230,9 +232,144 @@ func (m *Reader) Read(ctx context.Context, readRequest model.PlcReadRequest) <-c
 								}
 								addPlcValue(fieldNameCopy, spiValues.NewPlcList(plcListValues))
 							}
+						case readWriteModel.CALDataIdentifyReplyExactly:
+							switch identifyReplyCommand := calData.GetIdentifyReplyCommand().(type) {
+							case readWriteModel.IdentifyReplyCommandCurrentSenseLevelsExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetCurrentSenseLevels()))
+							case readWriteModel.IdentifyReplyCommandDelaysExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(map[string]values.PlcValue{
+									"ReStrikeDelay": spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetReStrikeDelay())),
+									"TerminalLevel": spiValues.NewPlcByteArray(identifyReplyCommand.GetTerminalLevels()),
+								}))
+							case readWriteModel.IdentifyReplyCommandDSIStatusExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(map[string]values.PlcValue{
+									"ChannelStatus1":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus1().String()),
+									"ChannelStatus2":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus2().String()),
+									"ChannelStatus3":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus3().String()),
+									"ChannelStatus4":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus4().String()),
+									"ChannelStatus5":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus5().String()),
+									"ChannelStatus6":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus6().String()),
+									"ChannelStatus7":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus7().String()),
+									"ChannelStatus8":          spiValues.NewPlcSTRING(identifyReplyCommand.GetChannelStatus8().String()),
+									"UnitStatus":              spiValues.NewPlcSTRING(identifyReplyCommand.GetUnitStatus().String()),
+									"DimmingUCRevisionNumber": spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetDimmingUCRevisionNumber())),
+								}))
+							case readWriteModel.IdentifyReplyCommandExtendedDiagnosticSummaryExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(map[string]values.PlcValue{
+									"LowApplication":         spiValues.NewPlcSTRING(identifyReplyCommand.GetLowApplication().String()),
+									"HighApplication":        spiValues.NewPlcSTRING(identifyReplyCommand.GetHighApplication().String()),
+									"Area":                   spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetArea())),
+									"Crc":                    spiValues.NewPlcUINT(identifyReplyCommand.GetCrc()),
+									"SerialNumber":           spiValues.NewPlcUDINT(identifyReplyCommand.GetSerialNumber()),
+									"NetworkVoltage":         spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetNetworkVoltage())),
+									"UnitInLearnMode":        spiValues.NewPlcBOOL(identifyReplyCommand.GetUnitInLearnMode()),
+									"NetworkVoltageLow":      spiValues.NewPlcBOOL(identifyReplyCommand.GetNetworkVoltageLow()),
+									"NetworkVoltageMarginal": spiValues.NewPlcBOOL(identifyReplyCommand.GetNetworkVoltageMarginal()),
+									"EnableChecksumAlarm":    spiValues.NewPlcBOOL(identifyReplyCommand.GetEnableChecksumAlarm()),
+									"OutputUnit":             spiValues.NewPlcBOOL(identifyReplyCommand.GetOutputUnit()),
+									"InstallationMMIError":   spiValues.NewPlcBOOL(identifyReplyCommand.GetInstallationMMIError()),
+									"EEWriteError":           spiValues.NewPlcBOOL(identifyReplyCommand.GetEEWriteError()),
+									"EEChecksumError":        spiValues.NewPlcBOOL(identifyReplyCommand.GetEEChecksumError()),
+									"EEDataError":            spiValues.NewPlcBOOL(identifyReplyCommand.GetEEDataError()),
+									"MicroReset":             spiValues.NewPlcBOOL(identifyReplyCommand.GetMicroReset()),
+									"CommsTxError":           spiValues.NewPlcBOOL(identifyReplyCommand.GetCommsTxError()),
+									"InternalStackOverflow":  spiValues.NewPlcBOOL(identifyReplyCommand.GetInternalStackOverflow()),
+									"MicroPowerReset":        spiValues.NewPlcBOOL(identifyReplyCommand.GetMicroPowerReset()),
+								}))
+							case readWriteModel.IdentifyReplyCommandFirmwareSummaryExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(map[string]values.PlcValue{
+									"FirmwareVersion": spiValues.NewPlcSTRING(identifyReplyCommand.GetFirmwareVersion()),
+									"UnitServiceType": spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetUnitServiceType())),
+									"Version":         spiValues.NewPlcSTRING(identifyReplyCommand.GetVersion()),
+								}))
+							case readWriteModel.IdentifyReplyCommandFirmwareVersionExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcSTRING(identifyReplyCommand.GetFirmwareVersion()))
+							case readWriteModel.IdentifyReplyCommandGAVPhysicalAddressesExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetValues()))
+							case readWriteModel.IdentifyReplyCommandGAVValuesCurrentExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetValues()))
+							case readWriteModel.IdentifyReplyCommandGAVValuesStoredExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetValues()))
+							case readWriteModel.IdentifyReplyCommandLogicalAssignmentExactly:
+								var plcValues []values.PlcValue
+								for _, logicAssigment := range identifyReplyCommand.GetLogicAssigment() {
+									plcValues = append(plcValues, spiValues.NewPlcStruct(map[string]values.PlcValue{
+										"GreaterOfOrLogic": spiValues.NewPlcBOOL(logicAssigment.GetGreaterOfOrLogic()),
+										"ReStrikeDelay":    spiValues.NewPlcBOOL(logicAssigment.GetReStrikeDelay()),
+										"AssignedToGav16":  spiValues.NewPlcBOOL(logicAssigment.GetAssignedToGav16()),
+										"AssignedToGav15":  spiValues.NewPlcBOOL(logicAssigment.GetAssignedToGav15()),
+										"AssignedToGav14":  spiValues.NewPlcBOOL(logicAssigment.GetAssignedToGav14()),
+										"AssignedToGav13":  spiValues.NewPlcBOOL(logicAssigment.GetAssignedToGav13()),
+									}))
+								}
+								addPlcValue(fieldNameCopy, spiValues.NewPlcList(plcValues))
+							case readWriteModel.IdentifyReplyCommandManufacturerExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcSTRING(identifyReplyCommand.GetManufacturerName()))
+							case readWriteModel.IdentifyReplyCommandMaximumLevelsExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetMaximumLevels()))
+							case readWriteModel.IdentifyReplyCommandMinimumLevelsExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetMinimumLevels()))
+							case readWriteModel.IdentifyReplyCommandNetworkTerminalLevelsExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetNetworkTerminalLevels()))
+							case readWriteModel.IdentifyReplyCommandNetworkVoltageExactly:
+								volts := identifyReplyCommand.GetVolts()
+								voltsFloat, err := strconv.ParseFloat(volts, 0)
+								if err != nil {
+									return errors.Wrap(err, "Error parsing volts")
+								}
+								voltsDecimalPlace := identifyReplyCommand.GetVoltsDecimalPlace()
+								voltsDecimalPlaceFloat, err := strconv.ParseFloat(voltsDecimalPlace, 0)
+								if err != nil {
+									return errors.Wrap(err, "Error parsing volts decimal place")
+								}
+								voltsFloat += voltsDecimalPlaceFloat / 10
+								addPlcValue(fieldNameCopy, spiValues.NewPlcLREAL(voltsFloat))
+							case readWriteModel.IdentifyReplyCommandOutputUnitSummaryExactly:
+								unitFlags := identifyReplyCommand.GetUnitFlags()
+								structContent := map[string]values.PlcValue{
+									"UnitFlags": spiValues.NewPlcStruct(map[string]values.PlcValue{
+										"AssertingNetworkBurden": spiValues.NewPlcBOOL(unitFlags.GetAssertingNetworkBurden()),
+										"RestrikeTimingActive":   spiValues.NewPlcBOOL(unitFlags.GetRestrikeTimingActive()),
+										"RemoteOFFInputAsserted": spiValues.NewPlcBOOL(unitFlags.GetRemoteOFFInputAsserted()),
+										"RemoteONInputAsserted":  spiValues.NewPlcBOOL(unitFlags.GetRemoteONInputAsserted()),
+										"LocalToggleEnabled":     spiValues.NewPlcBOOL(unitFlags.GetLocalToggleEnabled()),
+										"LocalToggleActiveState": spiValues.NewPlcBOOL(unitFlags.GetLocalToggleActiveState()),
+										"ClockGenerationEnabled": spiValues.NewPlcBOOL(unitFlags.GetClockGenerationEnabled()),
+										"UnitGeneratingClock":    spiValues.NewPlcBOOL(unitFlags.GetUnitGeneratingClock()),
+									}),
+									"TimeFromLastRecoverOfMainsInSeconds": spiValues.NewPlcUINT(uint16(identifyReplyCommand.GetTimeFromLastRecoverOfMainsInSeconds())),
+								}
+								if gavStoreEnabledByte1 := identifyReplyCommand.GetGavStoreEnabledByte1(); gavStoreEnabledByte1 != nil {
+									structContent["GavStoreEnabledByte1"] = spiValues.NewPlcUINT(uint16(*gavStoreEnabledByte1))
+								}
+								if gavStoreEnabledByte2 := identifyReplyCommand.GetGavStoreEnabledByte2(); gavStoreEnabledByte2 != nil {
+									structContent["GavStoreEnabledByte2"] = spiValues.NewPlcUINT(uint16(*gavStoreEnabledByte2))
+								}
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(structContent))
+							case readWriteModel.IdentifyReplyCommandTerminalLevelsExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcByteArray(identifyReplyCommand.GetTerminalLevels()))
+							case readWriteModel.IdentifyReplyCommandTypeExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcSTRING(identifyReplyCommand.GetUnitType()))
+							case readWriteModel.IdentifyReplyCommandUnitSummaryExactly:
+								addPlcValue(fieldNameCopy, spiValues.NewPlcStruct(map[string]values.PlcValue{
+									"AssertingNetworkBurden": spiValues.NewPlcBOOL(identifyReplyCommand.GetAssertingNetworkBurden()),
+									"RestrikeTimingActive":   spiValues.NewPlcBOOL(identifyReplyCommand.GetRestrikeTimingActive()),
+									"RemoteOFFInputAsserted": spiValues.NewPlcBOOL(identifyReplyCommand.GetRemoteOFFInputAsserted()),
+									"RemoteONInputAsserted":  spiValues.NewPlcBOOL(identifyReplyCommand.GetRemoteONInputAsserted()),
+									"LocalToggleEnabled":     spiValues.NewPlcBOOL(identifyReplyCommand.GetLocalToggleEnabled()),
+									"LocalToggleActiveState": spiValues.NewPlcBOOL(identifyReplyCommand.GetLocalToggleActiveState()),
+									"ClockGenerationEnabled": spiValues.NewPlcBOOL(identifyReplyCommand.GetClockGenerationEnabled()),
+									"UnitGeneratingClock":    spiValues.NewPlcBOOL(identifyReplyCommand.GetUnitGeneratingClock()),
+								}))
+							default:
+								log.Error().Msgf("Unmapped type %T", identifyReplyCommand)
+								requestWasOk <- false
+								return transaction.EndRequest()
+							}
+						default:
+							log.Warn().Msgf("Unmapped cal data type %T. Returning raw to string", calData)
+							addPlcValue(fieldNameCopy, spiValues.NewPlcSTRING(fmt.Sprintf("%s", calData)))
 						}
-						// TODO: how should we serialize that???
-						addPlcValue(fieldNameCopy, spiValues.NewPlcSTRING(fmt.Sprintf("%s", calData)))
 					default:
 						panic(fmt.Sprintf("All types should be mapped here. Not mapped: %T", reply))
 					}
diff --git a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
index a20948503..98fcadc37 100644
--- a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
+++ b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
@@ -127,3 +127,34 @@ func TestManualCBusBrowse(t *testing.T) {
 	})
 	fmt.Printf("%s", browseRequestResult.GetResponse())
 }
+
+func TestManualCBusRead(t *testing.T) {
+	log.Logger = log.
+		With().Caller().Logger().
+		Output(zerolog.ConsoleWriter{Out: os.Stderr}).
+		Level(zerolog.InfoLevel)
+	config.TraceTransactionManagerWorkers = false
+	config.TraceTransactionManagerTransactions = false
+	config.TraceDefaultMessageCodecWorker = false
+	t.Skip()
+
+	connectionString := "c-bus://192.168.178.101?Monitor=false&MonitoredApplication1=0x00&MonitoredApplication2=0x00"
+	driverManager := plc4go.NewPlcDriverManager()
+	driverManager.RegisterDriver(cbus.NewDriver())
+	transports.RegisterTcpTransport(driverManager)
+	connectionResult := <-driverManager.GetConnection(connectionString)
+	if err := connectionResult.GetErr(); err != nil {
+		t.Error(err)
+		t.FailNow()
+	}
+	connection := connectionResult.GetConnection()
+	defer connection.Close()
+	readRequest, err := connection.ReadRequestBuilder().
+		AddQuery("asd", "cal/3/identify=OutputUnitSummary").
+		Build()
+	if err != nil {
+		panic(err)
+	}
+	readRequestResult := <-readRequest.Execute()
+	fmt.Printf("%s", readRequestResult.GetResponse())
+}