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/09 14:28:58 UTC

[plc4x] branch develop updated (758e078e1 -> decf45ef9)

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 758e078e1 fix(plc4go/cbus): fixed vet error
     new e8ab0a7ed refactor(plc4xbrowser): put ui parts into a ui package
     new ff7d444e1 feat(plc4xbrowser): increase display area
     new decf45ef9 feat(plc4xpcapanalyzer): added frontend for interactive work with the cli

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/tools/plc4xbrowser/common.go                |  31 --
 plc4go/tools/plc4xbrowser/main.go                  |  62 +--
 plc4go/tools/plc4xbrowser/{ => ui}/actions.go      |   4 +-
 plc4go/tools/plc4xbrowser/{ => ui}/commands.go     |   2 +-
 .../tools/plc4xbrowser/{main.go => ui/common.go}   |  28 +-
 plc4go/tools/plc4xbrowser/{ => ui}/config.go       |   4 +-
 plc4go/tools/plc4xbrowser/{ => ui}/ui.go           |   6 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go       |   4 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go         |   4 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/extract.go      |   6 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/root.go         |   7 +-
 .../plc4xpcapanalyzer/cmd/{bacnet.go => ui.go}     |  45 +--
 .../internal/analyzer/analyzer.go                  |  36 +-
 .../internal/bacnetanalyzer/analyzer.go            |   5 +-
 .../internal/cbusanalyzer/analyzer.go              |   5 +-
 .../internal/extractor/extractor.go                |  20 +-
 .../internal/pcaphandler/pcaphandler.go            |  42 +-
 .../ui}/actions.go                                 |  42 +-
 .../ui}/commands.go                                | 431 +++++++--------------
 .../main.go => plc4xpcapanalyzer/ui/common.go}     |  63 ++-
 .../ui}/config.go                                  |  31 +-
 .../{plc4xbrowser => plc4xpcapanalyzer/ui}/ui.go   |  54 ++-
 22 files changed, 393 insertions(+), 539 deletions(-)
 delete mode 100644 plc4go/tools/plc4xbrowser/common.go
 copy plc4go/tools/plc4xbrowser/{ => ui}/actions.go (98%)
 copy plc4go/tools/plc4xbrowser/{ => ui}/commands.go (99%)
 copy plc4go/tools/plc4xbrowser/{main.go => ui/common.go} (90%)
 copy plc4go/tools/plc4xbrowser/{ => ui}/config.go (99%)
 copy plc4go/tools/plc4xbrowser/{ => ui}/ui.go (99%)
 copy plc4go/tools/plc4xpcapanalyzer/cmd/{bacnet.go => ui.go} (55%)
 rename plc4go/tools/{plc4xbrowser => plc4xpcapanalyzer/ui}/actions.go (76%)
 rename plc4go/tools/{plc4xbrowser => plc4xpcapanalyzer/ui}/commands.go (51%)
 copy plc4go/tools/{plc4xbrowser/main.go => plc4xpcapanalyzer/ui/common.go} (71%)
 rename plc4go/tools/{plc4xbrowser => plc4xpcapanalyzer/ui}/config.go (82%)
 rename plc4go/tools/{plc4xbrowser => plc4xpcapanalyzer/ui}/ui.go (82%)


[plc4x] 01/03: refactor(plc4xbrowser): put ui parts into a ui package

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 e8ab0a7ed589db8cab7bc85f5a00fd3f0d6a453d
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 9 10:55:08 2022 +0200

    refactor(plc4xbrowser): put ui parts into a ui package
---
 plc4go/tools/plc4xbrowser/common.go                | 31 -----------
 plc4go/tools/plc4xbrowser/main.go                  | 62 ++--------------------
 plc4go/tools/plc4xbrowser/{ => ui}/actions.go      |  4 +-
 plc4go/tools/plc4xbrowser/{ => ui}/commands.go     |  2 +-
 .../tools/plc4xbrowser/{main.go => ui/common.go}   | 28 +++++-----
 plc4go/tools/plc4xbrowser/{ => ui}/config.go       |  4 +-
 plc4go/tools/plc4xbrowser/{ => ui}/ui.go           |  4 +-
 7 files changed, 24 insertions(+), 111 deletions(-)

diff --git a/plc4go/tools/plc4xbrowser/common.go b/plc4go/tools/plc4xbrowser/common.go
deleted file mode 100644
index b42f76a1b..000000000
--- a/plc4go/tools/plc4xbrowser/common.go
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   https://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package main
-
-import (
-	"github.com/rs/zerolog"
-	"strings"
-)
-
-const protocols = "ads,bacnetip,c-bus,s7"
-
-var protocolList = strings.Split(protocols, ",")
-
-var plc4xBrowserLog = zerolog.Nop()
diff --git a/plc4go/tools/plc4xbrowser/main.go b/plc4go/tools/plc4xbrowser/main.go
index 1f5ebcd1c..0427d9c39 100644
--- a/plc4go/tools/plc4xbrowser/main.go
+++ b/plc4go/tools/plc4xbrowser/main.go
@@ -20,68 +20,16 @@
 package main
 
 import (
-	"io"
-	"sync"
-	"time"
-
-	plc4go "github.com/apache/plc4x/plc4go/pkg/api"
-	plc4goModel "github.com/apache/plc4x/plc4go/pkg/api/model"
-)
-
-var driverManager plc4go.PlcDriverManager
-var driverAdded func(string)
-var connections map[string]plc4go.PlcConnection
-var connectionsChanged func()
-
-var messageReceived func(messageNumber int, receiveTime time.Time, message plc4goModel.PlcMessage)
-var numberOfMessagesReceived int
-var messageOutput io.Writer
-var messageOutputClear func()
-
-var consoleOutput io.Writer
-var consoleOutputClear func()
-
-var commandsExecuted int
-var commandOutput io.Writer
-var commandOutputClear func()
-
-type inputMode int
-
-const (
-	normalMode inputMode = iota
-	readEditMode
-	writeEditMode
-	subscribeEditMode
+	"github.com/apache/plc4x/plc4go/tools/plc4xbrowser/ui"
 )
 
-func init() {
-	hasShutdown = false
-	connections = make(map[string]plc4go.PlcConnection)
-}
-
-var shutdownMutex sync.Mutex
-var hasShutdown bool
-
-func shutdown() {
-	shutdownMutex.Lock()
-	defer shutdownMutex.Unlock()
-	if hasShutdown {
-		return
-	}
-	for _, connection := range connections {
-		connection.Close()
-	}
-	saveConfig()
-	hasShutdown = true
-}
-
 func main() {
-	loadConfig()
-	application := setupApplication()
-	initSubsystem()
+	ui.LoadConfig()
+	application := ui.SetupApplication()
+	ui.InitSubsystem()
 
 	if err := application.Run(); err != nil {
 		panic(err)
 	}
-	shutdown()
+	ui.Shutdown()
 }
diff --git a/plc4go/tools/plc4xbrowser/actions.go b/plc4go/tools/plc4xbrowser/ui/actions.go
similarity index 98%
rename from plc4go/tools/plc4xbrowser/actions.go
rename to plc4go/tools/plc4xbrowser/ui/actions.go
index 6f9f581bb..25fce0a5f 100644
--- a/plc4go/tools/plc4xbrowser/actions.go
+++ b/plc4go/tools/plc4xbrowser/ui/actions.go
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package main
+package ui
 
 import (
 	"fmt"
@@ -33,7 +33,7 @@ import (
 	"github.com/sruehl/tview"
 )
 
-func initSubsystem() {
+func InitSubsystem() {
 	logLevel := zerolog.InfoLevel
 	if configuredLevel := config.LogLevel; configuredLevel != "" {
 		if parsedLevel, err := zerolog.ParseLevel(configuredLevel); err != nil {
diff --git a/plc4go/tools/plc4xbrowser/commands.go b/plc4go/tools/plc4xbrowser/ui/commands.go
similarity index 99%
rename from plc4go/tools/plc4xbrowser/commands.go
rename to plc4go/tools/plc4xbrowser/ui/commands.go
index 44fbe8829..26fb8c191 100644
--- a/plc4go/tools/plc4xbrowser/commands.go
+++ b/plc4go/tools/plc4xbrowser/ui/commands.go
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package main
+package ui
 
 import (
 	"fmt"
diff --git a/plc4go/tools/plc4xbrowser/main.go b/plc4go/tools/plc4xbrowser/ui/common.go
similarity index 90%
copy from plc4go/tools/plc4xbrowser/main.go
copy to plc4go/tools/plc4xbrowser/ui/common.go
index 1f5ebcd1c..84e31683b 100644
--- a/plc4go/tools/plc4xbrowser/main.go
+++ b/plc4go/tools/plc4xbrowser/ui/common.go
@@ -17,17 +17,24 @@
  * under the License.
  */
 
-package main
+package ui
 
 import (
+	plc4go "github.com/apache/plc4x/plc4go/pkg/api"
+	plc4goModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/rs/zerolog"
 	"io"
+	"strings"
 	"sync"
 	"time"
-
-	plc4go "github.com/apache/plc4x/plc4go/pkg/api"
-	plc4goModel "github.com/apache/plc4x/plc4go/pkg/api/model"
 )
 
+const protocols = "ads,bacnetip,c-bus,s7"
+
+var protocolList = strings.Split(protocols, ",")
+
+var plc4xBrowserLog = zerolog.Nop()
+
 var driverManager plc4go.PlcDriverManager
 var driverAdded func(string)
 var connections map[string]plc4go.PlcConnection
@@ -62,7 +69,7 @@ func init() {
 var shutdownMutex sync.Mutex
 var hasShutdown bool
 
-func shutdown() {
+func Shutdown() {
 	shutdownMutex.Lock()
 	defer shutdownMutex.Unlock()
 	if hasShutdown {
@@ -74,14 +81,3 @@ func shutdown() {
 	saveConfig()
 	hasShutdown = true
 }
-
-func main() {
-	loadConfig()
-	application := setupApplication()
-	initSubsystem()
-
-	if err := application.Run(); err != nil {
-		panic(err)
-	}
-	shutdown()
-}
diff --git a/plc4go/tools/plc4xbrowser/config.go b/plc4go/tools/plc4xbrowser/ui/config.go
similarity index 99%
rename from plc4go/tools/plc4xbrowser/config.go
rename to plc4go/tools/plc4xbrowser/ui/config.go
index 24da0242c..d10f990ec 100644
--- a/plc4go/tools/plc4xbrowser/config.go
+++ b/plc4go/tools/plc4xbrowser/ui/config.go
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package main
+package ui
 
 import (
 	"github.com/pkg/errors"
@@ -60,7 +60,7 @@ func init() {
 	configFile = path.Join(plc4xBrowserConfigDir, "config.yml")
 }
 
-func loadConfig() {
+func LoadConfig() {
 	f, err := os.Open(configFile)
 	if err != nil {
 		log.Info().Err(err).Msg("No config file found")
diff --git a/plc4go/tools/plc4xbrowser/ui.go b/plc4go/tools/plc4xbrowser/ui/ui.go
similarity index 99%
rename from plc4go/tools/plc4xbrowser/ui.go
rename to plc4go/tools/plc4xbrowser/ui/ui.go
index d4a5f4619..fda7476c0 100644
--- a/plc4go/tools/plc4xbrowser/ui.go
+++ b/plc4go/tools/plc4xbrowser/ui/ui.go
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-package main
+package ui
 
 import (
 	"fmt"
@@ -30,7 +30,7 @@ import (
 	"time"
 )
 
-func setupApplication() *tview.Application {
+func SetupApplication() *tview.Application {
 	application := tview.NewApplication()
 
 	newPrimitive := func(text string) tview.Primitive {


[plc4x] 03/03: feat(plc4xpcapanalyzer): added frontend for interactive work with the cli

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 decf45ef9ffaf6276db4f1abfbe42c841fdd2456
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 9 16:28:51 2022 +0200

    feat(plc4xpcapanalyzer): added frontend for interactive work with the cli
---
 plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go       |   4 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go         |   4 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/extract.go      |   6 +-
 plc4go/tools/plc4xpcapanalyzer/cmd/root.go         |   7 +-
 .../plc4xpcapanalyzer/cmd/{bacnet.go => ui.go}     |  45 +-
 .../internal/analyzer/analyzer.go                  |  36 +-
 .../internal/bacnetanalyzer/analyzer.go            |   5 +-
 .../internal/cbusanalyzer/analyzer.go              |   5 +-
 .../internal/extractor/extractor.go                |  20 +-
 .../internal/pcaphandler/pcaphandler.go            |  42 +-
 plc4go/tools/plc4xpcapanalyzer/ui/actions.go       | 130 +++++
 plc4go/tools/plc4xpcapanalyzer/ui/commands.go      | 613 +++++++++++++++++++++
 plc4go/tools/plc4xpcapanalyzer/ui/common.go        |  82 +++
 plc4go/tools/plc4xpcapanalyzer/ui/config.go        | 179 ++++++
 plc4go/tools/plc4xpcapanalyzer/ui/ui.go            | 310 +++++++++++
 15 files changed, 1420 insertions(+), 68 deletions(-)

diff --git a/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go b/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go
index 31dbc33f0..143051fc5 100644
--- a/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go
+++ b/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go
@@ -56,7 +56,9 @@ TODO: document me
 		} else {
 			log.Info().Msg("All filtering disabled")
 		}
-		analyzer.Analyze(pcapFile, "bacnet")
+		if err := analyzer.Analyze(pcapFile, "bacnet"); err != nil {
+			panic(err)
+		}
 		println("Done")
 	},
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go b/plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go
index 8ae7f0f8b..0b997047c 100644
--- a/plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go
+++ b/plc4go/tools/plc4xpcapanalyzer/cmd/cbus.go
@@ -56,7 +56,9 @@ TODO: document me
 		} else {
 			log.Info().Msg("All filtering disabled")
 		}
-		analyzer.Analyze(pcapFile, "c-bus")
+		if err := analyzer.Analyze(pcapFile, "c-bus"); err != nil {
+			panic(err)
+		}
 		println("Done")
 	},
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/cmd/extract.go b/plc4go/tools/plc4xpcapanalyzer/cmd/extract.go
index 55b21ede1..441d1a9aa 100644
--- a/plc4go/tools/plc4xpcapanalyzer/cmd/extract.go
+++ b/plc4go/tools/plc4xpcapanalyzer/cmd/extract.go
@@ -32,7 +32,7 @@ import (
 var extractCmd = &cobra.Command{
 	Use:   "extract [protocolType] [pcapfile]",
 	Short: "extract a pcap file using a driver supplied driver",
-	Long: `Analyzes a pcap file using a driver
+	Long: `Extract a pcap file using a driver
 TODO: document me
 `,
 	Args: func(cmd *cobra.Command, args []string) error {
@@ -51,7 +51,9 @@ TODO: document me
 	Run: func(cmd *cobra.Command, args []string) {
 		protocolType := args[0]
 		pcapFile := args[1]
-		extractor.Extract(pcapFile, protocolType)
+		if err := extractor.Extract(pcapFile, protocolType); err != nil {
+			panic(err)
+		}
 		println("Done")
 	},
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/cmd/root.go b/plc4go/tools/plc4xpcapanalyzer/cmd/root.go
index b367fa622..205a6e6f5 100644
--- a/plc4go/tools/plc4xpcapanalyzer/cmd/root.go
+++ b/plc4go/tools/plc4xpcapanalyzer/cmd/root.go
@@ -68,14 +68,13 @@ func initConfig() {
 		// Use config file from the flag.
 		viper.SetConfigFile(config.RootConfigInstance.CfgFile)
 	} else {
-		// Find home directory.
-		home, err := os.UserHomeDir()
+		// Find user config directory.
+		home, err := os.UserConfigDir()
 		cobra.CheckErr(err)
 
-		// Search config in home directory with name ".plc4xpcapanalyzer" (without extension).
 		viper.AddConfigPath(home)
 		viper.SetConfigType("yaml")
-		viper.SetConfigName(".plc4xpcapanalyzer")
+		viper.SetConfigName("plc4xpcapanalyzer-viper")
 	}
 
 	viper.AutomaticEnv() // read in environment variables that match
diff --git a/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go b/plc4go/tools/plc4xpcapanalyzer/cmd/ui.go
similarity index 55%
copy from plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go
copy to plc4go/tools/plc4xpcapanalyzer/cmd/ui.go
index 31dbc33f0..0a259a15d 100644
--- a/plc4go/tools/plc4xpcapanalyzer/cmd/bacnet.go
+++ b/plc4go/tools/plc4xpcapanalyzer/cmd/ui.go
@@ -20,8 +20,7 @@
 package cmd
 
 import (
-	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/config"
-	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/analyzer"
+	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/ui"
 	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
 	"os"
@@ -29,16 +28,16 @@ import (
 	"github.com/spf13/cobra"
 )
 
-// bacnetCmd represents the bacnet command
-var bacnetCmd = &cobra.Command{
-	Use:   "bacnet [pcapfile]",
-	Short: "analyzes a pcap file using a bacnet driver",
+// uiCmd represents the ui command
+var uiCmd = &cobra.Command{
+	Use:   "ui [pcapfile]",
+	Short: "Start the ui with optional pcapfile",
 	Long: `Analyzes a pcap file using a bacnet driver
 TODO: document me
 `,
 	Args: func(cmd *cobra.Command, args []string) error {
 		if len(args) < 1 {
-			return errors.New("requires exactly one arguments")
+			return nil
 		}
 		pcapFile := args[0]
 		if _, err := os.Stat(pcapFile); errors.Is(err, os.ErrNotExist) {
@@ -47,24 +46,26 @@ TODO: document me
 		return nil
 	},
 	Run: func(cmd *cobra.Command, args []string) {
-		pcapFile := args[0]
-		if !config.BacnetConfigInstance.NoFilter {
-			if config.BacnetConfigInstance.Filter == "" && config.BacnetConfigInstance.BacnetFilter != "" {
-				log.Debug().Str("filter", config.BacnetConfigInstance.Filter).Msg("Setting bacnet filter")
-				config.BacnetConfigInstance.Filter = config.BacnetConfigInstance.BacnetFilter
-			}
-		} else {
-			log.Info().Msg("All filtering disabled")
+		ui.LoadConfig()
+		application := ui.SetupApplication()
+		ui.InitSubsystem()
+		if len(args) > 0 {
+			pcapFile := args[0]
+			go func() {
+				err := ui.OpenFile(pcapFile)
+				if err != nil {
+					log.Error().Err(err).Msg("Error opening argument file")
+				}
+			}()
+		}
+
+		if err := application.Run(); err != nil {
+			panic(err)
 		}
-		analyzer.Analyze(pcapFile, "bacnet")
-		println("Done")
+		ui.Shutdown()
 	},
 }
 
 func init() {
-	analyzeCmd.AddCommand(bacnetCmd)
-
-	bacnetCmd.PersistentFlags().StringVarP(&config.BacnetConfigInstance.BacnetFilter, "default-bacnet-filter", "", "udp port 47808 and udp[4:2] > 29", "Defines the default filter when bacnet is selected")
-
-	addAnalyzeFlags(bacnetCmd)
+	rootCmd.AddCommand(uiCmd)
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/internal/analyzer/analyzer.go b/plc4go/tools/plc4xpcapanalyzer/internal/analyzer/analyzer.go
index 1e74430fe..22fa9302a 100644
--- a/plc4go/tools/plc4xpcapanalyzer/internal/analyzer/analyzer.go
+++ b/plc4go/tools/plc4xpcapanalyzer/internal/analyzer/analyzer.go
@@ -23,6 +23,7 @@ import (
 	"bytes"
 	"encoding/hex"
 	"fmt"
+	"github.com/apache/plc4x/plc4go/internal/spi"
 	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/config"
 	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/bacnetanalyzer"
 	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/cbusanalyzer"
@@ -31,16 +32,30 @@ import (
 	"github.com/google/gopacket"
 	"github.com/google/gopacket/layers"
 	"github.com/k0kubun/go-ansi"
+	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
 	"github.com/schollz/progressbar/v3"
+	"io"
 	"net"
+	"os"
 	"time"
 )
 
-func Analyze(pcapFile, protocolType string) {
+func Analyze(pcapFile, protocolType string) error {
+	return AnalyzeWithOutput(pcapFile, protocolType, os.Stdout, os.Stderr)
+}
+
+func AnalyzeWithOutput(pcapFile, protocolType string, stdout, stderr io.Writer) error {
+	return AnalyzeWithOutputAndCallback(pcapFile, protocolType, stdout, stderr, nil)
+}
+
+func AnalyzeWithOutputAndCallback(pcapFile, protocolType string, stdout, stderr io.Writer, messageCallback func(parsed spi.Message)) error {
 	log.Info().Msgf("Analyzing pcap file '%s' with protocolType '%s' and filter '%s' now", pcapFile, protocolType, config.AnalyzeConfigInstance.Filter)
 
-	handle, numberOfPackage, timestampToIndexMap := pcaphandler.GetIndexedPcapHandle(pcapFile, config.AnalyzeConfigInstance.Filter)
+	handle, numberOfPackage, timestampToIndexMap, err := pcaphandler.GetIndexedPcapHandle(pcapFile, config.AnalyzeConfigInstance.Filter)
+	if err != nil {
+		return errors.Wrap(err, "Error getting handle")
+	}
 	log.Info().Msgf("Starting to analyze %d packages", numberOfPackage)
 	defer handle.Close()
 	log.Debug().Interface("handle", handle).Int("numberOfPackage", numberOfPackage).Msg("got handle")
@@ -48,10 +63,10 @@ func Analyze(pcapFile, protocolType string) {
 	var mapPackets = func(in chan gopacket.Packet, packetInformationCreator func(packet gopacket.Packet) common.PacketInformation) chan gopacket.Packet {
 		return in
 	}
-	var packageParse func(common.PacketInformation, []byte) (interface{}, error)
-	var serializePackage func(interface{}) ([]byte, error)
-	var prettyPrint = func(item interface{}) {
-		fmt.Printf("%v\n", item)
+	var packageParse func(common.PacketInformation, []byte) (spi.Message, error)
+	var serializePackage func(spi.Message) ([]byte, error)
+	var prettyPrint = func(item spi.Message) {
+		_, _ = fmt.Fprintf(stdout, "%v\n", item)
 	}
 	var byteOutput = hex.Dump
 	switch protocolType {
@@ -134,6 +149,9 @@ func Analyze(pcapFile, protocolType string) {
 			}
 			continue
 		} else {
+			if messageCallback != nil {
+				messageCallback(parsed)
+			}
 			log.Info().Stringer("packetInformation", packetInformation).Msgf("No.[%d] Parsed", realPacketNumber)
 			if config.AnalyzeConfigInstance.Verbosity > 1 {
 				prettyPrint(parsed)
@@ -158,16 +176,14 @@ func Analyze(pcapFile, protocolType string) {
 				// TODO: write report to xml or something
 				log.Warn().Stringer("packetInformation", packetInformation).Msgf("No.[%d] Bytes don't match.\nOriginal:\n%sSerialized:\n%s", realPacketNumber, byteOutput(payload), byteOutput(serializedBytes))
 				if config.AnalyzeConfigInstance.Verbosity > 0 {
-					println("Original bytes")
-					println(hex.Dump(payload))
-					println("Serialized bytes")
-					println(hex.Dump(serializedBytes))
+					_, _ = fmt.Fprintf(stdout, "Original bytes\n%s\n%s\n", hex.Dump(payload), hex.Dump(serializedBytes))
 				}
 			}
 		}
 	}
 
 	log.Info().Msgf("Done evaluating %d of %d packages (%d failed to parse, %d failed to serialize and %d failed in byte comparison)", currentPackageNum, numberOfPackage, parseFails, serializeFails, compareFails)
+	return nil
 }
 
 func createPacketInformation(pcapFile string, packet gopacket.Packet, timestampToIndexMap map[time.Time]int) common.PacketInformation {
diff --git a/plc4go/tools/plc4xpcapanalyzer/internal/bacnetanalyzer/analyzer.go b/plc4go/tools/plc4xpcapanalyzer/internal/bacnetanalyzer/analyzer.go
index f614ae435..562ae9969 100644
--- a/plc4go/tools/plc4xpcapanalyzer/internal/bacnetanalyzer/analyzer.go
+++ b/plc4go/tools/plc4xpcapanalyzer/internal/bacnetanalyzer/analyzer.go
@@ -20,6 +20,7 @@
 package bacnetanalyzer
 
 import (
+	"github.com/apache/plc4x/plc4go/internal/spi"
 	"github.com/apache/plc4x/plc4go/internal/spi/utils"
 	"github.com/apache/plc4x/plc4go/protocols/bacnetip/readwrite/model"
 	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/common"
@@ -27,7 +28,7 @@ import (
 	"github.com/rs/zerolog/log"
 )
 
-func PackageParse(packetInformation common.PacketInformation, payload []byte) (interface{}, error) {
+func PackageParse(packetInformation common.PacketInformation, payload []byte) (spi.Message, error) {
 	log.Debug().Msgf("Parsing %s", packetInformation)
 	parse, err := model.BVLCParse(utils.NewReadBufferByteBased(payload))
 	if err != nil {
@@ -37,7 +38,7 @@ func PackageParse(packetInformation common.PacketInformation, payload []byte) (i
 	return parse, nil
 }
 
-func SerializePackage(bvlc interface{}) ([]byte, error) {
+func SerializePackage(bvlc spi.Message) ([]byte, error) {
 	if bvlc, ok := bvlc.(model.BVLC); !ok {
 		log.Fatal().Msgf("Unsupported type %T supplied", bvlc)
 		panic("unreachable statement")
diff --git a/plc4go/tools/plc4xpcapanalyzer/internal/cbusanalyzer/analyzer.go b/plc4go/tools/plc4xpcapanalyzer/internal/cbusanalyzer/analyzer.go
index 6997dc2b8..addabf8b3 100644
--- a/plc4go/tools/plc4xpcapanalyzer/internal/cbusanalyzer/analyzer.go
+++ b/plc4go/tools/plc4xpcapanalyzer/internal/cbusanalyzer/analyzer.go
@@ -22,6 +22,7 @@ package cbusanalyzer
 import (
 	"fmt"
 	"github.com/apache/plc4x/plc4go/internal/cbus"
+	"github.com/apache/plc4x/plc4go/internal/spi"
 	"github.com/apache/plc4x/plc4go/internal/spi/utils"
 	"github.com/apache/plc4x/plc4go/protocols/cbus/readwrite/model"
 	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/config"
@@ -58,7 +59,7 @@ func (a *Analyzer) Init() {
 	a.initialized = true
 }
 
-func (a *Analyzer) PackageParse(packetInformation common.PacketInformation, payload []byte) (interface{}, error) {
+func (a *Analyzer) PackageParse(packetInformation common.PacketInformation, payload []byte) (spi.Message, error) {
 	if !a.initialized {
 		log.Warn().Msg("Not initialized... doing that now")
 		a.Init()
@@ -214,7 +215,7 @@ func filterXOnXOff(payload []byte) []byte {
 	return payload[:n]
 }
 
-func (a *Analyzer) SerializePackage(message interface{}) ([]byte, error) {
+func (a *Analyzer) SerializePackage(message spi.Message) ([]byte, error) {
 	if message, ok := message.(model.CBusMessage); !ok {
 		log.Fatal().Msgf("Unsupported type %T supplied", message)
 		panic("unreachable statement")
diff --git a/plc4go/tools/plc4xpcapanalyzer/internal/extractor/extractor.go b/plc4go/tools/plc4xpcapanalyzer/internal/extractor/extractor.go
index 4495ea7cf..bfd201f95 100644
--- a/plc4go/tools/plc4xpcapanalyzer/internal/extractor/extractor.go
+++ b/plc4go/tools/plc4xpcapanalyzer/internal/extractor/extractor.go
@@ -27,21 +27,30 @@ import (
 	"github.com/fatih/color"
 	"github.com/google/gopacket/layers"
 	"github.com/k0kubun/go-ansi"
+	"github.com/pkg/errors"
 	"github.com/rs/zerolog/log"
 	"github.com/schollz/progressbar/v3"
+	"io"
 	"net"
 )
 
-func Extract(pcapFile, protocolType string) {
+func Extract(pcapFile, protocolType string) error {
+	return ExtractWithOutput(pcapFile, protocolType, ansi.NewAnsiStdout(), ansi.NewAnsiStderr())
+}
+
+func ExtractWithOutput(pcapFile, protocolType string, stdout, stderr io.Writer) error {
 	log.Info().Msgf("Analyzing pcap file '%s' with protocolType '%s' and filter '%s' now", pcapFile, protocolType, config.ExtractConfigInstance.Filter)
 
-	handle, numberOfPackage, timestampToIndexMap := pcaphandler.GetIndexedPcapHandle(pcapFile, config.ExtractConfigInstance.Filter)
+	handle, numberOfPackage, timestampToIndexMap, err := pcaphandler.GetIndexedPcapHandle(pcapFile, config.ExtractConfigInstance.Filter)
+	if err != nil {
+		return errors.Wrap(err, "Error getting handle")
+	}
 	log.Info().Msgf("Starting to analyze %d packages", numberOfPackage)
 	defer handle.Close()
 	log.Debug().Interface("handle", handle).Int("numberOfPackage", numberOfPackage).Msg("got handle")
 	source := pcaphandler.GetPacketSource(handle)
 	var printPayload = func(packetInformation common.PacketInformation, item []byte) {
-		fmt.Printf("%x\n", item)
+		_, _ = fmt.Fprintf(stdout, "%x\n", item)
 	}
 	switch protocolType {
 	case "bacnet":
@@ -49,8 +58,6 @@ func Extract(pcapFile, protocolType string) {
 	case "c-bus":
 		// c-bus is string based so we consume the string and print it
 		clientIp := net.ParseIP(config.ExtractConfigInstance.Client)
-		stdout := ansi.NewAnsiStdout()
-		stderr := ansi.NewAnsiStderr()
 		serverResponseWriter := color.New(color.FgRed)
 		serverResponseIndicatorWriter := color.New(color.FgHiRed)
 		clientRequestWriter := color.New(color.FgGreen)
@@ -143,7 +150,8 @@ func Extract(pcapFile, protocolType string) {
 			printPayload(packetInformation, payload)
 		}
 	}
-	println()
+	_, _ = fmt.Fprintf(stdout, "\n")
 
 	log.Info().Msgf("Done evaluating %d of %d packages (%d failed to parse, %d failed to serialize and %d failed in byte comparison)", currentPackageNum, numberOfPackage, parseFails, serializeFails, compareFails)
+	return nil
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/internal/pcaphandler/pcaphandler.go b/plc4go/tools/plc4xpcapanalyzer/internal/pcaphandler/pcaphandler.go
index a94c4cf8f..957d50f8c 100644
--- a/plc4go/tools/plc4xpcapanalyzer/internal/pcaphandler/pcaphandler.go
+++ b/plc4go/tools/plc4xpcapanalyzer/internal/pcaphandler/pcaphandler.go
@@ -22,6 +22,7 @@ package pcaphandler
 import (
 	"github.com/google/gopacket"
 	"github.com/google/gopacket/pcap"
+	"github.com/pkg/errors"
 	"time"
 )
 
@@ -31,11 +32,14 @@ func GetPacketSource(handle *pcap.Handle) *gopacket.PacketSource {
 }
 
 // GetIndexedPcapHandle returns a *pcap.Handle, the number of packages found and an index which maps timestamp to
-// absolute package number and panics if an error occurs
-func GetIndexedPcapHandle(file, filterExpression string) (*pcap.Handle, int, map[time.Time]int) {
-	timestampToIndexMap := make(map[time.Time]int)
+// absolute package number
+func GetIndexedPcapHandle(file, filterExpression string) (handle *pcap.Handle, numberOfPackages int, timestampToIndexMap map[time.Time]int, err error) {
+	timestampToIndexMap = make(map[time.Time]int)
 	// Count absolute packages and set timestamp map
-	temporaryHandle := GetPcapHandle(file, "")
+	temporaryHandle, err := GetPcapHandle(file, "")
+	if err != nil {
+		return nil, 0, nil, err
+	}
 	defer temporaryHandle.Close()
 	packetSource := GetPacketSource(temporaryHandle)
 	packages := 0
@@ -47,7 +51,10 @@ func GetIndexedPcapHandle(file, filterExpression string) (*pcap.Handle, int, map
 		timestampToIndexMap[packet.Metadata().Timestamp] = packages
 	}
 	// Just count filtered packages
-	temporaryFilteredHandle := GetPcapHandle(file, filterExpression)
+	temporaryFilteredHandle, err := GetPcapHandle(file, filterExpression)
+	if err != nil {
+		return nil, 0, nil, err
+	}
 	defer temporaryFilteredHandle.Close()
 	filteredPacketSource := GetPacketSource(temporaryFilteredHandle)
 	packages = 0
@@ -57,24 +64,23 @@ func GetIndexedPcapHandle(file, filterExpression string) (*pcap.Handle, int, map
 		}
 		packages++
 	}
-	return GetPcapHandle(file, filterExpression), packages, timestampToIndexMap
+	pcapHandle, err := GetPcapHandle(file, filterExpression)
+	if err != nil {
+		return nil, 0, nil, err
+	}
+	return pcapHandle, packages, timestampToIndexMap, nil
 }
 
 // GetPcapHandle returns a *pcap.Handle and panics if an error occurs
-func GetPcapHandle(file, filterExpression string) *pcap.Handle {
-	handle := getPcapHandleOrPanic(file)
+func GetPcapHandle(file, filterExpression string) (*pcap.Handle, error) {
+	handle, err := pcap.OpenOffline(file)
+	if err != nil {
+		return nil, errors.Wrap(err, "error open offline")
+	}
 	if filterExpression != "" {
 		if err := handle.SetBPFFilter(filterExpression); err != nil {
-			panic(err)
+			return nil, errors.Wrap(err, "error setting BPF filter")
 		}
 	}
-	return handle
-}
-
-func getPcapHandleOrPanic(file string) *pcap.Handle {
-	handle, err := pcap.OpenOffline(file)
-	if err != nil {
-		panic(err)
-	}
-	return handle
+	return handle, nil
 }
diff --git a/plc4go/tools/plc4xpcapanalyzer/ui/actions.go b/plc4go/tools/plc4xpcapanalyzer/ui/actions.go
new file mode 100644
index 000000000..fbf0625a8
--- /dev/null
+++ b/plc4go/tools/plc4xpcapanalyzer/ui/actions.go
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package ui
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/internal/ads"
+	"github.com/apache/plc4x/plc4go/internal/bacnetip"
+	"github.com/apache/plc4x/plc4go/internal/cbus"
+	"github.com/apache/plc4x/plc4go/internal/s7"
+	"github.com/apache/plc4x/plc4go/internal/spi"
+	"github.com/apache/plc4x/plc4go/internal/spi/transports/pcap"
+	plc4go "github.com/apache/plc4x/plc4go/pkg/api"
+	"github.com/pkg/errors"
+	"github.com/rs/zerolog"
+	"github.com/rs/zerolog/log"
+	"github.com/sruehl/tview"
+	"os"
+	"path"
+	"strings"
+)
+
+func InitSubsystem() {
+	logLevel := zerolog.InfoLevel
+	if configuredLevel := config.LogLevel; configuredLevel != "" {
+		if parsedLevel, err := zerolog.ParseLevel(configuredLevel); err != nil {
+			panic(err)
+		} else {
+			logLevel = parsedLevel
+		}
+	}
+	driverManager = plc4go.NewPlcDriverManager()
+
+	log.Logger = log.
+		//// Enable below if you want to see the filenames
+		//With().Caller().Logger().
+		Output(zerolog.ConsoleWriter{Out: tview.ANSIWriter(consoleOutput)}).
+		Level(logLevel)
+
+	// We offset the commands executed with the last commands
+	commandsExecuted = len(config.History.Last10Commands)
+	outputCommandHistory()
+
+	for _, driver := range config.AutoRegisterDrivers {
+		log.Info().Msgf("Auto register driver %s", driver)
+		if err := validateDriverParam(driver); err != nil {
+			log.Err(err).Msgf("Invalid configuration")
+			continue
+		}
+		_ = registerDriver(driver)
+	}
+}
+
+func OpenFile(pcapFile string) error {
+	if !strings.HasPrefix(pcapFile, string(os.PathSeparator)) {
+		pcapFile = path.Join(currentDir, pcapFile)
+	}
+	var name string
+	if stat, err := os.Stat(pcapFile); err != nil {
+		return err
+	} else if stat.IsDir() {
+		stat.Name()
+		return errors.Errorf("%s is a dir", pcapFile)
+	} else {
+		name = stat.Name()
+	}
+	for _, loadedPcapFile := range loadedPcapFiles {
+		if loadedPcapFile.path == pcapFile {
+			return errors.Errorf("%s already loaded", name)
+		}
+	}
+	addRecentFilesEntry(pcapFile)
+	loadedPcapFiles = append(loadedPcapFiles, loadedPcapFile{
+		name: name,
+		path: pcapFile,
+	})
+	loadedPcapFilesChanged()
+	return nil
+}
+
+func outputCommandHistory() {
+	_, _ = fmt.Fprintln(commandOutput, "[#0000ff]Last 10 commands[white]")
+	for i, command := range config.History.Last10Commands {
+		_, _ = fmt.Fprintf(commandOutput, "   [#00ff00]%d[white]: [\"%d\"]%s[\"\"]\n", i, i, tview.Escape(command))
+	}
+}
+
+func validateDriverParam(driver string) error {
+	for _, protocol := range protocolList {
+		if protocol == driver {
+			return nil
+		}
+	}
+	return errors.Errorf("protocol %s not found", driver)
+}
+
+func registerDriver(driver string) error {
+	switch driver {
+	case "ads":
+		driverManager.RegisterDriver(ads.NewDriver())
+	case "bacnetip":
+		driverManager.RegisterDriver(bacnetip.NewDriver())
+	case "c-bus":
+		driverManager.RegisterDriver(cbus.NewDriver())
+	case "s7":
+		driverManager.RegisterDriver(s7.NewDriver())
+	default:
+		return errors.Errorf("Unknown driver %s", driver)
+	}
+	driverManager.(spi.TransportAware).RegisterTransport(pcap.NewTransport())
+	go driverAdded(driver)
+	return nil
+}
diff --git a/plc4go/tools/plc4xpcapanalyzer/ui/commands.go b/plc4go/tools/plc4xpcapanalyzer/ui/commands.go
new file mode 100644
index 000000000..a0890874d
--- /dev/null
+++ b/plc4go/tools/plc4xpcapanalyzer/ui/commands.go
@@ -0,0 +1,613 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package ui
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/internal/spi"
+	plc4x_config "github.com/apache/plc4x/plc4go/pkg/api/config"
+	cliConfig "github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/config"
+	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/analyzer"
+	"github.com/apache/plc4x/plc4go/tools/plc4xpcapanalyzer/internal/extractor"
+	"github.com/pkg/errors"
+	"github.com/rs/zerolog"
+	"github.com/rs/zerolog/log"
+	"github.com/sruehl/tview"
+	"os"
+	"path"
+	"strings"
+	"time"
+)
+
+const rootCommandIndicator = "rootCommand"
+
+var rootCommand = Command{
+	Name: rootCommandIndicator,
+	subCommands: []Command{
+		{
+			Name:        "ls",
+			Description: "list directories",
+			action: func(_ Command, dir string) error {
+				if dir == "" {
+					dir = currentDir
+				}
+				_, _ = fmt.Fprintf(commandOutput, "dir cotents of %s\n", dir)
+				readDir, err := os.ReadDir(dir)
+				if err != nil {
+					return err
+				}
+				for _, dirEntry := range readDir {
+					isDir := dirEntry.IsDir()
+					name := dirEntry.Name()
+					name = strings.TrimPrefix(name, dir)
+					if isDir {
+						name = fmt.Sprintf("[#0000ff]%s[white]", name)
+					} else if strings.HasSuffix(name, ".pcap") || strings.HasSuffix(name, ".pcapng") {
+						name = fmt.Sprintf("[#00ff00]%s[white]", name)
+					}
+					_, _ = fmt.Fprintf(commandOutput, "%s\n", name)
+				}
+				return nil
+			},
+			// TODO: add parameter suggestions
+		},
+		{
+			Name:        "cd",
+			Description: "changes directory",
+			action: func(_ Command, newDir string) error {
+				var proposedCurrentDir string
+				if newDir == "" {
+					var err error
+					proposedCurrentDir, err = os.UserHomeDir()
+					if err != nil {
+						return err
+					}
+				} else if strings.HasPrefix(newDir, "."+string(os.PathSeparator)) {
+					proposedCurrentDir = currentDir + strings.TrimPrefix(newDir, ".")
+				} else if strings.HasPrefix(newDir, ""+string(os.PathSeparator)) {
+					proposedCurrentDir = newDir
+				}
+				stat, err := os.Stat(proposedCurrentDir)
+				if err != nil {
+					return err
+				}
+				if !stat.IsDir() {
+					return errors.Errorf("%s is not a dir", newDir)
+				}
+				currentDir = proposedCurrentDir
+				_, _ = fmt.Fprintf(commandOutput, "current directory: %s\n", currentDir)
+				return nil
+			},
+			parameterSuggestions: func(currentText string) (entries []string) {
+				if strings.HasPrefix(currentText, string(os.PathSeparator)) {
+					dirEntries, err := os.ReadDir(currentText)
+					if err != nil {
+						plc4xpcapanalyzerLog.Warn().Err(err).Msg("Error suggesting directories")
+						return
+					}
+					for _, dirEntry := range dirEntries {
+						entry := path.Join(currentText, dirEntry.Name())
+						entries = append(entries, entry)
+					}
+				} else {
+					dirEntries, err := os.ReadDir(currentDir)
+					if err != nil {
+						plc4xpcapanalyzerLog.Warn().Err(err).Msg("Error suggesting directories")
+						return
+					}
+					for _, dirEntry := range dirEntries {
+						entry := path.Join(".", dirEntry.Name())
+						entries = append(entries, entry)
+					}
+				}
+				return
+			},
+		},
+		{
+			Name:        "pwd",
+			Description: "shows current directory",
+			action: func(_ Command, _ string) error {
+				_, _ = fmt.Fprintf(commandOutput, "current directory: %s\n", currentDir)
+				return nil
+			},
+		},
+		{
+			Name:        "open",
+			Description: "open file",
+			action: func(_ Command, pcapFile string) error {
+				return OpenFile(pcapFile)
+			},
+			parameterSuggestions: func(currentText string) (entries []string) {
+				entries = append(entries, config.History.Last10Files...)
+				readDir, err := os.ReadDir(currentDir)
+				if err != nil {
+					return
+				}
+				for _, dirEntry := range readDir {
+					name := dirEntry.Name()
+					name = strings.TrimPrefix(name, currentDir)
+					if strings.HasSuffix(dirEntry.Name(), ".pcap") || strings.HasSuffix(name, ".pcapng") {
+						entries = append(entries, name)
+					}
+				}
+				return
+			},
+		},
+		{
+			Name:        "analyze",
+			Description: "Analyzes a pcap file using a driver",
+			action: func(_ Command, protocolTypeAndPcapFile string) error {
+				split := strings.Split(protocolTypeAndPcapFile, " ")
+				if len(split) != 2 {
+					return errors.Errorf("expect protocol and pcapfile")
+				}
+				protocolType := split[0]
+				pcapFile := strings.TrimPrefix(protocolTypeAndPcapFile, protocolType+" ")
+				cliConfig.PcapConfigInstance.Client = config.HostIp
+				cliConfig.RootConfigInstance.HideProgressBar = true
+				cliConfig.RootConfigInstance.Verbosity = 4
+				return analyzer.AnalyzeWithOutputAndCallback(pcapFile, protocolType, tview.ANSIWriter(messageOutput), tview.ANSIWriter(messageOutput), func(parsed spi.Message) {
+					spiNumberOfMessagesReceived++
+					spiMessageReceived(spiNumberOfMessagesReceived, time.Now(), parsed)
+				})
+			},
+			parameterSuggestions: func(currentText string) (entries []string) {
+				for _, file := range loadedPcapFiles {
+					for _, protocol := range protocolList {
+						entries = append(entries, protocol+" "+file.path)
+					}
+				}
+				return
+			},
+		},
+		{
+			Name:        "extract",
+			Description: "Extract a pcap file using a driver",
+			action: func(_ Command, protocolTypeAndPcapFile string) error {
+				split := strings.Split(protocolTypeAndPcapFile, " ")
+				if len(split) != 2 {
+					return errors.Errorf("expect protocol and pcapfile")
+				}
+				protocolType := split[0]
+				pcapFile := strings.TrimPrefix(protocolTypeAndPcapFile, protocolType+" ")
+				cliConfig.PcapConfigInstance.Client = config.HostIp
+				cliConfig.RootConfigInstance.HideProgressBar = true
+				cliConfig.RootConfigInstance.Verbosity = 4
+				return extractor.ExtractWithOutput(pcapFile, protocolType, tview.ANSIWriter(messageOutput), tview.ANSIWriter(messageOutput))
+			},
+			parameterSuggestions: func(currentText string) (entries []string) {
+				for _, file := range loadedPcapFiles {
+					for _, protocol := range protocolList {
+						entries = append(entries, protocol+" "+file.path)
+					}
+				}
+				return
+			},
+		},
+		{
+			Name:        "host",
+			Description: "The host which is assumed to be the sender (important for protocols that are directional)",
+			subCommands: []Command{
+				{
+					Name: "set",
+					action: func(_ Command, host string) error {
+						config.HostIp = host
+						return nil
+					},
+				},
+				{
+					Name: "get",
+					action: func(_ Command, host string) error {
+						_, _ = fmt.Fprintf(consoleOutput, "current set host %s", config.HostIp)
+						return nil
+					},
+				},
+			},
+		},
+		{
+			Name:        "register",
+			Description: "register a driver in the subsystem",
+			action: func(_ Command, driver string) error {
+				return registerDriver(driver)
+			},
+			parameterSuggestions: func(currentText string) (entries []string) {
+				for _, protocol := range protocolList {
+					if strings.HasPrefix(protocol, currentText) {
+						entries = append(entries, protocol)
+					}
+				}
+				return
+			},
+		},
+		{
+			Name:        "quit",
+			Description: "Quits the application",
+		},
+		{
+			Name:        "log",
+			Description: "Log related operations",
+			subCommands: []Command{
+				{
+					Name:        "get",
+					Description: "Get a log level",
+					action: func(_ Command, _ string) error {
+						_, _ = fmt.Fprintf(commandOutput, "Current log level %s", log.Logger.GetLevel())
+						return nil
+					},
+				},
+				{
+					Name:        "set",
+					Description: "Sets a log level",
+					action: func(_ Command, level string) error {
+						parseLevel, err := zerolog.ParseLevel(level)
+						if err != nil {
+							return errors.Wrapf(err, "Error setting log level")
+						}
+						setLevel(parseLevel)
+						log.Logger = log.Logger.Level(parseLevel)
+						return nil
+					},
+					parameterSuggestions: func(currentText string) (entries []string) {
+						levels := []string{
+							zerolog.LevelTraceValue,
+							zerolog.LevelDebugValue,
+							zerolog.LevelInfoValue,
+							zerolog.LevelWarnValue,
+							zerolog.LevelErrorValue,
+							zerolog.LevelFatalValue,
+							zerolog.LevelPanicValue,
+						}
+						for _, level := range levels {
+							entries = append(entries, level)
+						}
+						return
+					},
+				},
+			},
+		},
+		{
+			Name:        "plc4x-conf",
+			Description: "plc4x related settings",
+			subCommands: []Command{
+				{
+					Name:        "TraceTransactionManagerWorkers",
+					Description: "print information about transaction manager workers",
+					subCommands: []Command{
+						{
+							Name:        "on",
+							Description: "trace on",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceTransactionManagerWorkers = true
+								return nil
+							},
+						},
+						{
+							Name:        "off",
+							Description: "trace off",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceTransactionManagerWorkers = false
+								return nil
+							},
+						},
+					},
+				},
+				{
+					Name:        "TraceTransactionManagerTransactions",
+					Description: "print information about transaction manager transactions",
+					subCommands: []Command{
+						{
+							Name:        "on",
+							Description: "trace on",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceTransactionManagerTransactions = true
+								return nil
+							},
+						},
+						{
+							Name:        "off",
+							Description: "trace off",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceTransactionManagerTransactions = false
+								return nil
+							},
+						},
+					},
+				},
+				{
+					Name:        "TraceDefaultMessageCodecWorker",
+					Description: "print information about message codec workers",
+					subCommands: []Command{
+						{
+							Name:        "on",
+							Description: "trace on",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceDefaultMessageCodecWorker = true
+								return nil
+							},
+						},
+						{
+							Name:        "off",
+							Description: "trace off",
+							action: func(_ Command, _ string) error {
+								plc4x_config.TraceDefaultMessageCodecWorker = false
+								return nil
+							},
+						},
+					},
+				},
+				{
+					Name:        "plc4xpcapanalyzer-debug",
+					Description: "Prints out debug information of the pcap analyzer itself",
+					subCommands: []Command{
+						{
+							Name:        "on",
+							Description: "debug on",
+							action: func(_ Command, _ string) error {
+								plc4xpcapanalyzerLog = zerolog.New(zerolog.ConsoleWriter{Out: tview.ANSIWriter(consoleOutput)})
+								return nil
+							},
+						},
+						{
+							Name:        "off",
+							Description: "debug off",
+							action: func(_ Command, _ string) error {
+								plc4xpcapanalyzerLog = zerolog.Nop()
+								return nil
+							},
+						},
+					},
+				},
+				{
+					Name:        "auto-register",
+					Description: "autoregister driver at startup",
+					subCommands: []Command{
+						{
+							Name: "list",
+							action: func(currentCommand Command, argument string) error {
+								_, _ = fmt.Fprintf(commandOutput, "Auto-register enabled drivers:\n  %s\n", strings.Join(config.AutoRegisterDrivers, "\n  "))
+								return nil
+							},
+						},
+						{
+							Name: "enable",
+							action: func(_ Command, argument string) error {
+								return enableAutoRegister(argument)
+							},
+							parameterSuggestions: func(currentText string) (entries []string) {
+								for _, protocol := range protocolList {
+									if strings.HasPrefix(protocol, currentText) {
+										entries = append(entries, protocol)
+									}
+								}
+								return
+							},
+						},
+						{
+							Name: "disable",
+							action: func(_ Command, argument string) error {
+								return disableAutoRegister(argument)
+							},
+							parameterSuggestions: func(currentText string) (entries []string) {
+								for _, protocol := range protocolList {
+									if strings.HasPrefix(protocol, currentText) {
+										entries = append(entries, protocol)
+									}
+								}
+								return
+							},
+						},
+					},
+				},
+			},
+		},
+		{
+			Name:        "history",
+			Description: "outputs the last commands",
+			action: func(_ Command, _ string) error {
+				outputCommandHistory()
+				return nil
+			},
+		},
+		{
+			Name:        "clear",
+			Description: "clear all outputs",
+			action: func(_ Command, _ string) error {
+				messageOutputClear()
+				consoleOutputClear()
+				commandOutputClear()
+				return nil
+			},
+			subCommands: []Command{
+				{
+					Name:        "message",
+					Description: "clears message output",
+					action: func(_ Command, _ string) error {
+						messageOutputClear()
+						return nil
+					},
+				},
+				{
+					Name:        "console",
+					Description: "clears console output",
+					action: func(_ Command, _ string) error {
+						consoleOutputClear()
+						return nil
+					},
+				},
+				{
+					Name:        "command",
+					Description: "clears command output",
+					action: func(_ Command, _ string) error {
+						commandOutputClear()
+						return nil
+					},
+				},
+			},
+		},
+	},
+}
+
+func init() {
+	// Because of the cycle we need to define the help command here as it needs access to the to command
+	rootCommand.subCommands = append(rootCommand.subCommands, Command{
+		Name:        "help",
+		Description: "prints out this help",
+		action: func(_ Command, _ string) error {
+			_, _ = fmt.Fprintf(commandOutput, "[#0000ff]Available commands[white]\n")
+			rootCommand.visit(0, func(currentIndent int, command Command) {
+				indentString := strings.Repeat("  ", currentIndent)
+				description := command.Description
+				if description == "" {
+					description = command.Name + "s"
+				}
+				_, _ = fmt.Fprintf(commandOutput, "%s [#00ff00]%s[white]: %s\n", indentString, command.Name, description)
+			})
+			return nil
+		},
+	})
+}
+
+var NotDirectlyExecutable = errors.New("Not directly executable")
+
+type Command struct {
+	Name                 string
+	Description          string
+	action               func(currentCommand Command, argument string) error
+	subCommands          []Command
+	parameterSuggestions func(currentText string) (entries []string)
+}
+
+func (c Command) Completions(currentCommandText string) (entries []string) {
+	if c.Name == rootCommandIndicator && len(currentCommandText) == 0 {
+		// We don't return anything here to not pollute the command text by default
+		return
+	}
+	if c.acceptsCurrentText(currentCommandText) {
+		currentCommandPrefix := c.currentCommandPrefix()
+		doesCommandTextTargetSubCommand := c.doesCommandTextTargetSubCommand(currentCommandPrefix)
+		if c.hasDirectExecution() && !doesCommandTextTargetSubCommand {
+			if c.parameterSuggestions != nil {
+				preparedForParameters := c.prepareForParameters(currentCommandText)
+				for _, parameterSuggestion := range c.parameterSuggestions(preparedForParameters) {
+					entries = append(entries, currentCommandPrefix+parameterSuggestion)
+				}
+			} else if currentCommandText == "" {
+				entries = append(entries, c.Name)
+			}
+		}
+		if doesCommandTextTargetSubCommand {
+			remainder := c.prepareForSubCommand(currentCommandText)
+			for _, command := range c.subCommands {
+				for _, subCommandCompletions := range command.Completions(remainder) {
+					entries = append(entries, currentCommandPrefix+subCommandCompletions)
+				}
+			}
+		}
+	} else if strings.HasPrefix(c.Name, currentCommandText) {
+		// Suggest ourselves if we start with the current letter
+		entries = append(entries, c.Name)
+	}
+	return
+}
+
+func (c Command) acceptsCurrentText(currentCommandText string) bool {
+	if c.Name == rootCommandIndicator {
+		return true
+	}
+	hasThePrefix := strings.HasPrefix(currentCommandText, c.Name)
+	hasNoMatchingAlternative := !strings.HasPrefix(currentCommandText, c.Name+"-")
+	accepts := hasThePrefix && hasNoMatchingAlternative
+	plc4xpcapanalyzerLog.Debug().Msgf("%s accepts %t", c, accepts)
+	return accepts
+}
+
+func (c Command) doesCommandTextTargetSubCommand(currentCommandText string) bool {
+	if c.Name == rootCommandIndicator {
+		return true
+	}
+	if len(c.subCommands) == 0 {
+		return false
+	}
+	return strings.HasPrefix(currentCommandText, c.currentCommandPrefix())
+}
+
+func (c Command) prepareForParameters(currentCommandText string) string {
+	if currentCommandText == c.Name {
+		return ""
+	}
+	return strings.TrimPrefix(currentCommandText, c.currentCommandPrefix())
+}
+func (c Command) prepareForSubCommand(currentCommandText string) string {
+	return strings.TrimPrefix(currentCommandText, c.currentCommandPrefix())
+}
+
+func (c Command) currentCommandPrefix() string {
+	if c.Name == rootCommandIndicator {
+		return ""
+	}
+	return c.Name + " "
+}
+
+func (c Command) hasDirectExecution() bool {
+	return c.action != nil
+}
+
+func Execute(commandText string) error {
+	err := rootCommand.Execute(commandText)
+	if err == nil {
+		addCommandHistoryEntry(commandText)
+	}
+	return err
+}
+
+func (c Command) Execute(commandText string) error {
+	plc4xpcapanalyzerLog.Debug().Msgf("%s executes %s", c, commandText)
+	if !c.acceptsCurrentText(commandText) {
+		return errors.Errorf("%s doesn't understand %s", c.Name, commandText)
+	}
+	if c.doesCommandTextTargetSubCommand(commandText) {
+		prepareForSubCommandForSubCommand := c.prepareForSubCommand(commandText)
+		for _, command := range c.subCommands {
+			if command.acceptsCurrentText(prepareForSubCommandForSubCommand) {
+				plc4xpcapanalyzerLog.Debug().Msgf("%s delegates to sub %s", c, command)
+				return command.Execute(prepareForSubCommandForSubCommand)
+			}
+		}
+		return errors.Errorf("%s not accepted by any subcommands of %s", commandText, c.Name)
+	} else {
+		if c.action == nil {
+			return NotDirectlyExecutable
+		}
+		plc4xpcapanalyzerLog.Debug().Msgf("%s executes %s directly", c, commandText)
+		preparedForParameters := c.prepareForParameters(commandText)
+		return c.action(c, preparedForParameters)
+	}
+}
+
+func (c Command) visit(i int, f func(currentIndent int, command Command)) {
+	f(i, c)
+	for _, subCommand := range c.subCommands {
+		subCommand.visit(i+1, f)
+	}
+}
+
+func (c Command) String() string {
+	return c.Name
+}
diff --git a/plc4go/tools/plc4xpcapanalyzer/ui/common.go b/plc4go/tools/plc4xpcapanalyzer/ui/common.go
new file mode 100644
index 000000000..c716a66d8
--- /dev/null
+++ b/plc4go/tools/plc4xpcapanalyzer/ui/common.go
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package ui
+
+import (
+	"github.com/apache/plc4x/plc4go/internal/spi"
+	plc4go "github.com/apache/plc4x/plc4go/pkg/api"
+	plc4goModel "github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/rs/zerolog"
+	"io"
+	"os"
+	"strings"
+	"sync"
+	"time"
+)
+
+const protocols = "ads,bacnetip,c-bus,s7"
+
+var protocolList = strings.Split(protocols, ",")
+
+var plc4xpcapanalyzerLog = zerolog.Nop()
+
+var driverManager plc4go.PlcDriverManager
+var driverAdded func(string)
+
+type loadedPcapFile struct {
+	name string
+	path string
+}
+
+var loadedPcapFiles []loadedPcapFile
+var loadedPcapFilesChanged func()
+
+var messageReceived func(messageNumber int, receiveTime time.Time, message plc4goModel.PlcMessage)
+var numberOfMessagesReceived int
+var spiMessageReceived func(messageNumber int, receiveTime time.Time, message spi.Message)
+var spiNumberOfMessagesReceived int
+var messageOutput io.Writer
+var messageOutputClear func()
+
+var consoleOutput io.Writer
+var consoleOutputClear func()
+
+var commandsExecuted int
+var commandOutput io.Writer
+var commandOutputClear func()
+
+var currentDirChanged func()
+var currentDir = func() string {
+	dir, _ := os.Getwd()
+	return dir
+}()
+
+var shutdownMutex sync.Mutex
+var hasShutdown bool
+
+func Shutdown() {
+	shutdownMutex.Lock()
+	defer shutdownMutex.Unlock()
+	if hasShutdown {
+		return
+	}
+	saveConfig()
+	hasShutdown = true
+}
diff --git a/plc4go/tools/plc4xpcapanalyzer/ui/config.go b/plc4go/tools/plc4xpcapanalyzer/ui/config.go
new file mode 100644
index 000000000..100f8c440
--- /dev/null
+++ b/plc4go/tools/plc4xpcapanalyzer/ui/config.go
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package ui
+
+import (
+	"github.com/pkg/errors"
+	"github.com/rs/zerolog"
+	"github.com/rs/zerolog/log"
+	"gopkg.in/yaml.v3"
+	"os"
+	"path"
+	"time"
+)
+
+var plc4xpcapanalyzerConfigDir string
+var configFile string
+var config Config
+
+type Config struct {
+	HostIp  string `yaml:"host_ip"`
+	History struct {
+		Last10Files    []string `yaml:"last_hosts"`
+		Last10Commands []string `yaml:"last_commands"`
+	}
+	AutoRegisterDrivers []string  `yaml:"auto_register_driver"`
+	LastUpdated         time.Time `yaml:"last_updated"`
+	LogLevel            string    `yaml:"log_level"`
+	MaxConsoleLines     int       `yaml:"max_console_lines"`
+	MaxOutputLines      int       `yaml:"max_output_lines"`
+}
+
+func init() {
+	userConfigDir, err := os.UserConfigDir()
+	if err != nil {
+		panic(err)
+	}
+	plc4xpcapanalyzerConfigDir = path.Join(userConfigDir, "plc4xpcapanalyzer")
+	if _, err := os.Stat(plc4xpcapanalyzerConfigDir); os.IsNotExist(err) {
+		err := os.Mkdir(plc4xpcapanalyzerConfigDir, os.ModeDir|os.ModePerm)
+		if err != nil {
+			panic(err)
+		}
+	}
+	configFile = path.Join(plc4xpcapanalyzerConfigDir, "config.yml")
+}
+
+func LoadConfig() {
+	f, err := os.Open(configFile)
+	if err != nil {
+		log.Info().Err(err).Msg("No config file found")
+		return
+	}
+	defer func(f *os.File) {
+		err := f.Close()
+		if err != nil {
+			log.Error().Err(err).Msg("Error closing config file")
+		}
+	}(f)
+
+	decoder := yaml.NewDecoder(f)
+	if err = decoder.Decode(&config); err != nil {
+		log.Warn().Err(err).Msg("Can't decode config file")
+		return
+	}
+}
+
+func saveConfig() {
+	config.LastUpdated = time.Now()
+	f, err := os.OpenFile(configFile, os.O_RDWR|os.O_CREATE, 0755)
+	if err != nil {
+		log.Warn().Err(err).Msg("Can't save config file")
+		return
+	}
+	encoder := yaml.NewEncoder(f)
+	defer func(encoder *yaml.Encoder) {
+		err := encoder.Close()
+		if err != nil {
+			log.Error().Err(err).Msg("Error closing config file")
+		}
+	}(encoder)
+	if err := encoder.Encode(config); err != nil {
+		log.Warn().Err(err).Msg("Can't encode config file")
+		panic(err)
+	}
+}
+
+func addRecentFilesEntry(pcapFile string) {
+	existingIndex := -1
+	for i, lastPcapFile := range config.History.Last10Files {
+		if lastPcapFile == pcapFile {
+			existingIndex = i
+			break
+		}
+	}
+	if existingIndex >= 0 {
+		config.History.Last10Files = append(config.History.Last10Files[:existingIndex], config.History.Last10Files[existingIndex+1:]...)
+	}
+	if len(config.History.Last10Files) >= 10 {
+		config.History.Last10Files = config.History.Last10Files[1:]
+	}
+	config.History.Last10Files = append(config.History.Last10Files, pcapFile)
+}
+
+func addCommandHistoryEntry(command string) {
+	switch command {
+	case "clear":
+		return
+	case "history":
+		return
+	}
+	existingIndex := -1
+	for i, lastCommand := range config.History.Last10Commands {
+		if lastCommand == command {
+			existingIndex = i
+			break
+		}
+	}
+	if existingIndex >= 0 {
+		config.History.Last10Commands = append(config.History.Last10Commands[:existingIndex], config.History.Last10Commands[existingIndex+1:]...)
+	}
+	if len(config.History.Last10Commands) >= 10 {
+		config.History.Last10Commands = config.History.Last10Commands[1:]
+	}
+	config.History.Last10Commands = append(config.History.Last10Commands, command)
+}
+
+func setLevel(level zerolog.Level) {
+	config.LogLevel = level.String()
+}
+
+func enableAutoRegister(driver string) error {
+	if err := validateDriverParam(driver); err != nil {
+		return err
+	}
+	for _, autoRegisterDriver := range config.AutoRegisterDrivers {
+		if autoRegisterDriver == driver {
+			return errors.Errorf("%s already registered for auto register", driver)
+		}
+	}
+	config.AutoRegisterDrivers = append(config.AutoRegisterDrivers, driver)
+	log.Info().Msgf("Auto register enabled for %s", driver)
+	return nil
+}
+
+func disableAutoRegister(driver string) error {
+	if err := validateDriverParam(driver); err != nil {
+		return err
+	}
+	index := -1
+	for i, autoRegisterDriver := range config.AutoRegisterDrivers {
+		if autoRegisterDriver == driver {
+			index = i
+			break
+		}
+	}
+	if index < 0 {
+		return errors.Errorf("%s not registered for auto register", driver)
+	}
+	config.AutoRegisterDrivers = append(config.AutoRegisterDrivers[:index], config.AutoRegisterDrivers[index+1:]...)
+	log.Info().Msgf("Auto register disabled for %s", driver)
+	return nil
+}
diff --git a/plc4go/tools/plc4xpcapanalyzer/ui/ui.go b/plc4go/tools/plc4xpcapanalyzer/ui/ui.go
new file mode 100644
index 000000000..8ea89fa43
--- /dev/null
+++ b/plc4go/tools/plc4xpcapanalyzer/ui/ui.go
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   https://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package ui
+
+import (
+	"fmt"
+	"github.com/apache/plc4x/plc4go/internal/spi"
+	"github.com/apache/plc4x/plc4go/pkg/api/model"
+	"github.com/gdamore/tcell/v2"
+	"github.com/pkg/errors"
+	"github.com/sruehl/tview"
+	"regexp"
+	"strconv"
+	"time"
+)
+
+func SetupApplication() *tview.Application {
+	application := tview.NewApplication()
+
+	newPrimitive := func(text string) tview.Primitive {
+		return tview.NewTextView().
+			SetTextAlign(tview.AlignCenter).
+			SetText(text)
+	}
+	fileArea := buildFileArea(newPrimitive, application)
+	outputArea := buildOutputArea(newPrimitive, application)
+	commandArea := buildCommandArea(newPrimitive, application)
+
+	grid := tview.NewGrid().
+		SetRows(1, 0, 1).
+		SetColumns(30, 0, 30).
+		SetBorders(true).
+		AddItem(newPrimitive("PLC4X PCAP Analyzer"), 0, 0, 1, 3, 0, 0, false).
+		AddItem(newPrimitive("https://github.com/apache/plc4x"), 2, 0, 1, 3, 0, 0, false)
+
+	// Layout for screens narrower than 100 cells (fileArea and side bar are hidden).
+	grid.AddItem(fileArea, 0, 0, 0, 0, 0, 0, false).
+		AddItem(outputArea, 1, 0, 1, 3, 0, 0, false).
+		AddItem(commandArea, 0, 0, 0, 0, 0, 0, true)
+
+	// Layout for screens wider than 100 cells.
+	grid.AddItem(fileArea, 1, 0, 1, 1, 0, 100, false).
+		AddItem(outputArea, 1, 1, 1, 1, 0, 100, false).
+		AddItem(commandArea, 1, 2, 1, 1, 0, 100, false)
+
+	application.SetRoot(grid, true).EnableMouse(true)
+
+	return application
+}
+
+func buildFileArea(newPrimitive func(text string) tview.Primitive, application *tview.Application) tview.Primitive {
+	connectionAreaHeader := newPrimitive("Files")
+	connectionArea := tview.NewGrid().
+		SetRows(3, 0, 10).
+		SetColumns(0).
+		AddItem(connectionAreaHeader, 0, 0, 1, 1, 0, 0, false)
+	{
+		fileList := tview.NewList()
+		loadedPcapFilesChanged = func() {
+			application.QueueUpdateDraw(func() {
+				fileList.Clear()
+				for _, pcapFile := range loadedPcapFiles {
+					fileList.AddItem(pcapFile.name, pcapFile.path, 0x0, func() {
+						//TODO: disconnect popup
+						_ = pcapFile
+					})
+				}
+			})
+		}
+		connectionArea.AddItem(fileList, 1, 0, 1, 1, 0, 0, false)
+		{
+			registeredDriverAreaHeader := newPrimitive("Registered drivers")
+			registeredDriverArea := tview.NewGrid().
+				SetRows(3, 0).
+				SetColumns(0).
+				AddItem(registeredDriverAreaHeader, 0, 0, 1, 1, 0, 0, false)
+			{
+				driverList := tview.NewList()
+				driverAdded = func(driver string) {
+					application.QueueUpdateDraw(func() {
+						driverList.AddItem(driver, "", 0x0, func() {
+							//TODO: disconnect popup
+						})
+					})
+				}
+				registeredDriverArea.AddItem(driverList, 1, 0, 1, 1, 0, 0, false)
+			}
+			connectionArea.AddItem(registeredDriverArea, 2, 0, 1, 1, 0, 0, false)
+		}
+
+	}
+	return connectionArea
+}
+
+func buildCommandArea(newPrimitive func(text string) tview.Primitive, application *tview.Application) tview.Primitive {
+	commandAreaHeader := newPrimitive("Commands")
+	commandArea := tview.NewGrid().
+		SetRows(3, 0, 3).
+		SetColumns(0).
+		AddItem(commandAreaHeader, 0, 0, 1, 1, 0, 0, false)
+	{
+		enteredCommandsView := tview.NewTextView().
+			SetDynamicColors(true).
+			SetRegions(true).
+			SetWordWrap(true).
+			SetChangedFunc(func() {
+				application.Draw()
+			})
+		commandOutput = enteredCommandsView
+		commandOutputClear = func() {
+			enteredCommandsView.SetText("")
+		}
+
+		commandArea.AddItem(enteredCommandsView, 1, 0, 1, 1, 0, 0, false)
+
+		commandInputField := tview.NewInputField().
+			SetLabel("$").
+			SetFieldWidth(30)
+		application.SetInputCapture(func(event *tcell.EventKey) *tcell.EventKey {
+			switch event.Key() {
+			case tcell.KeyCtrlC:
+				commandInputField.SetText("")
+				application.SetFocus(commandInputField)
+				return nil
+			case tcell.KeyCtrlD:
+				// TODO: maybe add a modal here
+				application.Stop()
+				return nil
+			}
+			return event
+		})
+		commandInputField.
+			SetDoneFunc(func(key tcell.Key) {
+				commandText := commandInputField.GetText()
+				if commandText == "quit" {
+					// TODO: maybe add a modal here
+					application.Stop()
+					return
+				}
+				commandsExecuted++
+				go func() {
+					commandHistoryShortcut, _ := regexp.Compile("^[0-9]$")
+					if commandHistoryShortcut.MatchString(commandText) {
+						atoi, _ := strconv.Atoi(commandHistoryShortcut.FindString(commandText))
+						if atoi < len(config.History.Last10Commands) {
+							commandText = config.History.Last10Commands[atoi]
+						} else {
+							_, _ = fmt.Fprintf(enteredCommandsView, "[#ff0000]%s %s[white]\n", time.Now().Format("04:05"), errors.Errorf("No such elements %d in command history", atoi))
+							return
+						}
+					}
+					_, _ = fmt.Fprintf(enteredCommandsView, "%s [\"%d\"]%s[\"\"]\n", time.Now().Format("04:05"), commandsExecuted, commandText)
+					if err := Execute(commandText); err != nil {
+						_, _ = fmt.Fprintf(enteredCommandsView, "[#ff0000]%s %s[white]\n", time.Now().Format("04:05"), err)
+						return
+					}
+					application.QueueUpdateDraw(func() {
+						commandInputField.SetText("")
+					})
+				}()
+			})
+		commandInputField.SetAutocompleteFunc(rootCommand.Completions)
+
+		enteredCommandsView.SetDoneFunc(func(key tcell.Key) {
+			currentSelection := enteredCommandsView.GetHighlights()
+			if key == tcell.KeyEnter {
+				if len(currentSelection) > 0 {
+					enteredCommandsView.Highlight()
+				} else {
+					enteredCommandsView.Highlight("0").ScrollToHighlight()
+				}
+				if len(currentSelection) == 1 {
+					// TODO: currently this is broken due to https://github.com/rivo/tview/issues/751 (workaround active with sruehl fix fork)
+					commandInputField.SetText(enteredCommandsView.GetRegionText(currentSelection[0]))
+					application.SetFocus(commandInputField)
+				}
+			} else if len(currentSelection) > 0 {
+				index, _ := strconv.Atoi(currentSelection[0])
+				if key == tcell.KeyTab {
+					index = (index + 1) % commandsExecuted
+				} else if key == tcell.KeyBacktab {
+					index = (index - 1 + commandsExecuted) % commandsExecuted
+				} else {
+					return
+				}
+				enteredCommandsView.Highlight(strconv.Itoa(index)).ScrollToHighlight()
+			}
+		})
+
+		commandArea.AddItem(commandInputField, 2, 0, 1, 1, 0, 0, true)
+	}
+	return commandArea
+}
+
+func buildOutputArea(newPrimitive func(text string) tview.Primitive, application *tview.Application) *tview.Grid {
+	outputAreaHeader := newPrimitive("Output")
+	outputArea := tview.NewGrid().
+		SetRows(3, 0, 10).
+		SetColumns(0, 30).
+		AddItem(outputAreaHeader, 0, 0, 1, 1, 0, 0, false)
+	{
+		var jumpToMessageItem func(messageNumber int) bool
+		{
+			outputView := tview.NewTextView().
+				SetDynamicColors(true). // TODO: currently this is broken due to https://github.com/rivo/tview/issues/751 (workaround active with sruehl fix fork)
+				SetRegions(true).
+				SetWordWrap(false).
+				SetWrap(false).
+				SetChangedFunc(func() {
+					application.Draw()
+				})
+			jumpToMessageItem = func(messageNumber int) bool {
+				regionId := strconv.Itoa(messageNumber)
+				if outputView.GetRegionText(regionId) == "" {
+					return false
+				}
+				outputView.Highlight(regionId).ScrollToHighlight()
+				return true
+			}
+			messageOutput = outputView
+			messageOutputClear = func() {
+				outputView.SetText("")
+			}
+
+			outputView.SetDoneFunc(func(key tcell.Key) {
+				currentSelection := outputView.GetHighlights()
+				if key == tcell.KeyEnter {
+					if len(currentSelection) > 0 {
+						outputView.Highlight()
+					} else {
+						outputView.Highlight("0").ScrollToHighlight()
+					}
+				} else if len(currentSelection) > 0 {
+					index, _ := strconv.Atoi(currentSelection[0])
+					if key == tcell.KeyTab {
+						index = (index + 1) % numberOfMessagesReceived
+					} else if key == tcell.KeyBacktab {
+						index = (index - 1 + numberOfMessagesReceived) % numberOfMessagesReceived
+					} else {
+						return
+					}
+					outputView.Highlight(strconv.Itoa(index)).ScrollToHighlight()
+				}
+			})
+			outputView.SetBorder(false)
+			outputArea.AddItem(outputView, 1, 0, 1, 1, 0, 0, false)
+		}
+
+		{
+			consoleView := tview.NewTextView().
+				SetDynamicColors(true).
+				SetMaxLines(config.MaxConsoleLines).
+				SetChangedFunc(func() {
+					application.Draw()
+				})
+			consoleOutput = consoleView
+			consoleOutputClear = func() {
+				consoleView.SetText("")
+			}
+
+			consoleView.SetBorder(false)
+			outputArea.AddItem(consoleView, 2, 0, 1, 1, 0, 0, false)
+		}
+
+		{
+			receivedMessagesList := tview.NewList()
+			messageReceived = func(messageNumber int, receiveTime time.Time, message model.PlcMessage) {
+				application.QueueUpdateDraw(func() {
+					receivedMessagesList.AddItem(fmt.Sprintf("No %d @%s (api)", messageNumber, receiveTime.Format("15:04:05.999999")), "", 0x0, func() {
+						if ok := jumpToMessageItem(messageNumber); !ok {
+							plc4xpcapanalyzerLog.Debug().Msgf("Adding new message to console output")
+							_, _ = fmt.Fprintf(messageOutput, "Message nr: %d\n[\"%d\"]%s[\"\"]\n", messageNumber, messageNumber, message)
+							jumpToMessageItem(messageNumber)
+						}
+					})
+				})
+			}
+			spiMessageReceived = func(messageNumber int, receiveTime time.Time, message spi.Message) {
+				application.QueueUpdateDraw(func() {
+					receivedMessagesList.AddItem(fmt.Sprintf("No %d @%s (spi)", messageNumber, receiveTime.Format("15:04:05.999999")), "", 0x0, func() {
+						if ok := jumpToMessageItem(messageNumber); !ok {
+							plc4xpcapanalyzerLog.Debug().Msgf("Adding new spi message to console output")
+							_, _ = fmt.Fprintf(messageOutput, "Message nr: %d\n[\"%d\"]%s[\"\"]\n", messageNumber, messageNumber, message)
+							jumpToMessageItem(messageNumber)
+						}
+					})
+				})
+			}
+			outputArea.AddItem(receivedMessagesList, 0, 1, 3, 1, 0, 0, false)
+		}
+	}
+	return outputArea
+}


[plc4x] 02/03: feat(plc4xbrowser): increase display area

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 ff7d444e1fefbca98ef7db372d126197c88ff440
Author: Sebastian Rühl <sr...@apache.org>
AuthorDate: Tue Aug 9 11:02:45 2022 +0200

    feat(plc4xbrowser): increase display area
---
 plc4go/tools/plc4xbrowser/ui/ui.go | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/plc4go/tools/plc4xbrowser/ui/ui.go b/plc4go/tools/plc4xbrowser/ui/ui.go
index fda7476c0..7d3c30caf 100644
--- a/plc4go/tools/plc4xbrowser/ui/ui.go
+++ b/plc4go/tools/plc4xbrowser/ui/ui.go
@@ -43,7 +43,7 @@ func SetupApplication() *tview.Application {
 	commandArea := buildCommandArea(newPrimitive, application)
 
 	grid := tview.NewGrid().
-		SetRows(3, 0, 1).
+		SetRows(1, 0, 1).
 		SetColumns(30, 0, 30).
 		SetBorders(true).
 		AddItem(newPrimitive("PLC4X Browser"), 0, 0, 1, 3, 0, 0, false).