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/30 10:33:02 UTC

[plc4x] branch develop updated (a58916131 -> 80ba5d0cc)

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 a58916131 feat(plc4go/cbus): handle context in browse field
     new 43f4b87f1 fix(plc-simulator): fixed issue where the mmi monitor would reset the srchk option
     new 324ef77f3 feat(plc4go/spi): improved output of some types
     new 80ba5d0cc refactor(plc4go/cbus): improved debug output

The 3 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/MessageCodec.go               |  10 +-
 plc4go/spi/default/DefaultCodec.go                 |   2 +-
 plc4go/spi/model/DefaultPlcBrowseResponse.go       |  52 ++
 plc4go/spi/model/DefaultRequest.go                 |  60 ++-
 plc4go/spi/model/DefaultResponse.go                |  41 +-
 .../tests/drivers/tests/manual_cbus_driver_test.go |   6 +-
 .../server/cbus/protocol/CBusServerAdapter.java    | 540 +++++++++++----------
 7 files changed, 435 insertions(+), 276 deletions(-)


[plc4x] 02/03: feat(plc4go/spi): improved output of some types

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 324ef77f3d93ad9e4a25502366ea3e9d13c0937d
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 30 12:32:12 2022 +0200

    feat(plc4go/spi): improved output of some types
---
 plc4go/spi/default/DefaultCodec.go           |  2 +-
 plc4go/spi/model/DefaultPlcBrowseResponse.go | 52 ++++++++++++++++++++++++
 plc4go/spi/model/DefaultRequest.go           | 60 +++++++++++++++++++++++++++-
 plc4go/spi/model/DefaultResponse.go          | 41 ++++++++++++++++++-
 4 files changed, 152 insertions(+), 3 deletions(-)

diff --git a/plc4go/spi/default/DefaultCodec.go b/plc4go/spi/default/DefaultCodec.go
index 25937d8a5..ec70c0ef9 100644
--- a/plc4go/spi/default/DefaultCodec.go
+++ b/plc4go/spi/default/DefaultCodec.go
@@ -332,7 +332,7 @@ mainLoop:
 				}
 			case <-timeout.C:
 				timeout.Stop()
-				workerLog.Warn().Msgf("Message discarded %s", message)
+				workerLog.Warn().Msgf("Message discarded\n%s", message)
 			}
 		}
 	}
diff --git a/plc4go/spi/model/DefaultPlcBrowseResponse.go b/plc4go/spi/model/DefaultPlcBrowseResponse.go
index 4bc37e2b3..36c19c9e9 100644
--- a/plc4go/spi/model/DefaultPlcBrowseResponse.go
+++ b/plc4go/spi/model/DefaultPlcBrowseResponse.go
@@ -21,6 +21,8 @@ package model
 
 import (
 	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+	"github.com/pkg/errors"
 )
 
 type DefaultPlcBrowseResponse struct {
@@ -63,3 +65,53 @@ func (d DefaultPlcBrowseResponse) GetQueryNames() []string {
 func (d DefaultPlcBrowseResponse) GetQueryResults(queryName string) []model.PlcBrowseFoundField {
 	return d.results[queryName]
 }
+
+func (d DefaultPlcBrowseResponse) Serialize(writeBuffer utils.WriteBuffer) error {
+	if err := writeBuffer.PushContext("PlcBrowseResponse"); err != nil {
+		return err
+	}
+
+	if serializableRequest, ok := d.request.(utils.Serializable); ok {
+		if err := serializableRequest.Serialize(writeBuffer); err != nil {
+			return err
+		}
+	} else {
+		return errors.Errorf("Error serializing. Request %T doesn't implement Serializable", d.request)
+	}
+
+	if err := writeBuffer.PushContext("results"); err != nil {
+		return err
+	}
+	for fieldName, foundFields := range d.results {
+		if err := writeBuffer.PushContext(fieldName); err != nil {
+			return err
+		}
+		for _, field := range foundFields {
+			if serializableField, ok := field.(utils.Serializable); ok {
+				if err := serializableField.Serialize(writeBuffer); err != nil {
+					return err
+				}
+			} else {
+				return errors.Errorf("Error serializing. Field %T doesn't implement Serializable", field)
+			}
+		}
+		if err := writeBuffer.PopContext(fieldName); err != nil {
+			return err
+		}
+	}
+	if err := writeBuffer.PopContext("results"); err != nil {
+		return err
+	}
+	if err := writeBuffer.PopContext("PlcBrowseResponse"); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (d DefaultPlcBrowseResponse) String() string {
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(d); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/spi/model/DefaultRequest.go b/plc4go/spi/model/DefaultRequest.go
index ce1993b4d..5f8263874 100644
--- a/plc4go/spi/model/DefaultRequest.go
+++ b/plc4go/spi/model/DefaultRequest.go
@@ -19,7 +19,11 @@
 
 package model
 
-import "github.com/apache/plc4x/plc4go/pkg/api/model"
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+)
 
 type DefaultRequest struct {
 	fields     map[string]model.PlcField
@@ -44,3 +48,57 @@ func (m DefaultRequest) GetField(name string) model.PlcField {
 	}
 	return nil
 }
+
+func (m DefaultRequest) Serialize(writeBuffer utils.WriteBuffer) error {
+	if err := writeBuffer.PushContext("Request"); err != nil {
+		return err
+	}
+
+	if err := writeBuffer.PushContext("fields"); err != nil {
+		return err
+	}
+	for fieldName, field := range m.fields {
+		if serializableField, ok := field.(utils.Serializable); ok {
+			if err := writeBuffer.PushContext(fieldName); err != nil {
+				return err
+			}
+			if err := serializableField.Serialize(writeBuffer); err != nil {
+				return err
+			}
+			if err := writeBuffer.PopContext(fieldName); err != nil {
+				return err
+			}
+		} else {
+			fieldString := fmt.Sprintf("%v", field)
+			if err := writeBuffer.WriteString(fieldName, uint32(len(fieldString)*8), "UTF-8", fieldString); err != nil {
+				return err
+			}
+		}
+	}
+	if err := writeBuffer.PopContext("fields"); err != nil {
+		return err
+	}
+	if err := writeBuffer.PushContext("fieldNames"); err != nil {
+		return err
+	}
+	for _, name := range m.fieldNames {
+		if err := writeBuffer.WriteString("value", uint32(len(name)*8), "UTF-8", name); err != nil {
+			return err
+		}
+	}
+	if err := writeBuffer.PopContext("fieldNames"); err != nil {
+		return err
+	}
+	if err := writeBuffer.PopContext("Request"); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (m DefaultRequest) String() string {
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}
diff --git a/plc4go/spi/model/DefaultResponse.go b/plc4go/spi/model/DefaultResponse.go
index 2d3ff5666..8df022127 100644
--- a/plc4go/spi/model/DefaultResponse.go
+++ b/plc4go/spi/model/DefaultResponse.go
@@ -19,7 +19,10 @@
 
 package model
 
-import "github.com/apache/plc4x/plc4go/pkg/api/model"
+import (
+	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/apache/plc4x/plc4go/spi/utils"
+)
 
 type DefaultResponse struct {
 	responseCodes map[string]model.PlcResponseCode
@@ -36,3 +39,39 @@ func (m DefaultResponse) GetResponseCode(name string) model.PlcResponseCode {
 func NewDefaultResponse(responseCodes map[string]model.PlcResponseCode) DefaultResponse {
 	return DefaultResponse{responseCodes}
 }
+
+func (m DefaultResponse) Serialize(writeBuffer utils.WriteBuffer) error {
+	if err := writeBuffer.PushContext("Response"); err != nil {
+		return err
+	}
+
+	if err := writeBuffer.PushContext("responseCodes"); err != nil {
+		return err
+	}
+	for fieldName, code := range m.responseCodes {
+		if err := writeBuffer.PushContext(fieldName); err != nil {
+			return err
+		}
+		if err := writeBuffer.WriteUint8("code", 8, uint8(code), utils.WithAdditionalStringRepresentation(code.String())); err != nil {
+			return err
+		}
+		if err := writeBuffer.PopContext(fieldName); err != nil {
+			return err
+		}
+	}
+	if err := writeBuffer.PopContext("responseCodes"); err != nil {
+		return err
+	}
+	if err := writeBuffer.PopContext("Response"); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (m DefaultResponse) String() string {
+	writeBuffer := utils.NewWriteBufferBoxBasedWithOptions(true, true)
+	if err := writeBuffer.WriteSerializable(m); err != nil {
+		return err.Error()
+	}
+	return writeBuffer.GetBox().String()
+}


[plc4x] 03/03: refactor(plc4go/cbus): improved debug output

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 80ba5d0cc156b98b92c5855245ab750d2edfa87e
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 30 12:32:54 2022 +0200

    refactor(plc4go/cbus): improved debug output
---
 plc4go/internal/cbus/MessageCodec.go                  | 10 +++++++---
 plc4go/tests/drivers/tests/manual_cbus_driver_test.go |  6 +++---
 2 files changed, 10 insertions(+), 6 deletions(-)

diff --git a/plc4go/internal/cbus/MessageCodec.go b/plc4go/internal/cbus/MessageCodec.go
index ee5ed3a36..5d68716e9 100644
--- a/plc4go/internal/cbus/MessageCodec.go
+++ b/plc4go/internal/cbus/MessageCodec.go
@@ -67,6 +67,7 @@ func (m *MessageCodec) Send(message spi.Message) error {
 
 	// Set the right request context
 	m.requestContext = CreateRequestContext(cbusMessage)
+	log.Debug().Msgf("Created request context\n%s", m.requestContext)
 
 	// Serialize the request
 	wb := utils.NewWriteBufferByteBased()
@@ -107,7 +108,7 @@ func (m *MessageCodec) Receive() (spi.Message, error) {
 				return true
 			}
 		}); err != nil {
-			return nil, err
+			return nil, errors.Wrap(err, "error filling buffer")
 		}
 	}
 
@@ -129,6 +130,7 @@ func (m *MessageCodec) Receive() (spi.Message, error) {
 	// Check for an isolated error
 	if bytes, err := ti.PeekReadableBytes(1); err == nil && (bytes[0] == byte(readWriteModel.ConfirmationType_CHECKSUM_FAILURE)) {
 		_, _ = ti.Read(1)
+		// Report one Error at a time
 		return readWriteModel.CBusMessageParse(utils.NewReadBufferByteBased(bytes), true, m.requestContext, m.cbusOptions)
 	}
 
@@ -257,13 +259,15 @@ lookingForTheEnd:
 			}
 		}
 	}
+	log.Debug().Msgf("Parsing %q", sanitizedInput)
 	rb := utils.NewReadBufferByteBased(sanitizedInput)
 	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(sanitizedInput)
-			cBusMessage, secondErr := readWriteModel.CBusMessageParse(rb, pciResponse, readWriteModel.NewRequestContext(false), m.cbusOptions)
+			requestContext := readWriteModel.NewRequestContext(false)
+			cBusMessage, secondErr := readWriteModel.CBusMessageParse(rb, pciResponse, requestContext, m.cbusOptions)
 			if secondErr == nil {
 				return cBusMessage, nil
 			} else {
@@ -271,9 +275,9 @@ lookingForTheEnd:
 			}
 		}
 		{ // Try MMI
+			rb := utils.NewReadBufferByteBased(sanitizedInput)
 			requestContext := readWriteModel.NewRequestContext(false)
 			cbusOptions := readWriteModel.NewCBusOptions(false, false, false, false, false, false, false, false, false)
-			rb := utils.NewReadBufferByteBased(sanitizedInput)
 			cBusMessage, secondErr := readWriteModel.CBusMessageParse(rb, true, requestContext, cbusOptions)
 			if secondErr == nil {
 				return cBusMessage, nil
diff --git a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
index b385a4e09..541690fdf 100644
--- a/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
+++ b/plc4go/tests/drivers/tests/manual_cbus_driver_test.go
@@ -118,16 +118,16 @@ func TestManualCBusBrowse(t *testing.T) {
 	connection := connectionResult.GetConnection()
 	defer connection.Close()
 	browseRequest, err := connection.BrowseRequestBuilder().
-		AddQuery("asd", "info/*/*").
+		AddQuery("infoQuery", "info/*/*").
 		Build()
 	if err != nil {
 		panic(err)
 	}
 	browseRequestResult := <-browseRequest.ExecuteWithInterceptor(func(result model.PlcBrowseEvent) bool {
-		fmt.Printf("%s", result)
+		fmt.Printf("%s\n", result)
 		return true
 	})
-	fmt.Printf("%s", browseRequestResult.GetResponse())
+	fmt.Printf("%v\n", browseRequestResult.GetResponse())
 }
 
 func TestManualCBusRead(t *testing.T) {


[plc4x] 01/03: fix(plc-simulator): fixed issue where the mmi monitor would reset the srchk option

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 43f4b87f182f31e008755b47e778ff5fc520e3d7
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 30 12:31:36 2022 +0200

    fix(plc-simulator): fixed issue where the mmi monitor would reset the srchk option
---
 .../server/cbus/protocol/CBusServerAdapter.java    | 540 +++++++++++----------
 1 file changed, 273 insertions(+), 267 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 6f53d2011..ac543dc15 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
@@ -58,13 +58,14 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
     private static byte monitorApplicationAddress2;
     private static CBusOptions cBusOptions;
 
-    private final Lock writeLock = new ReentrantLock();
+    private final Lock outputLock = new ReentrantLock();
 
     private ScheduledFuture<?> salMonitorFuture;
 
     private ScheduledFuture<?> mmiMonitorFuture;
 
     public CBusServerAdapter(Context context) {
+        LOGGER.info("Creating adapter with context {}", context);
         this.context = context;
         cBusOptions = new CBusOptions(connect, smart, idmon, exstat, monitor, monall, pun, pcn, srchk);
     }
@@ -89,6 +90,17 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
         if (!(msg instanceof CBusMessage)) {
             return;
         }
+        try {
+            outputLock.lock();
+            syncChannelRead(ctx, msg);
+        } finally {
+            ctx.flush();
+            outputLock.unlock();
+        }
+    }
+
+    private void syncChannelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
+        LOGGER.debug("Working with cBusOptions\n{}", cBusOptions);
         // Serial is slow
         TimeUnit.MILLISECONDS.sleep(100);
         if (!smart && !connect) {
@@ -96,293 +108,287 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
             LOGGER.info("Sending echo");
             ctx.write(msg);
         }
-        try {
-            writeLock.lock();
-            CBusMessage packet = (CBusMessage) msg;
-            if (packet instanceof CBusMessageToClient) {
-                LOGGER.info("Message to client not supported\n{}", packet);
-                return;
-            }
-            CBusMessageToServer cBusMessageToServer = (CBusMessageToServer) packet;
-            Request request = cBusMessageToServer.getRequest();
-            if (request instanceof RequestEmpty || request instanceof RequestNull) {
-                LOGGER.debug("Ignoring\n{}", request);
-                return;
-            }
-            if (request instanceof RequestDirectCommandAccess) {
-                RequestDirectCommandAccess requestDirectCommandAccess = (RequestDirectCommandAccess) request;
-                CALData calData = requestDirectCommandAccess.getCalData();
-                LOGGER.info("Handling RequestDirectCommandAccess\n{}\n{}", requestDirectCommandAccess, calData);
+        CBusMessage packet = (CBusMessage) msg;
+        if (packet instanceof CBusMessageToClient) {
+            LOGGER.info("Message to client not supported\n{}", packet);
+            return;
+        }
+        CBusMessageToServer cBusMessageToServer = (CBusMessageToServer) packet;
+        Request request = cBusMessageToServer.getRequest();
+        if (request instanceof RequestEmpty || request instanceof RequestNull) {
+            LOGGER.debug("Ignoring\n{}", request);
+            return;
+        }
+        if (request instanceof RequestDirectCommandAccess) {
+            RequestDirectCommandAccess requestDirectCommandAccess = (RequestDirectCommandAccess) request;
+            CALData calData = requestDirectCommandAccess.getCalData();
+            LOGGER.info("Handling RequestDirectCommandAccess\n{}\n{}", requestDirectCommandAccess, calData);
 
-                // TODO: handle other cal data type
-                if (calData instanceof CALDataWrite) {
-                    CALDataWrite calDataWrite = (CALDataWrite) calData;
-                    Runnable acknowledger = () -> {
-                        CALDataAcknowledge calDataAcknowledge = new CALDataAcknowledge(CALCommandTypeContainer.CALCommandAcknowledge, null, calDataWrite.getParamNo(), (short) 0x0, requestContext);
-                        CALReplyShort calReply = new CALReplyShort((byte) 0x0, calDataAcknowledge, cBusOptions, requestContext);
-                        EncodedReplyCALReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
-                        ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0x0, encodedReply, null, cBusOptions, requestContext);
-                        ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0x0, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
-                        if (requestDirectCommandAccess.getAlpha() != null) {
-                            replyOrConfirmation = new ReplyOrConfirmationConfirmation((byte) 0x0, new Confirmation(requestDirectCommandAccess.getAlpha(), null, ConfirmationType.CONFIRMATION_SUCCESSFUL), replyOrConfirmation, cBusOptions, requestContext);
-                        }
-                        CBusMessageToClient cBusMessageToClient = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
-                        LOGGER.info("Sending ack\n{}\n{}", cBusMessageToClient, encodedReply);
-                        ctx.writeAndFlush(cBusMessageToClient);
-                    };
-                    switch (calDataWrite.getParamNo().getParameterType()) {
-                        case APPLICATION_ADDRESS_1:
-                            ApplicationAddress1 applicationAddress1 = ((ParameterValueApplicationAddress1) calDataWrite.getParameterValue()).getValue();
-                            monitorApplicationAddress1 = applicationAddress1.getAddress();
-                            acknowledger.run();
-                            return;
-                        case APPLICATION_ADDRESS_2:
-                            ApplicationAddress2 applicationAddress2 = ((ParameterValueApplicationAddress2) calDataWrite.getParameterValue()).getValue();
-                            monitorApplicationAddress2 = applicationAddress2.getAddress();
-                            acknowledger.run();
-                            return;
-                        case INTERFACE_OPTIONS_1:
-                            InterfaceOptions1 interfaceOptions1 = ((ParameterValueInterfaceOptions1) calDataWrite.getParameterValue()).getValue();
-                            idmon = interfaceOptions1.getIdmon();
-                            monitor = interfaceOptions1.getMonitor();
-                            if (monitor) startMMIMonitor(ctx);
-                            else stopMMIMonitor();
-                            smart = interfaceOptions1.getSmart();
-                            srchk = interfaceOptions1.getSrchk();
-                            // TODO: add support for xonxoff
-                            // xonxoff = interfaceOptions1.getXonXoff();
-                            connect = interfaceOptions1.getConnect();
-                            if (connect) startSALMonitor(ctx);
-                            else stopSALMonitor();
-                            buildCBusOptions();
-                            acknowledger.run();
-                            return;
-                        case INTERFACE_OPTIONS_2:
-                            InterfaceOptions2 interfaceOptions2 = ((ParameterValueInterfaceOptions2) calDataWrite.getParameterValue()).getValue();
-                            // TODO: add support for burden
-                            // burden =  interfaceOptions2.getBurden();
-                            // TODO: add support for clockgen
-                            // clockgen = interfaceOptions2.getClockGen();
-                            buildCBusOptions();
-                            acknowledger.run();
-                            return;
-                        case INTERFACE_OPTIONS_3:
-                            InterfaceOptions3 interfaceOptions3Value = ((ParameterValueInterfaceOptions3) calDataWrite.getParameterValue()).getValue();
-                            exstat = interfaceOptions3Value.getExstat();
-                            pun = interfaceOptions3Value.getPun();
-                            // TODO: add support for localsal
-                            // localsal = interfaceOptions3Value.getLocalSal();
-                            pcn = interfaceOptions3Value.getPcn();
-                            buildCBusOptions();
-                            acknowledger.run();
-                            return;
-                        case BAUD_RATE_SELECTOR:
-                            BaudRateSelector baudRateSelector = ((ParameterValueBaudRateSelector) calDataWrite.getParameterValue()).getValue();
-                            // TODO: add support for baudrate
-                            // baudrate = baudRateSelector.getValue();
-                            buildCBusOptions();
-                            acknowledger.run();
-                            return;
-                        case INTERFACE_OPTIONS_1_POWER_UP_SETTINGS:
-                            InterfaceOptions1 interfaceOptions1PowerUpSettings = ((ParameterValueInterfaceOptions1PowerUpSettings) calDataWrite.getParameterValue()).getValue().getInterfaceOptions1();
-                            idmon = interfaceOptions1PowerUpSettings.getIdmon();
-                            monitor = interfaceOptions1PowerUpSettings.getMonitor();
-                            if (monitor) startMMIMonitor(ctx);
-                            else stopMMIMonitor();
-                            smart = interfaceOptions1PowerUpSettings.getSmart();
-                            srchk = interfaceOptions1PowerUpSettings.getSrchk();
-                            // TODO: add support for xonxoff
-                            // xonxoff = interfaceOptions1PowerUpSettings.getXonXoff();
-                            connect = interfaceOptions1PowerUpSettings.getConnect();
-                            if (connect) startSALMonitor(ctx);
-                            else stopSALMonitor();
-                            buildCBusOptions();
-                            acknowledger.run();
-                            return;
-                        case CUSTOM_MANUFACTURER:
-                            // TODO: handle other parm typed
-                            acknowledger.run();
-                            return;
-                        case SERIAL_NUMBER:
-                            // TODO: handle other parm typed
-                            acknowledger.run();
-                            return;
-                        case CUSTOM_TYPE:
-                            // TODO: handle other parm typed
-                            acknowledger.run();
-                            return;
-                        default:
-                            throw new IllegalStateException("Unmapped type");
+            // TODO: handle other cal data type
+            if (calData instanceof CALDataWrite) {
+                CALDataWrite calDataWrite = (CALDataWrite) calData;
+                Runnable acknowledger = () -> {
+                    CALDataAcknowledge calDataAcknowledge = new CALDataAcknowledge(CALCommandTypeContainer.CALCommandAcknowledge, null, calDataWrite.getParamNo(), (short) 0x0, requestContext);
+                    CALReplyShort calReply = new CALReplyShort((byte) 0x0, calDataAcknowledge, cBusOptions, requestContext);
+                    EncodedReplyCALReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
+                    ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0x0, encodedReply, null, cBusOptions, requestContext);
+                    ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0x0, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
+                    if (requestDirectCommandAccess.getAlpha() != null) {
+                        replyOrConfirmation = new ReplyOrConfirmationConfirmation((byte) 0x0, new Confirmation(requestDirectCommandAccess.getAlpha(), null, ConfirmationType.CONFIRMATION_SUCCESSFUL), replyOrConfirmation, cBusOptions, requestContext);
                     }
+                    CBusMessageToClient cBusMessageToClient = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
+                    LOGGER.info("Sending ack\n{}\n{}", cBusMessageToClient, encodedReply);
+                    ctx.writeAndFlush(cBusMessageToClient);
+                };
+                switch (calDataWrite.getParamNo().getParameterType()) {
+                    case APPLICATION_ADDRESS_1:
+                        ApplicationAddress1 applicationAddress1 = ((ParameterValueApplicationAddress1) calDataWrite.getParameterValue()).getValue();
+                        monitorApplicationAddress1 = applicationAddress1.getAddress();
+                        acknowledger.run();
+                        return;
+                    case APPLICATION_ADDRESS_2:
+                        ApplicationAddress2 applicationAddress2 = ((ParameterValueApplicationAddress2) calDataWrite.getParameterValue()).getValue();
+                        monitorApplicationAddress2 = applicationAddress2.getAddress();
+                        acknowledger.run();
+                        return;
+                    case INTERFACE_OPTIONS_1:
+                        InterfaceOptions1 interfaceOptions1 = ((ParameterValueInterfaceOptions1) calDataWrite.getParameterValue()).getValue();
+                        idmon = interfaceOptions1.getIdmon();
+                        monitor = interfaceOptions1.getMonitor();
+                        if (monitor) startMMIMonitor(ctx);
+                        else stopMMIMonitor();
+                        smart = interfaceOptions1.getSmart();
+                        srchk = interfaceOptions1.getSrchk();
+                        // TODO: add support for xonxoff
+                        // xonxoff = interfaceOptions1.getXonXoff();
+                        connect = interfaceOptions1.getConnect();
+                        if (connect) startSALMonitor(ctx);
+                        else stopSALMonitor();
+                        buildCBusOptions();
+                        acknowledger.run();
+                        return;
+                    case INTERFACE_OPTIONS_2:
+                        InterfaceOptions2 interfaceOptions2 = ((ParameterValueInterfaceOptions2) calDataWrite.getParameterValue()).getValue();
+                        // TODO: add support for burden
+                        // burden =  interfaceOptions2.getBurden();
+                        // TODO: add support for clockgen
+                        // clockgen = interfaceOptions2.getClockGen();
+                        buildCBusOptions();
+                        acknowledger.run();
+                        return;
+                    case INTERFACE_OPTIONS_3:
+                        InterfaceOptions3 interfaceOptions3Value = ((ParameterValueInterfaceOptions3) calDataWrite.getParameterValue()).getValue();
+                        exstat = interfaceOptions3Value.getExstat();
+                        pun = interfaceOptions3Value.getPun();
+                        // TODO: add support for localsal
+                        // localsal = interfaceOptions3Value.getLocalSal();
+                        pcn = interfaceOptions3Value.getPcn();
+                        buildCBusOptions();
+                        acknowledger.run();
+                        return;
+                    case BAUD_RATE_SELECTOR:
+                        BaudRateSelector baudRateSelector = ((ParameterValueBaudRateSelector) calDataWrite.getParameterValue()).getValue();
+                        // TODO: add support for baudrate
+                        // baudrate = baudRateSelector.getValue();
+                        buildCBusOptions();
+                        acknowledger.run();
+                        return;
+                    case INTERFACE_OPTIONS_1_POWER_UP_SETTINGS:
+                        InterfaceOptions1 interfaceOptions1PowerUpSettings = ((ParameterValueInterfaceOptions1PowerUpSettings) calDataWrite.getParameterValue()).getValue().getInterfaceOptions1();
+                        idmon = interfaceOptions1PowerUpSettings.getIdmon();
+                        monitor = interfaceOptions1PowerUpSettings.getMonitor();
+                        if (monitor) startMMIMonitor(ctx);
+                        else stopMMIMonitor();
+                        smart = interfaceOptions1PowerUpSettings.getSmart();
+                        srchk = interfaceOptions1PowerUpSettings.getSrchk();
+                        // TODO: add support for xonxoff
+                        // xonxoff = interfaceOptions1PowerUpSettings.getXonXoff();
+                        connect = interfaceOptions1PowerUpSettings.getConnect();
+                        if (connect) startSALMonitor(ctx);
+                        else stopSALMonitor();
+                        buildCBusOptions();
+                        acknowledger.run();
+                        return;
+                    case CUSTOM_MANUFACTURER:
+                        // TODO: handle other parm typed
+                        acknowledger.run();
+                        return;
+                    case SERIAL_NUMBER:
+                        // TODO: handle other parm typed
+                        acknowledger.run();
+                        return;
+                    case CUSTOM_TYPE:
+                        // TODO: handle other parm typed
+                        acknowledger.run();
+                        return;
+                    default:
+                        throw new IllegalStateException("Unmapped type");
+                }
+            }
+            if (calData instanceof CALDataIdentify) {
+                handleCalDataIdentify(ctx, (CALDataIdentify) calData, requestDirectCommandAccess.getAlpha());
+            }
+            return;
+        }
+        if (request instanceof RequestCommand) {
+            RequestCommand requestCommand = (RequestCommand) request;
+            LOGGER.info("Handling RequestCommand\n{}", requestCommand);
+            CBusCommand cbusCommand = requestCommand.getCbusCommand();
+            LOGGER.info("Handling CBusCommand\n{}", cbusCommand);
+            if (cbusCommand instanceof CBusCommandPointToPoint) {
+                CBusCommandPointToPoint cBusCommandPointToPoint = (CBusCommandPointToPoint) cbusCommand;
+                CBusPointToPointCommand command = cBusCommandPointToPoint.getCommand();
+                UnitAddress unitAddress = null;
+                if (command instanceof CBusPointToPointCommandIndirect) {
+                    CBusPointToPointCommandIndirect cBusPointToPointCommandIndirect = (CBusPointToPointCommandIndirect) command;
+                    // TODO: handle bridgeAddress
+                    // TODO: handle networkRoute
+                    unitAddress = cBusPointToPointCommandIndirect.getUnitAddress();
+                }
+                if (command instanceof CBusPointToPointCommandDirect) {
+                    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());
+                if (!knownUnit) {
+                    LOGGER.warn("{} not a known unit", unitAddress);
+                    ReplyOrConfirmation replyOrConfirmation = new ServerErrorReply((byte) 0x0, cBusOptions, requestContext);
+                    CBusMessageToClient cBusMessageToClient = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
+                    ctx.writeAndFlush(cBusMessageToClient);
+                    return;
                 }
+                CALData calData = command.getCalData();
+                // TODO: handle other Datatypes
                 if (calData instanceof CALDataIdentify) {
-                    handleCalDataIdentify(ctx, (CALDataIdentify) calData, requestDirectCommandAccess.getAlpha());
+                    handleCalDataIdentify(ctx, (CALDataIdentify) calData, requestCommand.getAlpha());
                 }
                 return;
             }
-            if (request instanceof RequestCommand) {
-                RequestCommand requestCommand = (RequestCommand) request;
-                LOGGER.info("Handling RequestCommand\n{}", requestCommand);
-                CBusCommand cbusCommand = requestCommand.getCbusCommand();
-                LOGGER.info("Handling CBusCommand\n{}", cbusCommand);
-                if (cbusCommand instanceof CBusCommandPointToPoint) {
-                    CBusCommandPointToPoint cBusCommandPointToPoint = (CBusCommandPointToPoint) cbusCommand;
-                    CBusPointToPointCommand command = cBusCommandPointToPoint.getCommand();
-                    UnitAddress unitAddress = null;
-                    if (command instanceof CBusPointToPointCommandIndirect) {
-                        CBusPointToPointCommandIndirect cBusPointToPointCommandIndirect = (CBusPointToPointCommandIndirect) command;
-                        // TODO: handle bridgeAddress
-                        // TODO: handle networkRoute
-                        unitAddress = cBusPointToPointCommandIndirect.getUnitAddress();
-                    }
-                    if (command instanceof CBusPointToPointCommandDirect) {
-                        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());
-                    if (!knownUnit) {
-                        LOGGER.warn("{} not a known unit", unitAddress);
-                        ReplyOrConfirmation replyOrConfirmation = new ServerErrorReply((byte) 0x0, cBusOptions, requestContext);
-                        CBusMessageToClient cBusMessageToClient = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
-                        ctx.writeAndFlush(cBusMessageToClient);
-                        return;
-                    }
-                    CALData calData = command.getCalData();
-                    // TODO: handle other Datatypes
-                    if (calData instanceof CALDataIdentify) {
-                        handleCalDataIdentify(ctx, (CALDataIdentify) calData, requestCommand.getAlpha());
-                    }
-                    return;
-                }
-                if (cbusCommand instanceof CBusCommandPointToMultiPoint) {
-                    CBusCommandPointToMultiPoint cBusCommandPointToMultiPoint = (CBusCommandPointToMultiPoint) cbusCommand;
-                    CBusPointToMultiPointCommand command = cBusCommandPointToMultiPoint.getCommand();
-                    if (command instanceof CBusPointToMultiPointCommandStatus) {
-                        CBusPointToMultiPointCommandStatus cBusPointToMultiPointCommandStatus = (CBusPointToMultiPointCommandStatus) command;
-                        StatusRequest statusRequest = cBusPointToMultiPointCommandStatus.getStatusRequest();
-                        if (statusRequest instanceof StatusRequestBinaryState) {
-                            StatusRequestBinaryState statusRequestBinaryState = (StatusRequestBinaryState) statusRequest;
-                            CALReply calReply;
-                            if (exstat) {
-                                // TODO: map actuall values from simulator
-                                byte blockStart = 0x0;
-                                List<StatusByte> statusBytes = List.of(new StatusByte(GAVState.ON, GAVState.ERROR, GAVState.OFF, GAVState.DOES_NOT_EXIST));
-                                CALData calData = new CALDataStatusExtended(CALCommandTypeContainer.CALCommandReply_4Bytes, null, StatusCoding.BINARY_BY_THIS_SERIAL_INTERFACE, statusRequestBinaryState.getApplication(), blockStart, statusBytes, null, requestContext);
-                                calReply = new CALReplyLong((byte) 0x0, calData, (byte) 0x0, new UnitAddress((byte) 0x0), null, null, (byte) 0x0, null, cBusOptions, requestContext);
-                            } else {
-                                // TODO: map actuall values from simulator
-                                byte blockStart = 0x0;
-                                List<StatusByte> statusBytes = List.of(new StatusByte(GAVState.ON, GAVState.ERROR, GAVState.OFF, GAVState.DOES_NOT_EXIST));
-                                CALData calData = new CALDataStatus(CALCommandTypeContainer.CALCommandReply_3Bytes, null, statusRequestBinaryState.getApplication(), blockStart, statusBytes, requestContext);
-                                calReply = new CALReplyShort((byte) 0x0, calData, cBusOptions, requestContext);
-                            }
-                            EncodedReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
-                            ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0xC0, encodedReply, null, cBusOptions, requestContext);
-                            ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0xFF, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
-                            Alpha alpha = requestCommand.getAlpha();
-                            if (alpha != null) {
-                                Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.CONFIRMATION_SUCCESSFUL);
-                                replyOrConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, replyOrConfirmation, cBusOptions, requestContext);
-                            }
-                            CBusMessage response = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
-                            LOGGER.info("Send binary status response\n{}", response);
-                            ctx.writeAndFlush(response);
-                            return;
-                        }
-                        if (statusRequest instanceof StatusRequestBinaryStateDeprecated) {
-                            StatusRequestBinaryStateDeprecated statusRequestBinaryStateDeprecated = (StatusRequestBinaryStateDeprecated) statusRequest;
-                            LOGGER.info("Handling StatusRequestBinaryStateDeprecated\n{}", statusRequestBinaryStateDeprecated);
-                            // TODO: handle this
-                            return;
-                        }
-                        if (statusRequest instanceof StatusRequestLevel) {
-                            StatusRequestLevel statusRequestLevel = (StatusRequestLevel) statusRequest;
-                            StatusCoding coding = StatusCoding.LEVEL_BY_THIS_SERIAL_INTERFACE;
+            if (cbusCommand instanceof CBusCommandPointToMultiPoint) {
+                CBusCommandPointToMultiPoint cBusCommandPointToMultiPoint = (CBusCommandPointToMultiPoint) cbusCommand;
+                CBusPointToMultiPointCommand command = cBusCommandPointToMultiPoint.getCommand();
+                if (command instanceof CBusPointToMultiPointCommandStatus) {
+                    CBusPointToMultiPointCommandStatus cBusPointToMultiPointCommandStatus = (CBusPointToMultiPointCommandStatus) command;
+                    StatusRequest statusRequest = cBusPointToMultiPointCommandStatus.getStatusRequest();
+                    if (statusRequest instanceof StatusRequestBinaryState) {
+                        StatusRequestBinaryState statusRequestBinaryState = (StatusRequestBinaryState) statusRequest;
+                        CALReply calReply;
+                        if (exstat) {
+                            // TODO: map actuall values from simulator
+                            byte blockStart = 0x0;
+                            List<StatusByte> statusBytes = List.of(new StatusByte(GAVState.ON, GAVState.ERROR, GAVState.OFF, GAVState.DOES_NOT_EXIST));
+                            CALData calData = new CALDataStatusExtended(CALCommandTypeContainer.CALCommandReply_4Bytes, null, StatusCoding.BINARY_BY_THIS_SERIAL_INTERFACE, statusRequestBinaryState.getApplication(), blockStart, statusBytes, null, requestContext);
+                            calReply = new CALReplyLong((byte) 0x0, calData, (byte) 0x0, new UnitAddress((byte) 0x0), null, null, (byte) 0x0, null, cBusOptions, requestContext);
+                        } else {
                             // TODO: map actuall values from simulator
-                            byte blockStart = statusRequestLevel.getStartingGroupAddressLabel();
-                            List<LevelInformation> levelInformations = List.of(new LevelInformationNormal(0x5555, LevelInformationNibblePair.Value_F, LevelInformationNibblePair.Value_F));
-                            CALData calData = new CALDataStatusExtended(CALCommandTypeContainer.CALCommandReply_4Bytes, null, coding, statusRequestLevel.getApplication(), blockStart, null, levelInformations, requestContext);
-                            CALReply calReply = new CALReplyLong((byte) 0x0, calData, (byte) 0x0, new UnitAddress((byte) 0x0), null, null, (byte) 0x0, null, cBusOptions, requestContext);
-                            EncodedReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
-                            ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0xC0, encodedReply, null, cBusOptions, requestContext);
-                            ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0xFF, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
-                            Alpha alpha = requestCommand.getAlpha();
-                            if (alpha != null) {
-                                Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.CONFIRMATION_SUCCESSFUL);
-                                replyOrConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, replyOrConfirmation, cBusOptions, requestContext);
-                            }
-                            CBusMessage response = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
-                            LOGGER.info("Send level status response\n{}", response);
-                            ctx.writeAndFlush(response);
-                            return;
+                            byte blockStart = 0x0;
+                            List<StatusByte> statusBytes = List.of(new StatusByte(GAVState.ON, GAVState.ERROR, GAVState.OFF, GAVState.DOES_NOT_EXIST));
+                            CALData calData = new CALDataStatus(CALCommandTypeContainer.CALCommandReply_3Bytes, null, statusRequestBinaryState.getApplication(), blockStart, statusBytes, requestContext);
+                            calReply = new CALReplyShort((byte) 0x0, calData, cBusOptions, requestContext);
+                        }
+                        EncodedReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
+                        ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0xC0, encodedReply, null, cBusOptions, requestContext);
+                        ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0xFF, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
+                        Alpha alpha = requestCommand.getAlpha();
+                        if (alpha != null) {
+                            Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.CONFIRMATION_SUCCESSFUL);
+                            replyOrConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, replyOrConfirmation, cBusOptions, requestContext);
                         }
+                        CBusMessage response = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
+                        LOGGER.info("Send binary status response\n{}", response);
+                        ctx.writeAndFlush(response);
+                        return;
+                    }
+                    if (statusRequest instanceof StatusRequestBinaryStateDeprecated) {
+                        StatusRequestBinaryStateDeprecated statusRequestBinaryStateDeprecated = (StatusRequestBinaryStateDeprecated) statusRequest;
+                        LOGGER.info("Handling StatusRequestBinaryStateDeprecated\n{}", statusRequestBinaryStateDeprecated);
                         // TODO: handle this
                         return;
                     }
-                    if (command instanceof CBusPointToMultiPointCommandNormal) {
-                        CBusPointToMultiPointCommandNormal cBusPointToMultiPointCommandNormal = (CBusPointToMultiPointCommandNormal) command;
-                        LOGGER.info("Handling CBusPointToMultiPointCommandNormal\n{}", cBusPointToMultiPointCommandNormal);
+                    if (statusRequest instanceof StatusRequestLevel) {
+                        StatusRequestLevel statusRequestLevel = (StatusRequestLevel) statusRequest;
+                        StatusCoding coding = StatusCoding.LEVEL_BY_THIS_SERIAL_INTERFACE;
+                        // TODO: map actuall values from simulator
+                        byte blockStart = statusRequestLevel.getStartingGroupAddressLabel();
+                        List<LevelInformation> levelInformations = List.of(new LevelInformationNormal(0x5555, LevelInformationNibblePair.Value_F, LevelInformationNibblePair.Value_F));
+                        CALData calData = new CALDataStatusExtended(CALCommandTypeContainer.CALCommandReply_4Bytes, null, coding, statusRequestLevel.getApplication(), blockStart, null, levelInformations, requestContext);
+                        CALReply calReply = new CALReplyLong((byte) 0x0, calData, (byte) 0x0, new UnitAddress((byte) 0x0), null, null, (byte) 0x0, null, cBusOptions, requestContext);
+                        EncodedReply encodedReply = new EncodedReplyCALReply((byte) 0x0, calReply, cBusOptions, requestContext);
+                        ReplyEncodedReply replyEncodedReply = new ReplyEncodedReply((byte) 0xC0, encodedReply, null, cBusOptions, requestContext);
+                        ReplyOrConfirmation replyOrConfirmation = new ReplyOrConfirmationReply((byte) 0xFF, replyEncodedReply, new ResponseTermination(), cBusOptions, requestContext);
+                        Alpha alpha = requestCommand.getAlpha();
+                        if (alpha != null) {
+                            Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.CONFIRMATION_SUCCESSFUL);
+                            replyOrConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, replyOrConfirmation, cBusOptions, requestContext);
+                        }
+                        CBusMessage response = new CBusMessageToClient(replyOrConfirmation, requestContext, cBusOptions);
+                        LOGGER.info("Send level status response\n{}", response);
+                        ctx.writeAndFlush(response);
                         return;
                     }
                     // TODO: handle this
                     return;
                 }
-                if (cbusCommand instanceof CBusCommandPointToPointToMultiPoint) {
-                    CBusCommandPointToPointToMultiPoint cBusCommandPointToPointToMultiPoint = (CBusCommandPointToPointToMultiPoint) cbusCommand;
-                    LOGGER.info("Handling CBusCommandPointToPointToMultiPoint\n{}", cBusCommandPointToPointToMultiPoint);
-                    // TODO: handle this
+                if (command instanceof CBusPointToMultiPointCommandNormal) {
+                    CBusPointToMultiPointCommandNormal cBusPointToMultiPointCommandNormal = (CBusPointToMultiPointCommandNormal) command;
+                    LOGGER.info("Handling CBusPointToMultiPointCommandNormal\n{}", cBusPointToMultiPointCommandNormal);
                     return;
                 }
-                if (cbusCommand instanceof CBusCommandDeviceManagement) {
-                    CBusCommandDeviceManagement cBusCommandDeviceManagement = (CBusCommandDeviceManagement) cbusCommand;
-                    LOGGER.info("Handling CBusCommandDeviceManagement\n{}", cBusCommandDeviceManagement);
-                    // TODO: handle this
-                    return;
-                }
-
-                Alpha alpha = requestCommand.getAlpha();
-                if (alpha != null) {
-                    Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.NOT_TRANSMITTED_CORRUPTION);
-                    ReplyOrConfirmationConfirmation replyOrConfirmationConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, null, cBusOptions, requestContext);
-                    CBusMessage response = new CBusMessageToClient(replyOrConfirmationConfirmation, requestContext, cBusOptions);
-                    LOGGER.info("Send response\n{}", response);
-                    ctx.writeAndFlush(response);
-                }
-                return;
-            }
-            if (request instanceof RequestObsolete) {
-                RequestObsolete requestObsolete = (RequestObsolete) request;
-                LOGGER.info("Handling RequestObsolete\n{}", requestObsolete);
                 // TODO: handle this
                 return;
             }
-            if (request instanceof RequestReset) {
-                RequestReset requestReset = (RequestReset) request;
-                LOGGER.info("Handling RequestReset\n{}", requestReset);
-                connect = false;
-                smart = false;
-                idmon = false;
-                exstat = false;
-                monitor = false;
-                monall = false;
-                pun = false;
-                pcn = false;
-                srchk = false;
-                stopSALMonitor();
+            if (cbusCommand instanceof CBusCommandPointToPointToMultiPoint) {
+                CBusCommandPointToPointToMultiPoint cBusCommandPointToPointToMultiPoint = (CBusCommandPointToPointToMultiPoint) cbusCommand;
+                LOGGER.info("Handling CBusCommandPointToPointToMultiPoint\n{}", cBusCommandPointToPointToMultiPoint);
+                // TODO: handle this
                 return;
             }
-            if (request instanceof RequestSmartConnectShortcut) {
-                RequestSmartConnectShortcut requestSmartConnectShortcut = (RequestSmartConnectShortcut) request;
-                LOGGER.info("Handling RequestSmartConnectShortcut\n{}", requestSmartConnectShortcut);
+            if (cbusCommand instanceof CBusCommandDeviceManagement) {
+                CBusCommandDeviceManagement cBusCommandDeviceManagement = (CBusCommandDeviceManagement) cbusCommand;
+                LOGGER.info("Handling CBusCommandDeviceManagement\n{}", cBusCommandDeviceManagement);
                 // TODO: handle this
                 return;
             }
-        } finally {
-            ctx.flush();
-            writeLock.unlock();
+
+            Alpha alpha = requestCommand.getAlpha();
+            if (alpha != null) {
+                Confirmation confirmation = new Confirmation(alpha, null, ConfirmationType.NOT_TRANSMITTED_CORRUPTION);
+                ReplyOrConfirmationConfirmation replyOrConfirmationConfirmation = new ReplyOrConfirmationConfirmation(alpha.getCharacter(), confirmation, null, cBusOptions, requestContext);
+                CBusMessage response = new CBusMessageToClient(replyOrConfirmationConfirmation, requestContext, cBusOptions);
+                LOGGER.info("Send response\n{}", response);
+                ctx.writeAndFlush(response);
+            }
+            return;
+        }
+        if (request instanceof RequestObsolete) {
+            RequestObsolete requestObsolete = (RequestObsolete) request;
+            LOGGER.info("Handling RequestObsolete\n{}", requestObsolete);
+            // TODO: handle this
+            return;
+        }
+        if (request instanceof RequestReset) {
+            RequestReset requestReset = (RequestReset) request;
+            LOGGER.info("Handling RequestReset\n{}", requestReset);
+            connect = false;
+            smart = false;
+            idmon = false;
+            exstat = false;
+            monitor = false;
+            monall = false;
+            pun = false;
+            pcn = false;
+            srchk = false;
+            stopSALMonitor();
+            return;
+        }
+        if (request instanceof RequestSmartConnectShortcut) {
+            RequestSmartConnectShortcut requestSmartConnectShortcut = (RequestSmartConnectShortcut) request;
+            LOGGER.info("Handling RequestSmartConnectShortcut\n{}", requestSmartConnectShortcut);
+            // TODO: handle this
+            return;
         }
     }
 
@@ -392,7 +398,7 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
         switch (calDataIdentify.getAttribute()) {
             case Manufacturer:
                 numBytes = 0x08;
-                identifyReplyCommand = new IdentifyReplyCommandManufacturer("Apache", numBytes);
+                identifyReplyCommand = new IdentifyReplyCommandManufacturer("Apache  ", numBytes);
                 break;
             case Type:
                 numBytes = 0x08;
@@ -506,7 +512,7 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                 return;
             }
             try {
-                writeLock.lock();
+                outputLock.lock();
                 MonitoredSAL monitoredSAL;
                 if (cBusOptions.getExstat()) {
                     LightingData lightingData;
@@ -544,7 +550,7 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                 LOGGER.info("[SAL Monitor] Sending out\n{}\n{}", message, encodedReply);
                 ctx.writeAndFlush(message);
             } finally {
-                writeLock.unlock();
+                outputLock.unlock();
             }
         }, 5, 5, TimeUnit.SECONDS);
     }
@@ -566,9 +572,9 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
         LOGGER.info("Starting MMI monitor");
         mmiMonitorFuture = ctx.executor().scheduleAtFixedRate(() -> {
             // TODO: for whatever reason those are not send with a crc
-            cBusOptions = new CBusOptions(connect, smart, idmon, exstat, monitor, monall, pun, pcn, false);
+            CBusOptions cBusOptions = new CBusOptions(connect, smart, idmon, exstat, monitor, monall, pun, pcn, false);
             try {
-                writeLock.lock();
+                outputLock.lock();
                 CALReply calReply;
                 if (cBusOptions.getExstat()) {
                     List<StatusByte> statusBytes = new LinkedList<>();
@@ -592,7 +598,7 @@ public class CBusServerAdapter extends ChannelInboundHandlerAdapter {
                 LOGGER.info("[MMI Monitor] Sending out\n{}\n{}", message, encodedReply);
                 ctx.writeAndFlush(message);
             } finally {
-                writeLock.unlock();
+                outputLock.unlock();
             }
         }, 5, 5, TimeUnit.SECONDS);
     }