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/08 16:00:29 UTC

[plc4x] branch develop updated (c142f78f8 -> c7e1d90c1)

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

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


    from c142f78f8 fix(plc4go): fixed browse query builder
     new 1e2b13b23 feat(plc4xbrowser): set focus on ctrl+c
     new ff9fe8d5f fix(plc-simulator/cbus): fixed some wrong returns
     new 33b83d3f4 chore(cbus): fixed code commend
     new c7e1d90c1 feat(plc4go/cbus): first draft of browse functionality

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


Summary of changes:
 plc4go/internal/cbus/Browser.go                    | 69 ++++++++++++++++++++--
 plc4go/internal/cbus/Configuration.go              | 40 ++++++++++---
 plc4go/internal/cbus/Connection.go                 | 14 +++--
 plc4go/internal/cbus/Driver.go                     |  2 +-
 plc4go/internal/cbus/FieldHandler.go               | 12 ++--
 plc4go/internal/cbus/MessageCodec.go               |  5 +-
 plc4go/internal/cbus/Reader.go                     | 26 +++++---
 plc4go/internal/knxnetip/Browser.go                | 14 ++---
 .../spi/model/DefaultPlcBrowseQueryResult.go       | 66 +++++++++++++++++++++
 .../internal/spi/model/DefaultPlcBrowseRequest.go  |  1 +
 .../tests/drivers/tests/manual_cbus_driver_test.go | 30 ++++++++++
 plc4go/tools/plc4xbrowser/commands.go              |  2 +-
 plc4go/tools/plc4xbrowser/ui.go                    |  1 +
 .../src/main/resources/protocols/cbus/c-bus.mspec  |  2 +-
 .../server/cbus/protocol/CBusServerAdapter.java    | 19 ++++--
 15 files changed, 255 insertions(+), 48 deletions(-)


[plc4x] 01/04: feat(plc4xbrowser): set focus on ctrl+c

Posted by sr...@apache.org.
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 1e2b13b23891ea9d4b816168daa93b7f46a51ea5
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Mon Aug 8 17:59:04 2022 +0200

    feat(plc4xbrowser): set focus on ctrl+c
---
 plc4go/tools/plc4xbrowser/ui.go | 1 +
 1 file changed, 1 insertion(+)

diff --git a/plc4go/tools/plc4xbrowser/ui.go b/plc4go/tools/plc4xbrowser/ui.go
index 230606a6d..d4a5f4619 100644
--- a/plc4go/tools/plc4xbrowser/ui.go
+++ b/plc4go/tools/plc4xbrowser/ui.go
@@ -136,6 +136,7 @@ func buildCommandArea(newPrimitive func(text string) tview.Primitive, applicatio
 			switch event.Key() {
 			case tcell.KeyCtrlC:
 				commandInputField.SetText("")
+				application.SetFocus(commandInputField)
 				return nil
 			}
 			return event


[plc4x] 04/04: feat(plc4go/cbus): first draft of browse functionality

Posted by sr...@apache.org.
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 c7e1d90c13d2b32e2e54686f2440e6fecd66bf93
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Mon Aug 8 18:00:22 2022 +0200

    feat(plc4go/cbus): first draft of browse functionality
---
 plc4go/internal/cbus/Browser.go                    | 69 ++++++++++++++++++++--
 plc4go/internal/cbus/Configuration.go              | 40 ++++++++++---
 plc4go/internal/cbus/Connection.go                 | 14 +++--
 plc4go/internal/cbus/Driver.go                     |  2 +-
 plc4go/internal/cbus/FieldHandler.go               | 12 ++--
 plc4go/internal/cbus/MessageCodec.go               |  5 +-
 plc4go/internal/cbus/Reader.go                     | 26 +++++---
 plc4go/internal/knxnetip/Browser.go                | 14 ++---
 .../spi/model/DefaultPlcBrowseQueryResult.go       | 66 +++++++++++++++++++++
 .../internal/spi/model/DefaultPlcBrowseRequest.go  |  1 +
 .../tests/drivers/tests/manual_cbus_driver_test.go | 30 ++++++++++
 plc4go/tools/plc4xbrowser/commands.go              |  2 +-
 12 files changed, 239 insertions(+), 42 deletions(-)

diff --git a/plc4go/internal/cbus/Browser.go b/plc4go/internal/cbus/Browser.go
index 39c5ccbe0..d2abc351f 100644
--- a/plc4go/internal/cbus/Browser.go
+++ b/plc4go/internal/cbus/Browser.go
@@ -20,8 +20,13 @@
 package cbus
 
 import (
+	"fmt"
 	"github.com/apache/plc4x/plc4go/internal/spi"
+	"github.com/apache/plc4x/plc4go/internal/spi/model"
 	apiModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/pkg/api/values"
+	readWriteModel "github.com/apache/plc4x/plc4go/protocols/cbus/readwrite/model"
+	"github.com/rs/zerolog/log"
 )
 
 type Browser struct {
@@ -39,16 +44,72 @@ func NewBrowser(connection *Connection, messageCodec spi.MessageCodec) *Browser
 }
 
 func (m Browser) Browse(browseRequest apiModel.PlcBrowseRequest) <-chan apiModel.PlcBrowseRequestResult {
-	return m.BrowseWithInterceptor(browseRequest, func(result apiModel.PlcBrowseEvent) bool {
-		return true
-	})
+	return m.BrowseWithInterceptor(browseRequest, nil)
 }
 
 func (m Browser) BrowseWithInterceptor(browseRequest apiModel.PlcBrowseRequest, interceptor func(result apiModel.PlcBrowseEvent) bool) <-chan apiModel.PlcBrowseRequestResult {
 	result := make(chan apiModel.PlcBrowseRequestResult)
 
 	go func() {
-		// TODO: implement me
+		responseCodes := map[string]apiModel.PlcResponseCode{}
+		results := map[string][]apiModel.PlcBrowseFoundField{}
+		for _, fieldName := range browseRequest.GetFieldNames() {
+			field := browseRequest.GetField(fieldName)
+
+			var queryResults []apiModel.PlcBrowseFoundField
+			switch field := field.(type) {
+			case *unitInfoField:
+				var units []readWriteModel.UnitAddress
+				var attributes []readWriteModel.Attribute
+				if unitAddress := field.unitAddress; unitAddress != nil {
+					units = append(units, *unitAddress)
+				} else {
+					for i := 0; i <= 0xFF; i++ {
+						units = append(units, readWriteModel.NewUnitAddress(byte(i)))
+					}
+				}
+				if attribute := field.attribute; attribute != nil {
+					attributes = append(attributes, *attribute)
+				} else {
+					for _, attribute := range readWriteModel.AttributeValues {
+						attributes = append(attributes, attribute)
+					}
+				}
+			unitLoop:
+				for _, unit := range units {
+					for _, attribute := range attributes {
+						unitAddress := unit.GetAddress()
+						log.Info().Msgf("unit %d: Query %s", unitAddress, attribute)
+						readFieldName := fmt.Sprintf("%s/%d/%s", fieldName, unitAddress, attribute)
+						readRequest, _ := m.connection.ReadRequestBuilder().
+							AddField(readFieldName, NewCALIdentifyField(unit, attribute, 1)).
+							Build()
+						requestResult := <-readRequest.Execute()
+						if err := requestResult.GetErr(); err != nil {
+							log.Info().Err(err).Msgf("unit %d: Can't read attribute %s", unitAddress, attribute)
+							continue unitLoop
+						}
+						queryResults = append(queryResults, &model.DefaultPlcBrowseQueryResult{
+							Field:        NewCALIdentifyField(unit, attribute, 1),
+							Name:         fieldName,
+							Readable:     true,
+							Writable:     false,
+							Subscribable: false,
+							Attributes: map[string]values.PlcValue{
+								"CurrentValue": requestResult.GetResponse().GetValue(readFieldName),
+							},
+						})
+					}
+				}
+			default:
+				responseCodes[fieldName] = apiModel.PlcResponseCode_INTERNAL_ERROR
+			}
+		}
+		result <- &model.DefaultPlcBrowseRequestResult{
+			Request:  browseRequest,
+			Response: model.NewDefaultPlcBrowseResponse(browseRequest, results, responseCodes),
+			Err:      nil,
+		}
 	}()
 	return result
 }
diff --git a/plc4go/internal/cbus/Configuration.go b/plc4go/internal/cbus/Configuration.go
index 39f7911ee..0a807368c 100644
--- a/plc4go/internal/cbus/Configuration.go
+++ b/plc4go/internal/cbus/Configuration.go
@@ -20,6 +20,7 @@
 package cbus
 
 import (
+	"reflect"
 	"strconv"
 
 	"github.com/pkg/errors"
@@ -27,21 +28,46 @@ import (
 )
 
 type Configuration struct {
-	srchk bool
+	Srchk    bool
+	Exstat   bool
+	Pun      bool
+	LocalSal bool
+	Pcn      bool
+	Idmon    bool
+	Monitor  bool
+	Smart    bool
+	XonXoff  bool
+	Connect  bool
 }
 
 func ParseFromOptions(options map[string][]string) (Configuration, error) {
-	configuration := Configuration{}
-	if srchk := getFromOptions(options, "srchk"); srchk != "" {
-		parseBool, err := strconv.ParseBool(srchk)
-		if err != nil {
-			return Configuration{}, errors.Wrap(err, "Error parsing srchk")
+	configuration := createDefaultConfiguration()
+	reflectConfiguration := reflect.ValueOf(&configuration).Elem()
+	for i := 0; i < reflectConfiguration.NumField(); i++ {
+		key := reflectConfiguration.Type().Field(i).Name
+		if optionValue := getFromOptions(options, key); optionValue != "" {
+			parseBool, err := strconv.ParseBool(optionValue)
+			if err != nil {
+				return Configuration{}, errors.Wrapf(err, "Error parsing %s", key)
+			}
+			reflectConfiguration.FieldByName(key).SetBool(parseBool)
 		}
-		configuration.srchk = parseBool
 	}
 	return configuration, nil
 }
 
+func createDefaultConfiguration() Configuration {
+	return Configuration{
+		Exstat:   true,
+		LocalSal: true,
+		Idmon:    true,
+		Monitor:  true,
+		Smart:    true,
+		Srchk:    true,
+		Connect:  true,
+	}
+}
+
 func getFromOptions(options map[string][]string, key string) string {
 	if optionValues, ok := options[key]; ok {
 		if len(optionValues) <= 0 {
diff --git a/plc4go/internal/cbus/Connection.go b/plc4go/internal/cbus/Connection.go
index 5bed3562c..bcab1c603 100644
--- a/plc4go/internal/cbus/Connection.go
+++ b/plc4go/internal/cbus/Connection.go
@@ -243,30 +243,32 @@ func (c *Connection) setupConnection(ch chan plc4go.PlcConnectionConnectResult)
 	}
 	{
 		log.Debug().Msg("Set interface options 3")
-		interfaceOptions3 := readWriteModel.NewParameterValueInterfaceOptions3(readWriteModel.NewInterfaceOptions3(true, false, true, false), nil, 1)
+		interfaceOptions3 := readWriteModel.NewParameterValueInterfaceOptions3(readWriteModel.NewInterfaceOptions3(c.configuration.Exstat, c.configuration.Pun, c.configuration.LocalSal, c.configuration.Pcn), nil, 1)
 		if !c.sendCalDataWrite(ch, readWriteModel.Parameter_INTERFACE_OPTIONS_3, interfaceOptions3, requestContext, cbusOptions) {
 			return
 		}
 		// TODO: add localsal to the options
-		*cbusOptions = readWriteModel.NewCBusOptions(false, false, false, true, false, false, false, false, false)
+		*cbusOptions = readWriteModel.NewCBusOptions(false, false, false, c.configuration.Exstat, false, false, c.configuration.Pun, c.configuration.Pcn, false)
 		log.Debug().Msg("Interface options 3 set")
 	}
 	{
 		log.Debug().Msg("Set interface options 1 power up settings")
-		interfaceOptions1PowerUpSettings := readWriteModel.NewParameterValueInterfaceOptions1PowerUpSettings(readWriteModel.NewInterfaceOptions1PowerUpSettings(readWriteModel.NewInterfaceOptions1(true, true, true, true, false, true)), 1)
+		interfaceOptions1PowerUpSettings := readWriteModel.NewParameterValueInterfaceOptions1PowerUpSettings(readWriteModel.NewInterfaceOptions1PowerUpSettings(readWriteModel.NewInterfaceOptions1(c.configuration.Idmon, c.configuration.Monitor, c.configuration.Smart, c.configuration.Srchk, c.configuration.XonXoff, c.configuration.Connect)), 1)
 		if !c.sendCalDataWrite(ch, readWriteModel.Parameter_INTERFACE_OPTIONS_1_POWER_UP_SETTINGS, interfaceOptions1PowerUpSettings, requestContext, cbusOptions) {
 			return
 		}
-		*cbusOptions = readWriteModel.NewCBusOptions(true, true, true, true, true, false, false, false, true)
+		// TODO: what is with monall
+		*cbusOptions = readWriteModel.NewCBusOptions(c.configuration.Connect, c.configuration.Smart, c.configuration.Idmon, c.configuration.Exstat, c.configuration.Monitor, false, c.configuration.Pun, c.configuration.Pcn, c.configuration.Srchk)
 		log.Debug().Msg("Interface options 1 power up settings set")
 	}
 	{
 		log.Debug().Msg("Set interface options 1")
-		interfaceOptions1 := readWriteModel.NewParameterValueInterfaceOptions1(readWriteModel.NewInterfaceOptions1(true, true, true, true, false, true), nil, 1)
+		interfaceOptions1 := readWriteModel.NewParameterValueInterfaceOptions1(readWriteModel.NewInterfaceOptions1(c.configuration.Idmon, c.configuration.Monitor, c.configuration.Smart, c.configuration.Srchk, c.configuration.XonXoff, c.configuration.Connect), nil, 1)
 		if !c.sendCalDataWrite(ch, readWriteModel.Parameter_INTERFACE_OPTIONS_1, interfaceOptions1, requestContext, cbusOptions) {
 			return
 		}
-		*cbusOptions = readWriteModel.NewCBusOptions(true, true, true, true, true, false, false, false, true)
+		// TODO: what is with monall
+		*cbusOptions = readWriteModel.NewCBusOptions(c.configuration.Connect, c.configuration.Smart, c.configuration.Idmon, c.configuration.Exstat, c.configuration.Monitor, false, c.configuration.Pun, c.configuration.Pcn, c.configuration.Srchk)
 		log.Debug().Msg("Interface options 1 set")
 	}
 	c.fireConnected(ch)
diff --git a/plc4go/internal/cbus/Driver.go b/plc4go/internal/cbus/Driver.go
index 83164bd23..5f736d97b 100644
--- a/plc4go/internal/cbus/Driver.go
+++ b/plc4go/internal/cbus/Driver.go
@@ -82,7 +82,7 @@ func (m *Driver) GetConnection(transportUrl url.URL, transports map[string]trans
 		return ch
 	}
 
-	codec := NewMessageCodec(transportInstance, configuration.srchk)
+	codec := NewMessageCodec(transportInstance)
 	log.Debug().Msgf("working with codec %#v", codec)
 
 	driverContext, err := NewDriverContext(configuration)
diff --git a/plc4go/internal/cbus/FieldHandler.go b/plc4go/internal/cbus/FieldHandler.go
index c62cb7a8f..dc5330259 100644
--- a/plc4go/internal/cbus/FieldHandler.go
+++ b/plc4go/internal/cbus/FieldHandler.go
@@ -58,10 +58,10 @@ type FieldHandler struct {
 func NewFieldHandler() FieldHandler {
 	return FieldHandler{
 		statusRequestPattern: regexp.MustCompile(`^status/(?P<statusRequestType>(?P<binary>binary)|level=0x(?P<startingGroupAddressLabel>00|20|40|60|80|A0|C0|E0))/(?P<application>.*)`),
-		calPattern:           regexp.MustCompile(`^cal/(?P<unitAddress>.*)/(?P<calType>recall=\[(?P<recallParamNo>\w+), ?(?P<recallCount>\d+)]|identify=(?P<identifyAttribute>\w+)|getstatus=(?P<getstatusParamNo>\w+), ?(?P<getstatusCount>\d+))`),
-		salMonitorPattern:    regexp.MustCompile(`^salmonitor/(?P<unitAddress>.*)/(?P<application>.*)`),
-		mmiMonitorPattern:    regexp.MustCompile(`^mmimonitor/(?P<unitAddress>.*)/(?P<application>.*)`),
-		unityQuery:           regexp.MustCompile(`^info/(?P<unitAddress>.*)/(?P<identifyAttribute>\w+)`),
+		calPattern:           regexp.MustCompile(`^cal/(?P<unitAddress>.+)/(?P<calType>recall=\[(?P<recallParamNo>\w+), ?(?P<recallCount>\d+)]|identify=(?P<identifyAttribute>\w+)|getstatus=(?P<getstatusParamNo>\w+), ?(?P<getstatusCount>\d+))`),
+		salMonitorPattern:    regexp.MustCompile(`^salmonitor/(?P<unitAddress>.+)/(?P<application>.+)`),
+		mmiMonitorPattern:    regexp.MustCompile(`^mmimonitor/(?P<unitAddress>.+)/(?P<application>.+)`),
+		unityQuery:           regexp.MustCompile(`^info/(?P<unitAddress>.+)/(?P<identifyAttribute>.+)`),
 	}
 }
 
@@ -306,7 +306,9 @@ func (m FieldHandler) ParseQuery(query string) (model.PlcField, error) {
 
 		var attribute *readWriteModel.Attribute
 		attributeArgument := match["identifyAttribute"]
-		if strings.HasPrefix(attributeArgument, "0x") {
+		if attributeArgument == "*" {
+			attribute = nil
+		} else if strings.HasPrefix(attributeArgument, "0x") {
 			decodedHex, err := hex.DecodeString(attributeArgument[2:])
 			if err != nil {
 				return nil, errors.Wrap(err, "Not a valid hex")
diff --git a/plc4go/internal/cbus/MessageCodec.go b/plc4go/internal/cbus/MessageCodec.go
index bd9738a29..4c3a912db 100644
--- a/plc4go/internal/cbus/MessageCodec.go
+++ b/plc4go/internal/cbus/MessageCodec.go
@@ -42,10 +42,10 @@ type MessageCodec struct {
 	hashEncountered uint
 }
 
-func NewMessageCodec(transportInstance transports.TransportInstance, srchk bool) *MessageCodec {
+func NewMessageCodec(transportInstance transports.TransportInstance) *MessageCodec {
 	codec := &MessageCodec{
 		requestContext: readwriteModel.NewRequestContext(false),
-		cbusOptions:    readwriteModel.NewCBusOptions(false, false, false, false, false, false, false, false, srchk),
+		cbusOptions:    readwriteModel.NewCBusOptions(false, false, false, false, false, false, false, false, false),
 		monitoredSALs:  make(chan readwriteModel.MonitoredSAL, 100),
 	}
 	codec.DefaultCodec = _default.NewDefaultCodec(codec, transportInstance, _default.WithCustomMessageHandler(func(codec _default.DefaultCodecRequirements, message spi.Message) bool {
@@ -192,6 +192,7 @@ lookingForTheEnd:
 	rb := utils.NewReadBufferByteBased(read)
 	cBusMessage, err := readwriteModel.CBusMessageParse(rb, pciResponse, m.requestContext, m.cbusOptions)
 	if err != nil {
+		log.Debug().Err(err).Msg("First Parse Failed")
 		{ // Try SAL
 			rb := utils.NewReadBufferByteBased(read)
 			cBusMessage, secondErr := readwriteModel.CBusMessageParse(rb, pciResponse, readwriteModel.NewRequestContext(false), m.cbusOptions)
diff --git a/plc4go/internal/cbus/Reader.go b/plc4go/internal/cbus/Reader.go
index 8df3d1b18..8c87455a0 100644
--- a/plc4go/internal/cbus/Reader.go
+++ b/plc4go/internal/cbus/Reader.go
@@ -119,12 +119,10 @@ func (m *Reader) Read(readRequest model.PlcReadRequest) <-chan model.PlcReadRequ
 						log.Trace().Msg("convert response to ")
 						cbusMessage := receivedMessage.(readWriteModel.CBusMessage)
 						messageToClient := cbusMessage.(readWriteModel.CBusMessageToClient)
-						confirmation := messageToClient.GetReply().(readWriteModel.ReplyOrConfirmationConfirmationExactly)
-						if !confirmation.GetConfirmation().GetIsSuccess() {
+						replyOrConfirmationConfirmation := messageToClient.GetReply().(readWriteModel.ReplyOrConfirmationConfirmationExactly)
+						if !replyOrConfirmationConfirmation.GetConfirmation().GetIsSuccess() {
 							var responseCode model.PlcResponseCode
-							switch confirmation.GetConfirmation().GetConfirmationType() {
-							case readWriteModel.ConfirmationType_CONFIRMATION_SUCCESSFUL:
-								responseCode = model.PlcResponseCode_OK
+							switch replyOrConfirmationConfirmation.GetConfirmation().GetConfirmationType() {
 							case readWriteModel.ConfirmationType_NOT_TRANSMITTED_TO_MANY_RE_TRANSMISSIONS:
 								responseCode = model.PlcResponseCode_REMOTE_ERROR
 							case readWriteModel.ConfirmationType_NOT_TRANSMITTED_CORRUPTION:
@@ -133,14 +131,26 @@ func (m *Reader) Read(readRequest model.PlcReadRequest) <-chan model.PlcReadRequ
 								responseCode = model.PlcResponseCode_REMOTE_BUSY
 							case readWriteModel.ConfirmationType_NOT_TRANSMITTED_TOO_LONG:
 								responseCode = model.PlcResponseCode_INVALID_DATA
+							default:
+								panic("Every code should be mapped here")
 							}
+							log.Trace().Msgf("Was no success %s:%s", fieldNameCopy, responseCode)
 							addResponseCode(fieldNameCopy, responseCode)
-							return nil
+							requestWasOk <- true
+							return transaction.EndRequest()
 						}
 
+						alpha := replyOrConfirmationConfirmation.GetConfirmation().GetAlpha()
 						// TODO: it could be double confirmed but this is not implemented yet
-						embeddedReply := confirmation.GetEmbeddedReply().(readWriteModel.ReplyOrConfirmationReplyExactly)
+						embeddedReply, ok := replyOrConfirmationConfirmation.GetEmbeddedReply().(readWriteModel.ReplyOrConfirmationReplyExactly)
+						if !ok {
+							log.Trace().Msgf("Is a confirm only, no data. Alpha: %c", alpha.GetCharacter())
+							addResponseCode(fieldNameCopy, model.PlcResponseCode_NOT_FOUND)
+							requestWasOk <- true
+							return transaction.EndRequest()
+						}
 
+						log.Trace().Msg("Handling confirmed data")
 						switch reply := embeddedReply.GetReply().(readWriteModel.ReplyEncodedReply).GetEncodedReply().(type) {
 						case readWriteModel.EncodedReplyCALReplyExactly:
 							calData := reply.GetCalReply().GetCalData()
@@ -210,6 +220,8 @@ func (m *Reader) Read(readRequest model.PlcReadRequest) <-chan model.PlcReadRequ
 							}
 							// 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))
 						}
 						requestWasOk <- true
 						return transaction.EndRequest()
diff --git a/plc4go/internal/knxnetip/Browser.go b/plc4go/internal/knxnetip/Browser.go
index 247ca225b..3702158d4 100644
--- a/plc4go/internal/knxnetip/Browser.go
+++ b/plc4go/internal/knxnetip/Browser.go
@@ -58,14 +58,6 @@ func (m Browser) Browse(browseRequest apiModel.PlcBrowseRequest) <-chan apiModel
 
 func (m Browser) BrowseWithInterceptor(browseRequest apiModel.PlcBrowseRequest, interceptor func(result apiModel.PlcBrowseEvent) bool) <-chan apiModel.PlcBrowseRequestResult {
 	result := make(chan apiModel.PlcBrowseRequestResult)
-	sendResult := func(browseResponse apiModel.PlcBrowseResponse, err error) {
-		result <- &model.DefaultPlcBrowseRequestResult{
-			Request:  browseRequest,
-			Response: browseResponse,
-			Err:      err,
-		}
-	}
-
 	go func() {
 		responseCodes := map[string]apiModel.PlcResponseCode{}
 		results := map[string][]apiModel.PlcBrowseFoundField{}
@@ -93,7 +85,11 @@ func (m Browser) BrowseWithInterceptor(browseRequest apiModel.PlcBrowseRequest,
 				responseCodes[fieldName] = apiModel.PlcResponseCode_INTERNAL_ERROR
 			}
 		}
-		sendResult(model.NewDefaultPlcBrowseResponse(browseRequest, results, responseCodes), nil)
+		result <- &model.DefaultPlcBrowseRequestResult{
+			Request:  browseRequest,
+			Response: model.NewDefaultPlcBrowseResponse(browseRequest, results, responseCodes),
+			Err:      nil,
+		}
 	}()
 	return result
 }
diff --git a/plc4go/internal/spi/model/DefaultPlcBrowseQueryResult.go b/plc4go/internal/spi/model/DefaultPlcBrowseQueryResult.go
index 8888eb868..21bd11a9b 100644
--- a/plc4go/internal/spi/model/DefaultPlcBrowseQueryResult.go
+++ b/plc4go/internal/spi/model/DefaultPlcBrowseQueryResult.go
@@ -20,6 +20,8 @@
 package model
 
 import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/internal/spi/utils"
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
 	"github.com/apache/plc4x/plc4go/pkg/api/values"
 )
@@ -61,3 +63,67 @@ func (d *DefaultPlcBrowseQueryResult) GetPossibleDataTypes() []string {
 func (d *DefaultPlcBrowseQueryResult) GetAttributes() map[string]values.PlcValue {
 	return d.Attributes
 }
+
+func (d *DefaultPlcBrowseQueryResult) Serialize(writeBuffer utils.WriteBuffer) error {
+	if err := writeBuffer.PushContext("PlcBrowseQueryResult"); err != nil {
+		return err
+	}
+
+	var fieldAsString string
+	fieldAsString = fmt.Sprintf("%s", d.Field)
+	if err := writeBuffer.WriteString("field", uint32(len(fieldAsString)*8), "UTF-8", fieldAsString); err != nil {
+		return err
+	}
+	if err := writeBuffer.WriteString("name", uint32(len(d.Name)*8), "UTF-8", d.Name); err != nil {
+		return err
+	}
+	if err := writeBuffer.WriteBit("readable", d.Readable); err != nil {
+		return err
+	}
+	if err := writeBuffer.WriteBit("writable", d.Writable); err != nil {
+		return err
+	}
+	if err := writeBuffer.WriteBit("subscribable", d.Subscribable); err != nil {
+		return err
+	}
+	if err := writeBuffer.PushContext("possibleDataTypes", utils.WithRenderAsList(true)); err != nil {
+		return err
+	}
+	for _, dataType := range d.PossibleDataTypes {
+		if err := writeBuffer.WriteString("", uint32(len(dataType)*8), "UTF-8", dataType); err != nil {
+			return err
+		}
+	}
+	if err := writeBuffer.PopContext("possibleDataTypes", utils.WithRenderAsList(true)); err != nil {
+		return err
+	}
+	if err := writeBuffer.PushContext("attributes", utils.WithRenderAsList(true)); err != nil {
+		return err
+	}
+	for name, plcValue := range d.Attributes {
+		if serializable, ok := plcValue.(utils.Serializable); ok {
+			if err := writeBuffer.PushContext(name); err != nil {
+				return err
+			}
+			if err := serializable.Serialize(writeBuffer); err != nil {
+				return err
+			}
+			if err := writeBuffer.PopContext(name); err != nil {
+				return err
+			}
+		} else {
+			plcValueAsString := fmt.Sprintf("%v", plcValue)
+			if err := writeBuffer.WriteString(name, uint32(len(plcValueAsString)*8), "UTF-8", plcValueAsString); err != nil {
+				return err
+			}
+		}
+	}
+	if err := writeBuffer.PopContext("attributes", utils.WithRenderAsList(true)); err != nil {
+		return err
+	}
+
+	if err := writeBuffer.PopContext("PlcBrowseQueryResult"); err != nil {
+		return err
+	}
+	return nil
+}
diff --git a/plc4go/internal/spi/model/DefaultPlcBrowseRequest.go b/plc4go/internal/spi/model/DefaultPlcBrowseRequest.go
index beeb2fbc4..580cef577 100644
--- a/plc4go/internal/spi/model/DefaultPlcBrowseRequest.go
+++ b/plc4go/internal/spi/model/DefaultPlcBrowseRequest.go
@@ -40,6 +40,7 @@ func NewDefaultPlcBrowseRequestBuilder(fieldHandler spi.PlcFieldHandler, browser
 		fieldHandler: fieldHandler,
 		browser:      browser,
 		queries:      map[string]string{},
+		fields:       map[string]model.PlcField{},
 	}
 }
 
diff --git a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
index 57bd809c8..0f8302b9e 100644
--- a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
+++ b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
@@ -93,3 +93,33 @@ func TestManualCBusDriver(t *testing.T) {
 		t.Logf("Got %d monitors", monitorCount)
 	})
 }
+
+func TestManualCBusBrowse(t *testing.T) {
+	log.Logger = log.
+		With().Caller().Logger().
+		Output(zerolog.ConsoleWriter{Out: os.Stderr}).
+		Level(zerolog.TraceLevel)
+	config.TraceTransactionManagerWorkers = true
+	config.TraceTransactionManagerTransactions = true
+	config.TraceDefaultMessageCodecWorker = true
+	t.Skip()
+
+	connectionString := "c-bus://192.168.178.101?Monitor=false"
+	driverManager := plc4go.NewPlcDriverManager()
+	driverManager.RegisterDriver(cbus.NewDriver())
+	transports.RegisterTcpTransport(driverManager)
+	connectionResult := <-driverManager.GetConnection(connectionString)
+	if err := connectionResult.GetErr(); err != nil {
+		panic(err)
+	}
+	connection := connectionResult.GetConnection()
+	defer connection.Close()
+	browseRequest, err := connection.BrowseRequestBuilder().
+		AddQuery("asd", "info/*/*").
+		Build()
+	if err != nil {
+		panic(err)
+	}
+	browseRequestResult := <-browseRequest.Execute()
+	fmt.Printf("%s", browseRequestResult)
+}
diff --git a/plc4go/tools/plc4xbrowser/commands.go b/plc4go/tools/plc4xbrowser/commands.go
index 0241a659d..44fbe8829 100644
--- a/plc4go/tools/plc4xbrowser/commands.go
+++ b/plc4go/tools/plc4xbrowser/commands.go
@@ -264,7 +264,7 @@ var rootCommand = Command{
 				} else {
 					start := time.Now()
 					browseRequest, err := connection.BrowseRequestBuilder().
-						AddQuery("writeField", split[1]).
+						AddQuery("browseField", split[1]).
 						Build()
 					if err != nil {
 						return errors.Wrapf(err, "%s can't browse", connectionsString)


[plc4x] 03/04: chore(cbus): fixed code commend

Posted by sr...@apache.org.
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 33b83d3f4fbee10ffa8cedd346852727528941c4
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Mon Aug 8 17:59:51 2022 +0200

    chore(cbus): fixed code commend
---
 protocols/c-bus/src/main/resources/protocols/cbus/c-bus.mspec | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/protocols/c-bus/src/main/resources/protocols/cbus/c-bus.mspec b/protocols/c-bus/src/main/resources/protocols/cbus/c-bus.mspec
index 9fbd237a1..17ae4e22a 100644
--- a/protocols/c-bus/src/main/resources/protocols/cbus/c-bus.mspec
+++ b/protocols/c-bus/src/main/resources/protocols/cbus/c-bus.mspec
@@ -89,7 +89,7 @@
                               calDataDecoded 'calData'                      ]
         ]
         ['REQUEST_COMMAND' *Command
-            [const    byte  initiator 0x5C                                  ] // 0x5C == "/"
+            [const    byte  initiator 0x5C                                  ] // 0x5C == "\"
             [manual   CBusCommand
                               cbusCommand
                         'STATIC_CALL("readCBusCommand", readBuffer, cBusOptions, cBusOptions.srchk)'


[plc4x] 02/04: fix(plc-simulator/cbus): fixed some wrong returns

Posted by sr...@apache.org.
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 ff9fe8d5fe964cfa17e75f21b9c7530f4a266a40
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Mon Aug 8 17:59:31 2022 +0200

    fix(plc-simulator/cbus): fixed some wrong returns
---
 .../server/cbus/protocol/CBusServerAdapter.java       | 19 ++++++++++++++-----
 1 file changed, 14 insertions(+), 5 deletions(-)

diff --git a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/cbus/protocol/CBusServerAdapter.java b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/cbus/protocol/CBusServerAdapter.java
index 9a890561d..71d7e8a24 100644
--- a/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/cbus/protocol/CBusServerAdapter.java
+++ b/sandbox/plc-simulator/src/main/java/org/apache/plc4x/simulator/server/cbus/protocol/CBusServerAdapter.java
@@ -25,6 +25,7 @@ import org.apache.plc4x.simulator.model.Context;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.ScheduledFuture;
@@ -34,6 +35,8 @@ import java.util.concurrent.locks.ReentrantLock;
 
 public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
 
+    private static final List<Byte> AVAILABLE_UNITS = Arrays.asList((byte) 0, (byte) 23, (byte) 48);
+
     private static final Logger LOGGER = LoggerFactory.getLogger(CBusServerAdapter.class);
 
     private Context context;
@@ -82,6 +85,8 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
         if (!(msg instanceof CBusMessage)) {
             return;
         }
+        // Serial is slow
+        TimeUnit.MILLISECONDS.sleep(100);
         if (!smart && !connect) {
             // In this mode every message will be echoed
             LOGGER.info("Sending echo");
@@ -211,7 +216,7 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                 if (cbusCommand instanceof CBusCommandPointToPoint) {
                     CBusCommandPointToPoint cBusCommandPointToPoint = (CBusCommandPointToPoint) cbusCommand;
                     CBusPointToPointCommand command = cBusCommandPointToPoint.getCommand();
-                    UnitAddress unitAddress;
+                    UnitAddress unitAddress = null;
                     if (command instanceof CBusPointToPointCommandIndirect) {
                         CBusPointToPointCommandIndirect cBusPointToPointCommandIndirect = (CBusPointToPointCommandIndirect) command;
                         // TODO: handle bridgeAddress
@@ -222,6 +227,10 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                         CBusPointToPointCommandDirect cBusPointToPointCommandDirect = (CBusPointToPointCommandDirect) command;
                         unitAddress = cBusPointToPointCommandDirect.getUnitAddress();
                     }
+                    if (unitAddress == null) {
+                        throw new IllegalStateException("Unit address should be set at this point");
+                    }
+                    boolean knownUnit = AVAILABLE_UNITS.contains(unitAddress.getAddress());
                     CALData calData = command.getCalData();
                     // TODO: handle other Datatypes
                     if (calData instanceof CALDataIdentify) {
@@ -250,12 +259,12 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                                 identifyReplyCommand = new IdentifyReplyCommandExtendedDiagnosticSummary(ApplicationIdContainer.FREE_USAGE_01, ApplicationIdContainer.FREE_USAGE_0F, (byte) 0x0, 0x0, 4711l, (byte) 0x13, false, false, false, true, false, false, false, false, false, false, false, false, false, numBytes);
                                 break;
                             case NetworkTerminalLevels:
-                                numBytes = 0x0D;
-                                identifyReplyCommand = new IdentifyReplyCommandNetworkTerminalLevels(new byte[]{0x13}, numBytes);
+                                numBytes = 0x0C;
+                                identifyReplyCommand = new IdentifyReplyCommandNetworkTerminalLevels(new byte[]{0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13}, numBytes);
                                 break;
                             case TerminalLevel:
-                                numBytes = 0x0D;
-                                identifyReplyCommand = new IdentifyReplyCommandTerminalLevels(new byte[]{0x13}, numBytes);
+                                numBytes = 0x0C;
+                                identifyReplyCommand = new IdentifyReplyCommandTerminalLevels(new byte[]{0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13, 0x13}, numBytes);
                                 break;
                             case NetworkVoltage:
                                 numBytes = 0x05;