You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@devlake.apache.org by he...@apache.org on 2023/02/09 01:42:19 UTC

[incubator-devlake] branch main updated: [feat-3701][Framework]: Python framework support for plugins (#3702)

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

hez pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-devlake.git


The following commit(s) were added to refs/heads/main by this push:
     new c3ab5ddcb [feat-3701][Framework]: Python framework support for plugins (#3702)
c3ab5ddcb is described below

commit c3ab5ddcb977c741ae4cf40b367476835d867e40
Author: Keon Amini <ke...@merico.dev>
AuthorDate: Wed Feb 8 19:42:14 2023 -0600

    [feat-3701][Framework]: Python framework support for plugins (#3702)
    
    * feat: reworked the ipc library for more general operations
    
    * feat: Remote plugin connectivity Golang framework with CMD-implementation
    
    1. Support for runtime structs and integration with GORM
    2. Support for runtime swagger documentation
    3. Default plugin API implementation for remote plugins (CRUD connection API)
    4. RPC and streaming communication abstraction with remote plugins
    5. CMD-based RPC impl (for Python plugins)
    6. Remote plugins will have to expose functions that match the method names on Go Plugin API
    7. Async registration of plugins via an Http API (including their Swagger docs)
    8. Framework bootstraps all remote plugins via a shell script call on server startup
    
    * feat: Add poetry install to Dockerfile
    
    * feat: Python plugin framework
    
    * test: Implement fake python plugin for tests
    
    * test: Remote_test.go and helper classes added
    
    * feat: Dockerfiles adjusted for python and some necessary changes for gitextractor
    
    * feat: Add script to run remote plugin via DirectRun
    
    * feat: dependencies for remote debugging python
    
    * feat: Blueprint v2 support for devlake test client
    
    * fix: fixes for pr checks
    
    ---------
    
    Co-authored-by: Camille Teruel <ca...@meri.co>
---
 .gitignore                                         |   4 +
 .licenserc.yaml                                    |   2 +
 backend/Dockerfile                                 |   4 +-
 backend/core/config/config_viper.go                |   5 +-
 backend/core/models/dynamic_tabler.go              |  75 +++
 backend/core/utils/ipc.go                          | 267 +++++++--
 backend/go.mod                                     |   3 +-
 backend/go.sum                                     |   6 +
 .../helpers/pluginhelper/api/connection_helper.go  |  22 +-
 backend/helpers/pluginhelper/api/mapstructure.go   |   9 +-
 backend/helpers/pluginhelper/api/misc_helpers.go   |  33 ++
 .../helpers/pluginhelper/tap/singer_tap_impl.go    |  32 +-
 backend/helpers/pluginhelper/tap/tap.go            |  13 +-
 backend/helpers/pluginhelper/tap/tap_collector.go  |   4 +-
 backend/impls/dalgorm/encdec_serializer.go         |   5 +-
 .licenserc.yaml => backend/python/build.sh         |  55 +-
 .licenserc.yaml => backend/python/plugins/start.sh |  54 +-
 backend/python/pydevlake/README.md                 | 334 +++++++++++
 backend/python/pydevlake/poetry.lock               | 624 +++++++++++++++++++++
 .../python/pydevlake/pydevlake/__init__.py         |  52 +-
 backend/python/pydevlake/pydevlake/api.py          | 290 ++++++++++
 backend/python/pydevlake/pydevlake/context.py      |  52 ++
 .../python/pydevlake/pydevlake/doc.template.json   | 155 +++++
 .../python/pydevlake/pydevlake/docgen.py           |  67 +--
 .../pydevlake/pydevlake/domain_layer/__init__.py   |  48 +-
 .../pydevlake/domain_layer/crossdomain.py          |  50 +-
 .../pydevlake/pydevlake/domain_layer/devops.py     |  68 +++
 .../python/pydevlake/pydevlake/extractor.py        |  55 +-
 backend/python/pydevlake/pydevlake/ipc.py          |  93 +++
 .../python/pydevlake/pydevlake/logger.py           |  75 +--
 backend/python/pydevlake/pydevlake/message.py      |  90 +++
 backend/python/pydevlake/pydevlake/model.py        | 100 ++++
 backend/python/pydevlake/pydevlake/plugin.py       | 174 ++++++
 backend/python/pydevlake/pydevlake/stream.py       |  99 ++++
 backend/python/pydevlake/pydevlake/subtasks.py     | 215 +++++++
 .../python/pydevlake/pyproject.toml                |  66 +--
 backend/python/pydevlake/test/remote_test.go       | 100 ++++
 backend/python/pydevlake/test/stream_test.py       | 141 +++++
 backend/python/requirements.txt                    |   4 +-
 backend/server/api/api.go                          |  36 +-
 backend/server/api/remote/models.go                |  35 ++
 backend/server/api/remote/register.go              | 107 ++++
 backend/server/api/router.go                       |  32 +-
 backend/server/services/remote/bridge/bootstrap.go |  59 ++
 backend/server/services/remote/bridge/bridge.go    |  64 +++
 backend/server/services/remote/bridge/cmd.go       | 156 ++++++
 backend/server/services/remote/bridge/context.go   | 140 +++++
 .../server/services/remote/bridge/python_cmd.go    |  38 ++
 backend/server/services/remote/bridge/returns.go   |  88 +++
 backend/server/services/remote/bridge/utils.go     |  57 ++
 backend/server/services/remote/init.go             |  56 ++
 .../server/services/remote/models/conversion.go    | 100 ++++
 backend/server/services/remote/models/models.go    |  58 ++
 .../server/services/remote/models/plugin_models.go |  41 ++
 .../server/services/remote/models/plugin_remote.go |  31 +
 .../services/remote/plugin/connection_api.go       |  92 +++
 .../server/services/remote/plugin/default_api.go   |  47 ++
 backend/server/services/remote/plugin/init.go      |  61 ++
 .../services/remote/plugin/plugin_extensions.go    |  64 +++
 .../server/services/remote/plugin/plugin_impl.go   | 131 +++++
 backend/server/services/remote/run/run.go          |  65 +++
 backend/test/helper/api.go                         | 253 +++++++++
 backend/test/helper/client.go                      | 307 ++++++++++
 backend/test/helper/db_selector.go                 |  41 ++
 backend/test/helper/init.go                        |  51 ++
 backend/test/helper/json_helper.go                 | 104 ++++
 backend/test/helper/utils.go                       |  91 +++
 .../test/remote/docker-compose.test.yml            |  69 +--
 .../test/remote/fakeplugin/build.sh                |  55 +-
 .../test/remote/fakeplugin/fakeplugin/__init__.py  |  48 +-
 backend/test/remote/fakeplugin/fakeplugin/main.py  | 122 ++++
 backend/test/remote/fakeplugin/poetry.lock         | 513 +++++++++++++++++
 .../test/remote/fakeplugin/pyproject.toml          |  55 +-
 .../test/remote/fakeplugin/start.sh                |  54 +-
 backend/test/remote/remote_test.go                 | 113 ++++
 75 files changed, 6271 insertions(+), 708 deletions(-)

diff --git a/.gitignore b/.gitignore
index 69e9225c9..51750dad3 100644
--- a/.gitignore
+++ b/.gitignore
@@ -159,3 +159,7 @@ backend/server/api/docs/swagger.json
 backend/server/api/docs/swagger.yaml
 backend/server/api/docs/docs.go
 *.result/
+
+# python
+*.pyc
+__pycache__
diff --git a/.licenserc.yaml b/.licenserc.yaml
index b11fd8254..aa9cd51ef 100644
--- a/.licenserc.yaml
+++ b/.licenserc.yaml
@@ -34,6 +34,7 @@ header:
     - '**/env.example'
     - '**/*.csv'
     - '**/*.json'
+    - '**/*.template.json'
     - '**/*.sql'
     - '**/*.svg'
     - '**/*.png'
@@ -52,6 +53,7 @@ header:
     - '**/.eslintrc'
     - 'deployment/helm/templates/_helpers.tpl'
     - '**/.nvmrc'
+    - '**/poetry.lock'
 
 
 
diff --git a/backend/Dockerfile b/backend/Dockerfile
index dafbcd991..aab567301 100644
--- a/backend/Dockerfile
+++ b/backend/Dockerfile
@@ -138,7 +138,9 @@ COPY python/requirements.txt /app/requirements.txt
 RUN python3 -m pip install --no-cache --upgrade pip setuptools && \
     python3 -m pip install --no-cache dbt-mysql dbt-postgres && \
     python3 -m pip install --no-cache -r requirements.txt && \
-    rm -fr /usr/share/python-wheels/*
+    rm -fr /usr/share/python-wheels/* \
+RUN curl -sSL https://install.python-poetry.org | python3 -
+RUN ln -sf /root/.local/bin/poetry /usr/local/bin
 
 
 FROM base as devlake-base
diff --git a/backend/core/config/config_viper.go b/backend/core/config/config_viper.go
index da3306b4e..85b3da07d 100644
--- a/backend/core/config/config_viper.go
+++ b/backend/core/config/config_viper.go
@@ -19,12 +19,13 @@ package config
 
 import (
 	"fmt"
-	"github.com/apache/incubator-devlake/core/errors"
 	"os"
 	"path/filepath"
 	"regexp"
 	"strings"
 
+	"github.com/apache/incubator-devlake/core/errors"
+
 	goerror "github.com/cockroachdb/errors"
 
 	"github.com/sirupsen/logrus"
@@ -74,6 +75,8 @@ func setDefaultValue(v *viper.Viper) {
 	v.SetDefault("PLUGIN_DIR", "bin/plugins")
 	v.SetDefault("TEMPORAL_TASK_QUEUE", "DEVLAKE_TASK_QUEUE")
 	v.SetDefault("TAP_PROPERTIES_DIR", "resources/tap")
+	v.SetDefault("ENABLE_REMOTE_PLUGINS", "true")
+	v.SetDefault("REMOTE_PLUGINS_STARTUP_PATH", "python/plugins/start.sh")
 }
 
 // replaceNewEnvItemInOldContent replace old config to new config in env file content
diff --git a/backend/core/models/dynamic_tabler.go b/backend/core/models/dynamic_tabler.go
new file mode 100644
index 000000000..b7f92a7e7
--- /dev/null
+++ b/backend/core/models/dynamic_tabler.go
@@ -0,0 +1,75 @@
+/*
+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
+
+    http://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 models
+
+import (
+	"reflect"
+
+	"github.com/apache/incubator-devlake/core/dal"
+)
+
+// DynamicTabler is a core.Tabler that wraps a runtime (anonymously) generated data-model. Due to limitations of
+// reflection in Go and the GORM framework, the underlying model and the table have to be explicitly passed into dal.Dal's API
+// via Unwrap() and TableName()
+type DynamicTabler struct {
+	objType reflect.Type
+	wrapped any
+	table   string
+}
+
+func NewDynamicTabler(tableName string, objType reflect.Type) *DynamicTabler {
+	return &DynamicTabler{
+		objType: objType,
+		table:   tableName,
+	}
+}
+
+func (d *DynamicTabler) New() *DynamicTabler {
+	return &DynamicTabler{
+		objType: d.objType,
+		wrapped: reflect.New(d.objType).Interface(),
+		table:   d.table,
+	}
+}
+
+func (d *DynamicTabler) NewSlice() *DynamicTabler {
+	sliceType := reflect.SliceOf(d.objType)
+	return &DynamicTabler{
+		objType: sliceType,
+		wrapped: reflect.New(sliceType).Interface(),
+		table:   d.table,
+	}
+}
+
+func (d *DynamicTabler) Unwrap() any {
+	return d.wrapped
+}
+
+func (d *DynamicTabler) TableName() string {
+	return d.table
+}
+
+var _ dal.Tabler = (*DynamicTabler)(nil)
+
+// UnwrapObject if the actual object is wrapped in some proxy, it unwinds and returns it, otherwise this is idempotent
+func UnwrapObject(ifc any) any {
+	if dynamic, ok := ifc.(*DynamicTabler); ok {
+		return dynamic.Unwrap()
+	}
+	return ifc
+}
diff --git a/backend/core/utils/ipc.go b/backend/core/utils/ipc.go
index 2c5d5b67e..cbd11c8ba 100644
--- a/backend/core/utils/ipc.go
+++ b/backend/core/utils/ipc.go
@@ -20,97 +20,264 @@ package utils
 import (
 	"bufio"
 	"fmt"
-	"github.com/apache/incubator-devlake/core/errors"
+	"io"
 	"os"
 	"os/exec"
 	"strings"
 	"sync"
+	"syscall"
+
+	"github.com/apache/incubator-devlake/core/errors"
 )
 
+// NoopConverter no-op converter
+var NoopConverter = func(b []byte) (any, errors.Error) {
+	return b, nil
+}
+
 // ProcessResponse wraps output of a process
-type ProcessResponse[T any] struct {
-	Data T
-	Err  error
+type ProcessResponse struct {
+	stdout any
+	stderr any
+	fdOut  any
+	err    errors.Error
 }
 
-// RunProcess runs the cmd and returns its raw standard output. This is a blocking function.
-func RunProcess(cmd *exec.Cmd) (*ProcessResponse[[]byte], error) {
-	cmd.Env = append(cmd.Env, os.Environ()...)
-	stderr, err := cmd.StderrPipe()
+// ProcessStream wraps output of a process
+type ProcessStream struct {
+	receiveChannel <-chan *ProcessResponse
+	process        *os.Process
+	cancelled      bool
+}
+
+// StreamProcessOptions options for streaming a process
+type StreamProcessOptions struct {
+	OnStdout func(b []byte) (any, errors.Error)
+	OnStderr func(b []byte) (any, errors.Error)
+	// UseFdOut if true, it'll open this fd to be used by the child process. Useful to isolate stdout and custom outputs
+	UseFdOut bool
+	OnFdOut  func(b []byte) (any, errors.Error)
+}
+
+// RunProcessOptions options for running a process
+type RunProcessOptions struct {
+	OnStdout func(b []byte)
+	OnStderr func(b []byte)
+	UseFdOut bool
+	OnFdOut  func(b []byte)
+}
+
+type processPipes struct {
+	stdout io.ReadCloser
+	stderr io.ReadCloser
+	fdOut  io.ReadCloser
+}
+
+func (p *processPipes) close() {
+	_ = p.stderr.Close()
+	_ = p.stdout.Close()
+	if p.fdOut != nil {
+		_ = p.fdOut.Close()
+	}
+}
+
+// Receive listens to the process retrieval channel
+func (p *ProcessStream) Receive() <-chan *ProcessResponse {
+	return p.receiveChannel
+}
+
+// Cancel cancels the stream by sending a termination signal to the target.
+func (p *ProcessStream) Cancel() errors.Error {
+	err := errors.Convert(p.process.Signal(syscall.SIGTERM))
+	if err != nil {
+		return err
+	}
+	p.cancelled = true
+	return nil
+}
+
+// GetStdout gets the stdout. The type depends on the conversion defined by StreamProcessOptions.OnStdout, otherwise it is []byte
+func (resp *ProcessResponse) GetStdout() any {
+	return resp.stdout
+}
+
+// GetStderr gets the stderr. The type depends on the conversion defined by StreamProcessOptions.OnStderr, otherwise it is []byte
+func (resp *ProcessResponse) GetStderr() any {
+	return resp.stderr
+}
+
+// GetFdOut gets the custom fd output. The type depends on the conversion defined by StreamProcessOptions.OnFdOut, otherwise it is []byte
+func (resp *ProcessResponse) GetFdOut() any {
+	return resp.fdOut
+}
+
+// GetError gets the error on the response
+func (resp *ProcessResponse) GetError() errors.Error {
+	return resp.err
+}
+
+// RunProcess runs the cmd and blocks until its completion. All returned results will have type []byte.
+func RunProcess(cmd *exec.Cmd, opts *RunProcessOptions) (*ProcessResponse, errors.Error) {
+	stream, err := StreamProcess(cmd, &StreamProcessOptions{
+		OnStdout: func(b []byte) (any, errors.Error) {
+			if opts.OnStdout != nil {
+				opts.OnStdout(b)
+			}
+			return NoopConverter(b)
+		},
+		OnStderr: func(b []byte) (any, errors.Error) {
+			if opts.OnStderr != nil {
+				opts.OnStderr(b)
+			}
+			return NoopConverter(b)
+		},
+		UseFdOut: opts.UseFdOut,
+		OnFdOut: func(b []byte) (any, errors.Error) {
+			if opts.OnFdOut != nil {
+				opts.OnFdOut(b)
+			}
+			return NoopConverter(b)
+		},
+	})
 	if err != nil {
 		return nil, err
 	}
-	remoteErrorMsg := &strings.Builder{}
-	go func() {
-		scanner := bufio.NewScanner(stderr)
-		scanner.Split(bufio.ScanLines)
-		for scanner.Scan() {
-			_, _ = remoteErrorMsg.Write(scanner.Bytes())
-			_, _ = remoteErrorMsg.WriteString("\n")
+	var stdout []byte
+	var stderr []byte
+	var fdOut []byte
+	for result := range stream.Receive() {
+		if result.err != nil {
+			err = result.err
+			break
+		}
+		if result.stdout != nil {
+			stdout = append(stdout, result.stdout.([]byte)...)
+		}
+		if result.stderr != nil {
+			stderr = append(stderr, result.stderr.([]byte)...)
+		}
+		if result.fdOut != nil {
+			fdOut = append(fdOut, result.fdOut.([]byte)...)
 		}
-	}()
-	output, err := cmd.Output()
-	if err != nil {
-		return nil, errors.Default.Wrap(err, fmt.Sprintf("remote error message:\n%s", remoteErrorMsg.String()))
 	}
-	return &ProcessResponse[[]byte]{
-		Data: output,
+	return &ProcessResponse{
+		stdout: stdout,
+		stderr: stderr,
+		fdOut:  fdOut,
+		err:    err,
 	}, nil
 }
 
-// StreamProcess runs the cmd and returns its standard output on a line-by-line basis, on a channel. The converter functor will allow you
+// StreamProcess runs the cmd and returns its output on a line-by-line basis, on a channel. The converter functor will allow you
 // to convert the incoming raw to your custom data type T. This is a nonblocking function.
-func StreamProcess[T any](cmd *exec.Cmd, converter func(b []byte) (T, error)) (<-chan *ProcessResponse[T], error) {
-	cmd.Env = append(cmd.Env, os.Environ()...)
-	stdout, err := cmd.StdoutPipe()
-	if err != nil {
-		return nil, err
+func StreamProcess(cmd *exec.Cmd, opts *StreamProcessOptions) (*ProcessStream, errors.Error) {
+	if opts == nil {
+		opts = &StreamProcessOptions{}
 	}
-	stderr, err := cmd.StderrPipe()
+	cmd.Env = append(cmd.Env, os.Environ()...)
+	pipes, err := getPipes(cmd, opts)
 	if err != nil {
 		return nil, err
 	}
-	if err = cmd.Start(); err != nil {
+	if err = errors.Convert(cmd.Start()); err != nil {
 		return nil, err
 	}
-	stream := make(chan *ProcessResponse[T], 32)
+	receiveStream := make(chan *ProcessResponse, 32)
 	wg := &sync.WaitGroup{}
+	stdScanner := scanOutputPipe(pipes.stdout, wg, opts.OnStdout, func(result any) *ProcessResponse {
+		return &ProcessResponse{stdout: result}
+	}, receiveStream)
+	errScanner, remoteErrorMsg := scanErrorPipe(pipes.stderr, receiveStream)
+	fdOutScanner := scanOutputPipe(pipes.fdOut, wg, opts.OnFdOut, func(result any) *ProcessResponse {
+		return &ProcessResponse{fdOut: result}
+	}, receiveStream)
 	wg.Add(2)
+	if pipes.fdOut != nil {
+		wg.Add(1)
+	}
+	go stdScanner()
+	go errScanner()
+	if pipes.fdOut != nil {
+		go fdOutScanner()
+	}
+	processStream := &ProcessStream{
+		process:        cmd.Process,
+		receiveChannel: receiveStream,
+	}
 	go func() {
-		scanner := bufio.NewScanner(stdout)
+		defer pipes.close()
+		if err = errors.Convert(cmd.Wait()); err != nil {
+			if !processStream.cancelled {
+				receiveStream <- &ProcessResponse{err: errors.Default.Wrap(err, fmt.Sprintf("remote error response:\n%s", remoteErrorMsg))}
+			}
+		}
+		wg.Done()
+	}()
+	go func() {
+		defer close(receiveStream)
+		wg.Wait()
+	}()
+	return processStream, nil
+}
+
+func getPipes(cmd *exec.Cmd, opts *StreamProcessOptions) (*processPipes, errors.Error) {
+	stdout, err := cmd.StdoutPipe()
+	if err != nil {
+		return nil, errors.Convert(err)
+	}
+	stderr, err := cmd.StderrPipe()
+	if err != nil {
+		return nil, errors.Convert(err)
+	}
+	var fdOut *os.File
+	if opts.UseFdOut {
+		fdReader, fdOutWriter, err := os.Pipe()
+		if err != nil {
+			return nil, errors.Convert(err)
+		}
+		cmd.ExtraFiles = []*os.File{fdOutWriter}
+		fdOut = fdReader
+	}
+	return &processPipes{
+		stdout: stdout,
+		stderr: stderr,
+		fdOut:  fdOut,
+	}, nil
+}
+
+func scanOutputPipe(pipe io.ReadCloser, wg *sync.WaitGroup, onReceive func([]byte) (any, errors.Error),
+	responseCreator func(any) *ProcessResponse, outboundChannel chan<- *ProcessResponse) func() {
+	return func() {
+		scanner := bufio.NewScanner(pipe)
 		scanner.Split(bufio.ScanLines)
 		for scanner.Scan() {
 			src := scanner.Bytes()
 			data := make([]byte, len(src))
 			copy(data, src)
-			if result, err := converter(data); err != nil {
-				stream <- &ProcessResponse[T]{Err: err}
+			if result, err := onReceive(data); err != nil {
+				outboundChannel <- &ProcessResponse{err: err}
 			} else {
-				stream <- &ProcessResponse[T]{Data: result}
+				outboundChannel <- responseCreator(result)
 			}
 		}
 		wg.Done()
-	}()
+	}
+}
+
+func scanErrorPipe(pipe io.ReadCloser, outboundChannel chan<- *ProcessResponse) (func(), *strings.Builder) {
 	remoteErrorMsg := &strings.Builder{}
-	go func() {
-		scanner := bufio.NewScanner(stderr)
+	return func() {
+		scanner := bufio.NewScanner(pipe)
 		scanner.Split(bufio.ScanLines)
 		for scanner.Scan() {
-			_, _ = remoteErrorMsg.Write(scanner.Bytes())
+			src := scanner.Bytes()
+			data := make([]byte, len(src))
+			copy(data, src)
+			outboundChannel <- &ProcessResponse{stderr: data}
+			_, _ = remoteErrorMsg.Write(src)
 			_, _ = remoteErrorMsg.WriteString("\n")
 		}
-	}()
-	go func() {
-		if err = cmd.Wait(); err != nil {
-			stream <- &ProcessResponse[T]{Err: errors.Default.Wrap(err, fmt.Sprintf("remote error response:\n%s", remoteErrorMsg))}
-		}
-		wg.Done()
-	}()
-	go func() {
-		defer close(stream)
-		wg.Wait()
-	}()
-	return stream, nil
+	}, remoteErrorMsg
 }
 
 // CreateCmd wraps the args in "sh -c" for shell-level execution
diff --git a/backend/go.mod b/backend/go.mod
index c5f3e5b5b..6605d9227 100644
--- a/backend/go.mod
+++ b/backend/go.mod
@@ -3,6 +3,7 @@ module github.com/apache/incubator-devlake
 go 1.19
 
 require (
+	github.com/RaveNoX/go-jsonmerge v1.0.0
 	github.com/cockroachdb/errors v1.9.0
 	github.com/gin-contrib/cors v1.3.1
 	github.com/gin-gonic/gin v1.7.7
@@ -23,6 +24,7 @@ require (
 	github.com/robfig/cron/v3 v3.0.0
 	github.com/sirupsen/logrus v1.8.1
 	github.com/spf13/afero v1.6.0
+	github.com/spf13/cast v1.4.1
 	github.com/spf13/cobra v1.5.0
 	github.com/spf13/viper v1.8.1
 	github.com/stretchr/testify v1.8.0
@@ -108,7 +110,6 @@ require (
 	github.com/rogpeppe/go-internal v1.8.1 // indirect
 	github.com/russross/blackfriday/v2 v2.1.0 // indirect
 	github.com/sergi/go-diff v1.1.0 // indirect
-	github.com/spf13/cast v1.4.1 // indirect
 	github.com/spf13/jwalterweatherman v1.1.0 // indirect
 	github.com/spf13/pflag v1.0.6-0.20200504143853-81378bbcd8a1 // indirect
 	github.com/stretchr/objx v0.4.0 // indirect
diff --git a/backend/go.sum b/backend/go.sum
index 25f6dca46..5fea80b14 100644
--- a/backend/go.sum
+++ b/backend/go.sum
@@ -58,6 +58,9 @@ github.com/ProtonMail/go-crypto v0.0.0-20210428141323-04723f9f07d7 h1:YoJbenK9C6
 github.com/ProtonMail/go-crypto v0.0.0-20210428141323-04723f9f07d7/go.mod h1:z4/9nQmJSSwwds7ejkxaJwO37dru3geImFUdJlaLzQo=
 github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0=
 github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE=
+github.com/RaveNoX/go-jsoncommentstrip v1.0.0/go.mod h1:78ihd09MekBnJnxpICcwzCMzGrKSKYe4AqU6PDYYpjk=
+github.com/RaveNoX/go-jsonmerge v1.0.0 h1:2e0nqnadoGUP8rAvcA0hkQelZreVO5X3BHomT2XMrAk=
+github.com/RaveNoX/go-jsonmerge v1.0.0/go.mod h1:qYM/NA77LhO4h51JJM7Z+xBU3ovqrNIACZe+SkSNVFo=
 github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0=
 github.com/acomagu/bufpipe v1.0.3 h1:fxAGrHZTgQ9w5QqVItgzwj235/uYZYgbXitB+dLupOk=
 github.com/acomagu/bufpipe v1.0.3/go.mod h1:mxdxdup/WdsKVreO5GpW4+M/1CE2sMG4jeGJ2sYmHc4=
@@ -75,6 +78,7 @@ github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5/go.mod h1:wHh0iHkY
 github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g=
 github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs=
 github.com/bketelsen/crypt v0.0.4/go.mod h1:aI6NrJ0pMGgvZKL1iVgXLnfIFJtfV+bKCoqOes/6LfM=
+github.com/bmatcuk/doublestar v1.1.1/go.mod h1:UD6OnuiIn0yFxxA2le/rnRU1G4RaI4UvFv1sNto9p6w=
 github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU=
 github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
 github.com/chzyer/logex v1.1.10 h1:Swpa1K6QvQznwJRcfTfQJmTE72DqScAa40E+fbHEXEE=
@@ -445,6 +449,7 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X
 github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo=
 github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU=
 github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q=
+github.com/juju/gnuflag v0.0.0-20171113085948-2ce1bb71843d/go.mod h1:2PavIy+JPciBPrBUjwbNvtwB6RQlve+hkpll6QSNmOE=
 github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U=
 github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA=
 github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k=
@@ -655,6 +660,7 @@ github.com/spf13/pflag v1.0.6-0.20200504143853-81378bbcd8a1/go.mod h1:McXfInJRrz
 github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s=
 github.com/spf13/viper v1.8.1 h1:Kq1fyeebqsBfbjZj4EL7gj2IO0mMaiyjYUWcUsl2O44=
 github.com/spf13/viper v1.8.1/go.mod h1:o0Pch8wJ9BVSWGQMbra6iw0oQ5oktSIBaujf1rJH9Ns=
+github.com/spkg/bom v0.0.0-20160624110644-59b7046e48ad/go.mod h1:qLr4V1qq6nMqFKkMo8ZTx3f+BZEkzsRUY10Xsm2mwU0=
 github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
 github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
 github.com/stretchr/objx v0.2.0/go.mod h1:qt09Ya8vawLte6SNmTgCsAVtYtaKzEcn8ATUoHMkEqE=
diff --git a/backend/helpers/pluginhelper/api/connection_helper.go b/backend/helpers/pluginhelper/api/connection_helper.go
index 64760ed95..da825d90c 100644
--- a/backend/helpers/pluginhelper/api/connection_helper.go
+++ b/backend/helpers/pluginhelper/api/connection_helper.go
@@ -18,6 +18,7 @@ limitations under the License.
 package api
 
 import (
+	"github.com/apache/incubator-devlake/core/models"
 	"strconv"
 	"strings"
 
@@ -93,40 +94,33 @@ func (c *ConnectionApiHelper) First(connection interface{}, params map[string]st
 
 // FirstById finds connection from db by id and decrypt it
 func (c *ConnectionApiHelper) FirstById(connection interface{}, id uint64) errors.Error {
-	err := c.db.First(connection, dal.Where("id = ?", id))
-	if err != nil {
-		return err
-	}
-	return nil
+	return CallDB(c.db.First, connection, dal.Where("id = ?", id))
 }
 
 // List returns all connections with password/token decrypted
 func (c *ConnectionApiHelper) List(connections interface{}) errors.Error {
-	err := c.db.All(connections)
-	if err != nil {
-		return err
-	}
-	return nil
+	return CallDB(c.db.All, connections)
 }
 
 // Delete connection
 func (c *ConnectionApiHelper) Delete(connection interface{}) errors.Error {
-	return c.db.Delete(connection)
+	return CallDB(c.db.Delete, connection)
 }
 
 func (c *ConnectionApiHelper) merge(connection interface{}, body map[string]interface{}) errors.Error {
-	if connectionValdiator, ok := connection.(apihelperabstract.ConnectionValidator); ok {
+	connection = models.UnwrapObject(connection)
+	if connectionValidator, ok := connection.(apihelperabstract.ConnectionValidator); ok {
 		err := Decode(body, connection, nil)
 		if err != nil {
 			return err
 		}
-		return connectionValdiator.ValidateConnection(connection, c.validator)
+		return connectionValidator.ValidateConnection(connection, c.validator)
 	}
 	return Decode(body, connection, c.validator)
 }
 
 func (c *ConnectionApiHelper) save(connection interface{}) errors.Error {
-	err := c.db.CreateOrUpdate(connection)
+	err := CallDB(c.db.CreateOrUpdate, connection)
 	if err != nil {
 		if strings.Contains(strings.ToLower(err.Error()), "duplicate") {
 			return errors.BadInput.Wrap(err, "duplicated Connection Name")
diff --git a/backend/helpers/pluginhelper/api/mapstructure.go b/backend/helpers/pluginhelper/api/mapstructure.go
index 9a882e42c..4ecbc9e6f 100644
--- a/backend/helpers/pluginhelper/api/mapstructure.go
+++ b/backend/helpers/pluginhelper/api/mapstructure.go
@@ -19,11 +19,13 @@ package api
 
 import (
 	"encoding/json"
-	"github.com/apache/incubator-devlake/core/errors"
-	"github.com/go-playground/validator/v10"
 	"reflect"
 	"time"
 
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/go-playground/validator/v10"
+
 	"github.com/mitchellh/mapstructure"
 )
 
@@ -79,7 +81,8 @@ func DecodeMapStruct(input map[string]interface{}, result interface{}) errors.Er
 
 // Decode decodes `source` into `target`. Pass an optional validator to validate the target.
 func Decode(source interface{}, target interface{}, vld *validator.Validate) errors.Error {
-	if err := mapstructure.Decode(source, target); err != nil {
+	target = models.UnwrapObject(target)
+	if err := mapstructure.Decode(source, &target); err != nil {
 		return errors.Default.Wrap(err, "error decoding map into target type")
 	}
 	if vld != nil {
diff --git a/backend/helpers/pluginhelper/api/misc_helpers.go b/backend/helpers/pluginhelper/api/misc_helpers.go
new file mode 100644
index 000000000..f5bfb2445
--- /dev/null
+++ b/backend/helpers/pluginhelper/api/misc_helpers.go
@@ -0,0 +1,33 @@
+/*
+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
+
+    http://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 api
+
+import (
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models"
+)
+
+// CallDB wraps DB calls with this signature, and handles the case if the struct is wrapped in a models.DynamicTabler.
+func CallDB(f func(any, ...dal.Clause) errors.Error, x any, clauses ...dal.Clause) errors.Error {
+	if dynamic, ok := x.(*models.DynamicTabler); ok {
+		clauses = append(clauses, dal.From(dynamic.TableName()))
+		x = dynamic.Unwrap()
+	}
+	return f(x, clauses...)
+}
diff --git a/backend/helpers/pluginhelper/tap/singer_tap_impl.go b/backend/helpers/pluginhelper/tap/singer_tap_impl.go
index 98e62b4a4..b86856b74 100644
--- a/backend/helpers/pluginhelper/tap/singer_tap_impl.go
+++ b/backend/helpers/pluginhelper/tap/singer_tap_impl.go
@@ -19,6 +19,7 @@ package tap
 
 import (
 	"bufio"
+	"context"
 	"encoding/json"
 	"github.com/apache/incubator-devlake/core/config"
 	"github.com/apache/incubator-devlake/core/errors"
@@ -127,7 +128,7 @@ func (t *SingerTap) GetName() string {
 }
 
 // Run implements Tap.Run
-func (t *SingerTap) Run() (<-chan *utils.ProcessResponse[Output[json.RawMessage]], errors.Error) {
+func (t *SingerTap) Run(ctx context.Context) (<-chan *Response, errors.Error) {
 	cmd := utils.CreateCmd(
 		t.cmd,
 		"--config",
@@ -136,17 +137,32 @@ func (t *SingerTap) Run() (<-chan *utils.ProcessResponse[Output[json.RawMessage]
 		t.propertiesFile.path,
 		ifElse(t.stateFile.path != "", "--state "+t.stateFile.path, ""),
 	)
-	stream, err := utils.StreamProcess(cmd, func(b []byte) (Output[json.RawMessage], error) {
-		var output Output[json.RawMessage]
-		output, err := NewSingerTapOutput(b)
-		if err != nil {
-			return nil, err
-		}
-		return output, nil //data is expected to be JSON
+	rawStream, err := utils.StreamProcess(cmd, &utils.StreamProcessOptions{
+		OnStdout: func(b []byte) (any, errors.Error) {
+			var output Output[json.RawMessage]
+			output, err := NewSingerTapOutput(b)
+			if err != nil {
+				return nil, err
+			}
+			return output, nil //data is expected to be JSON
+		},
 	})
 	if err != nil {
 		return nil, errors.Default.Wrap(err, "error starting process stream from singer-tap")
 	}
+	stream := make(chan *Response)
+	go func() {
+		defer close(stream)
+		for result := range rawStream.Receive() {
+			if err = result.GetError(); err != nil {
+				stream <- &Response{Err: err}
+			}
+			out := result.GetStdout()
+			if out != nil {
+				stream <- &Response{Out: out.(Output[json.RawMessage])}
+			}
+		}
+	}()
 	return stream, nil
 }
 
diff --git a/backend/helpers/pluginhelper/tap/tap.go b/backend/helpers/pluginhelper/tap/tap.go
index 08fb9b54d..9e94b754a 100644
--- a/backend/helpers/pluginhelper/tap/tap.go
+++ b/backend/helpers/pluginhelper/tap/tap.go
@@ -18,15 +18,17 @@ limitations under the License.
 package tap
 
 import (
+	"context"
 	"encoding/json"
+
 	"github.com/apache/incubator-devlake/core/errors"
-	"github.com/apache/incubator-devlake/core/utils"
 )
 
 // Tap the abstract interface for Taps. Consumer code should not use concrete implementations directly.
 type Tap[Stream any] interface {
-	// Run runs the tap and returns a stream of results. Expected to be called after all the other Setters.
-	Run() (<-chan *utils.ProcessResponse[Output[json.RawMessage]], errors.Error)
+	// Run runs the tap and returns a stream of results. Expected to be called after all the other Setters. The Stdout of the response should
+	// be of type Output[json.RawMessage]
+	Run(ctx context.Context) (<-chan *Response, errors.Error)
 	// GetName the name of this tap
 	GetName() string
 	// SetProperties Sets the properties of the tap and allows you to modify the properties at runtime.
@@ -37,3 +39,8 @@ type Tap[Stream any] interface {
 	// SetConfig sets the config of this tap
 	SetConfig(config any) errors.Error
 }
+
+type Response struct {
+	Out Output[json.RawMessage]
+	Err errors.Error
+}
diff --git a/backend/helpers/pluginhelper/tap/tap_collector.go b/backend/helpers/pluginhelper/tap/tap_collector.go
index 14ab2f53b..37d259261 100644
--- a/backend/helpers/pluginhelper/tap/tap_collector.go
+++ b/backend/helpers/pluginhelper/tap/tap_collector.go
@@ -128,7 +128,7 @@ func (c *Collector[Stream]) Execute() (err errors.Error) {
 			return err
 		}
 	}
-	resultStream, err := c.tapClient.Run()
+	resultStream, err := c.tapClient.Run(c.ctx.GetContext())
 	if err != nil {
 		return err
 	}
@@ -156,7 +156,7 @@ func (c *Collector[Stream]) Execute() (err errors.Error) {
 			return err
 		default:
 		}
-		output := result.Data
+		output := result.Out
 		if tapRecord, ok := output.AsTapRecord(); ok {
 			batchedResults = append(batchedResults, tapRecord.Record)
 			c.ctx.IncProgress(1)
diff --git a/backend/impls/dalgorm/encdec_serializer.go b/backend/impls/dalgorm/encdec_serializer.go
index af3011107..ca140675c 100644
--- a/backend/impls/dalgorm/encdec_serializer.go
+++ b/backend/impls/dalgorm/encdec_serializer.go
@@ -21,9 +21,11 @@ import (
 	"context"
 	"encoding/json"
 	"fmt"
-	"github.com/apache/incubator-devlake/core/plugin"
 	"reflect"
 
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+
 	"gorm.io/gorm/schema"
 )
 
@@ -37,6 +39,7 @@ type EncDecSerializer struct {
 
 // Scan implements serializer interface
 func (es *EncDecSerializer) Scan(ctx context.Context, field *schema.Field, dst reflect.Value, dbValue interface{}) (err error) {
+	dbValue = models.UnwrapObject(dbValue)
 	fieldValue := reflect.New(field.FieldType)
 	if dbValue != nil {
 		var base64str string
diff --git a/.licenserc.yaml b/backend/python/build.sh
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/build.sh
index b11fd8254..f8d9a15d7 100644
--- a/.licenserc.yaml
+++ b/backend/python/build.sh
@@ -1,3 +1,5 @@
+#!/bin/sh
+#
 # 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.
@@ -13,46 +15,19 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
 
+cd "${0%/*}" # make sure we're in the correct dir
 
+poetry config virtualenvs.create true
 
-  comment: on-failure
+for plugin_dir in $(ls -d plugins/*/*.toml); do
+  plugin_dir=$(dirname $plugin_dir)
+  echo "installing dependencies of python plugin in: $plugin_dir" &&\
+  cd "$plugin_dir" &&\
+  poetry install &&\
+  cd -
+  exit_code=$?
+  if [ $exit_code != 0 ]; then
+    exit $exit_code
+  fi
+done
\ No newline at end of file
diff --git a/.licenserc.yaml b/backend/python/plugins/start.sh
old mode 100644
new mode 100755
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/plugins/start.sh
index b11fd8254..f5cdd9cf2
--- a/.licenserc.yaml
+++ b/backend/python/plugins/start.sh
@@ -1,3 +1,5 @@
+#!/bin/sh
+#
 # 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.
@@ -13,46 +15,18 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
 
+endpoint="$1"
 
+cd "${0%/*}" # make sure we're in the correct dir
 
-  comment: on-failure
+for plugin_dir in $(ls -d */*.toml); do
+  plugin_dir=$(dirname $plugin_dir)
+  cd $plugin_dir &&\
+  poetry run python $plugin_dir/main.py startup "$endpoint" &&\
+  cd -
+  exit_code=$?
+  if [ $exit_code != 0 ]; then
+    exit $exit_code
+  fi
+done
\ No newline at end of file
diff --git a/backend/python/pydevlake/README.md b/backend/python/pydevlake/README.md
new file mode 100644
index 000000000..ef675d307
--- /dev/null
+++ b/backend/python/pydevlake/README.md
@@ -0,0 +1,334 @@
+# Pydevlake
+
+A framework to write data collection plugins for [DevLake](https://devlake.apache.org/).
+
+
+# How to create a new plugin
+
+## Create plugin project
+
+
+Make sure you have [Poetry](https://python-poetry.org/docs/#installation) installed.
+Move to `python/plugins` and execute `poetry new myplugin`.
+This will generate a new directory for your plugin.
+
+In the `pyproject.toml` file and add the following line at the end of the `[tool.poetry.dependencies]` section:
+```
+pydevlake = { path = "../../pydevlake", develop = false }
+```
+
+Now run `poetry install`.
+
+## Create `main` file
+
+Create a `main.py` file with the following content:
+
+```python
+from pydevlake import Plugin, Connection
+
+
+class MyPluginConnection(Connection):
+    pass
+
+
+class MyPlugin(Plugin):
+    @property
+    def connection_type(self):
+        return MyPluginConnection
+
+    def test_connection(self, connection: MyPluginConnection):
+        pass
+
+    @property
+    def streams(self):
+        return []
+
+
+if __name__ == '__main__':
+    MyPlugin.start()
+```
+
+This file is the entry point to your plugin.
+It specifies three things:
+- the parameters that your plugin needs to collect data, e.g. the url and credentials to connect to the datasource or custom options
+- how to validate that some given parameters allows to connect to the datasource, e.g. test whether the credentials are correct
+- the list of data streams that this plugin can collect
+
+
+### Connection parameters
+
+The parameters of your plugin are defined as class attributes of the connection class.
+For example to add a `url` parameter of type `str` edit `MyPLuginConnection` as follow:
+
+```python
+class MyPluginConnection(Connection):
+    url: str
+```
+
+
+## Add a new data stream
+
+A data stream groups the logic for:
+- collecting the raw data from the datasource
+- extracting this raw data into a tool-specific model
+- converting the tool model into an equivalent [DevLake domain model](https://devlake.apache.org/docs/next/DataModels/DevLakeDomainLayerSchema)
+
+
+### Create a tool model
+
+Create a `models.py` file.
+Then create a class that modelizes the data your stream is going to collect.
+
+```python
+from pydevlake.model import ToolModel
+
+class User(ToolModel, table=True):
+    id: str = Field(primary_key=True)
+    name: str
+    email: str
+```
+
+Your tool model must declare at least one attribute as a primary key, like `id` in the example above.
+It must inherit from `ToolModel`, which in turn inherit from `SQLModel`, the base class of an [ORM of the same name](https://sqlmodel.tiangolo.com/).
+You can use `SQLModel` features like [declaring relationships with other models](https://sqlmodel.tiangolo.com/tutorial/relationship-attributes/).
+
+
+### Create the stream class
+
+Create a new file for your first stream in a `streams` directory.
+
+```python
+from pydevlake import Stream
+from pydevlake.domain_layer.crossdomain import User as DomainUser
+
+from myplugin.models import User as ToolUser
+
+
+class Users(Stream):
+    tool_model = ToolUser
+    domain_model = DomainUser
+
+    def collect(self, state, context) -> Iterable[Tuple[object, dict]]:
+        pass
+
+    def convert(self, user: ToolUser) -> Iterable[DomainUser]:
+        pass
+```
+
+This stream will collect raw user data, e.g. as parsed JSON objects, extract this raw data as your
+tool-specific user model, then convert them into domain-layer user models.
+
+It is possible to have a stream that produce several domain models of different types from a single tool model. In that case, declare the list of possible domain model types in a class attribute `domain_models` (plural) instead of a single domain model type in the `domain_model` (singular) class attribute.
+
+The `collect` method takes a `state` dictionary and a context object and yields tuples of raw data and new state.
+The last state that the plugin yielded for a given connection will be reused during the next collection.
+The plugin can use this `state` to store information necessary to perform incremental collection of data.
+
+
+The `convert` method takes a tool-specific user model and convert it into domain level user models.
+Here the two models align quite well, the conversion is trivial:
+
+```python
+def convert(self, user: ToolUser) -> Iterable[DomainUser]:
+    yield DomainUser(
+        id=user.id,
+        name=user.name
+        email=user.email
+    )
+```
+
+### Create an API wrapper
+
+Lets assume that your datasource is a REST API.
+We can create the following class to define it.
+
+```python
+from pydevlake.api import API
+
+
+class MyAPI(API):
+    def __init__(self, url: str):
+        self.url = url
+
+    def users(self):
+        return self.get(f'{self.url}/users')
+```
+
+By inheriting `API` you get access to facilities to wrap REST APIs.
+Here the `users` method will return a `Response` object that contains the results of calling `GET` on `<url>/users`.
+
+Now we can go back to our stream file and implement `collect`:
+
+```python
+from myplugin.api import MyAPI
+
+...
+
+    def collect(self, state, context) -> Iterable[Tuple[object, dict]]:
+        api = MyAPI(context.connection.url)
+        for user in api.users().json():
+            yield user, state
+
+...
+```
+
+If the API responds with a list of JSON object with properties matching your `User` model attributes, you're done!.
+Indeed extraction has a default implementation that takes of this common case.
+This is why it is important to make tool models that align with the data you collect.
+
+If this is not the case, e.g. the attribute case mismatch, you can redefine the `extract` method:
+
+```python
+...
+
+class Users(Stream):
+    ...
+
+    def extract(self, raw_data: dict) -> ToolModel:
+        return ToolUser(
+            id=raw_data["ID"],
+            name=raw_data["Name"],
+            email=raw_data["Email"]
+        )
+
+    ...
+```
+
+
+#### Request and response hook
+
+For each request sent and response received by your API wrapper,
+you can register hooks. Hooks allows you to implement 
+authentication, pagination, and generic API error handling.
+
+For example, lets assume that we are dealing with an API that
+require user to authenticate via a token set in a request header.
+
+A request hook is declared in your API with a `@request_hook` decorator.
+
+```python
+...
+class MyAPI(API):
+    def __init__(self, url, token):
+        self.url = url
+        self.token = token
+
+    ...
+    @request_hook
+    def authenticate(self, request):
+        if self.token:
+            request.headers['Token'] = self.token
+    ...
+```
+
+Here the method `authenticate` is a hook that is run on each request.
+Similarly you can declare response hooks with `@response_hook`.
+
+#### Pagination
+
+One usage of a response hook is for handling paginated results.
+A response hook can be used to wrap the `Response` object in a
+`PagedResponse` object that support iteration and fetching other pages.
+This response hook is actually defined in `API` base class and expect
+your API wrapper to declare a `paginator` property.
+
+You can subclass `Paginator` to provide API specific logic or reuse an
+existing implementation such as `TokenPaginator`.
+A token paginator assumes the API paginated responses are JSON object with one
+property that is an array of items and another that contains the token to the next
+page.
+
+For example, the following paginator fetch items from the `'results'` attribute,
+the next page from the `'nextPage'` attribute and will issue requests with a `page`
+query parameter.
+
+```
+...
+class MyAPI(API):
+    ...
+    paginator = TokenPaginator(
+        items_attr='results',
+        next_page_token_attr='nextPage',
+        next_page_token_param='page'
+    )
+    ...
+```
+
+## Substreams
+
+With REST APIs, you often need to fetch a stream of items, and then to collect additional
+data for each of those items.
+
+For example you might want to collect all `UserComments` written by each user collected via the `Users` stream.
+
+To handle such cases, `UserComments` would inherit from `Substream` instead of `Stream`.
+A substream needs to specify which is his parent stream. The `collect` method
+of a substream will be called with each item collected from the parent stream.
+
+```python
+...
+from pydevlake import Substream
+from myplugin.streams.users import Users
+
+class UserComments(Substream):
+    parent_stream = Users # Must specify the parent stream
+    ...
+    def collect(self, state: dict, context, user: User):
+        """
+        This method will be called for each user collected from parent stream Users. 
+        """
+        for json in MyPluginAPI(context.connection.token).user_comments(user.id):
+            yield json, state
+    ...
+```
+
+
+# Test the plugin standalone
+
+To test your plugin manually, you can run your `main.py` file with different commands.
+You can find all those commands with `--help` cli flag:
+
+```console
+poetry run myplugin/main.py --help
+```
+
+For testing, the interesting commands are `collect`/`extract`/`convert`.
+Each takes a context and a stream name.
+The context is a JSON object that must at least contain:
+- a `connection_id`
+- a `db_url`, e.g. you can use `"sqlite+pysqlite:///:memory:"` for an in-memory DB
+- a `connection` object containing the same attributes than your plugin connection type
+
+Also, python plugins communicate with go side over an extra file descriptor 3, so you should
+redirect to stdout when testing your plugin.
+
+```
+console
+CTX='{"connection_id": "1", "db_url":"sqlite+pysqlite:///:memory:", "connection": {...your connection attrs here...}}'
+poetry run myplugin/main.py $CTX users 3>&1
+```
+
+
+# Test the plugin with DevLake
+
+To test your plugin together with DevLake, you first need to create a connection for your plugin and get its id.
+One easy way to do that is to run DevLake with `make dev` and then to create the connection with a POST
+request to your plugin connection API:
+
+```console
+curl -X 'POST' \
+  'http://localhost:8080/plugins/myplugin/connections' \
+  -d '{...connection JSON object...}'
+```
+
+You should get the created connection with his id (which is 1 for the first created connection) in the response.
+
+Now that a connection for your plugin exists in DevLake database, we can try to run your plugin using `backend/server/services/remote/run/run.go` script: 
+
+```console
+cd backend
+go run server/services/remote/run/run.go  -c 1 -p python/plugins/myplugin/myplugin/main.py
+```
+
+This script takes a connection id (`-c` flag) and the path to your plugin `main.py` file (`-p` flag).
+You can also send options as a JSON object (`-o` flag).
diff --git a/backend/python/pydevlake/poetry.lock b/backend/python/pydevlake/poetry.lock
new file mode 100644
index 000000000..7c89577ec
--- /dev/null
+++ b/backend/python/pydevlake/poetry.lock
@@ -0,0 +1,624 @@
+# This file is automatically @generated by Poetry and should not be changed by hand.
+
+[[package]]
+name = "attrs"
+version = "22.2.0"
+description = "Classes Without Boilerplate"
+category = "dev"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "attrs-22.2.0-py3-none-any.whl", hash = "sha256:29e95c7f6778868dbd49170f98f8818f78f3dc5e0e37c0b1f474e3561b240836"},
+    {file = "attrs-22.2.0.tar.gz", hash = "sha256:c9227bfc2f01993c03f68db37d1d15c9690188323c067c641f1a35ca58185f99"},
+]
+
+[package.extras]
+cov = ["attrs[tests]", "coverage-enable-subprocess", "coverage[toml] (>=5.3)"]
+dev = ["attrs[docs,tests]"]
+docs = ["furo", "myst-parser", "sphinx", "sphinx-notfound-page", "sphinxcontrib-towncrier", "towncrier", "zope.interface"]
+tests = ["attrs[tests-no-zope]", "zope.interface"]
+tests-no-zope = ["cloudpickle", "cloudpickle", "hypothesis", "hypothesis", "mypy (>=0.971,<0.990)", "mypy (>=0.971,<0.990)", "pympler", "pympler", "pytest (>=4.3.0)", "pytest (>=4.3.0)", "pytest-mypy-plugins", "pytest-mypy-plugins", "pytest-xdist[psutil]", "pytest-xdist[psutil]"]
+
+[[package]]
+name = "certifi"
+version = "2022.12.7"
+description = "Python package for providing Mozilla's CA Bundle."
+category = "main"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "certifi-2022.12.7-py3-none-any.whl", hash = "sha256:4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18"},
+    {file = "certifi-2022.12.7.tar.gz", hash = "sha256:35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3"},
+]
+
+[[package]]
+name = "charset-normalizer"
+version = "3.0.1"
+description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet."
+category = "main"
+optional = false
+python-versions = "*"
+files = [
+    {file = "charset-normalizer-3.0.1.tar.gz", hash = "sha256:ebea339af930f8ca5d7a699b921106c6e29c617fe9606fa7baa043c1cdae326f"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:88600c72ef7587fe1708fd242b385b6ed4b8904976d5da0893e31df8b3480cb6"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c75ffc45f25324e68ab238cb4b5c0a38cd1c3d7f1fb1f72b5541de469e2247db"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:db72b07027db150f468fbada4d85b3b2729a3db39178abf5c543b784c1254539"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62595ab75873d50d57323a91dd03e6966eb79c41fa834b7a1661ed043b2d404d"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ff6f3db31555657f3163b15a6b7c6938d08df7adbfc9dd13d9d19edad678f1e8"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:772b87914ff1152b92a197ef4ea40efe27a378606c39446ded52c8f80f79702e"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:70990b9c51340e4044cfc394a81f614f3f90d41397104d226f21e66de668730d"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:292d5e8ba896bbfd6334b096e34bffb56161c81408d6d036a7dfa6929cff8783"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:2edb64ee7bf1ed524a1da60cdcd2e1f6e2b4f66ef7c077680739f1641f62f555"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:31a9ddf4718d10ae04d9b18801bd776693487cbb57d74cc3458a7673f6f34639"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:44ba614de5361b3e5278e1241fda3dc1838deed864b50a10d7ce92983797fa76"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:12db3b2c533c23ab812c2b25934f60383361f8a376ae272665f8e48b88e8e1c6"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c512accbd6ff0270939b9ac214b84fb5ada5f0409c44298361b2f5e13f9aed9e"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-win32.whl", hash = "sha256:502218f52498a36d6bf5ea77081844017bf7982cdbe521ad85e64cabee1b608b"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:601f36512f9e28f029d9481bdaf8e89e5148ac5d89cffd3b05cd533eeb423b59"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:0298eafff88c99982a4cf66ba2efa1128e4ddaca0b05eec4c456bbc7db691d8d"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a8d0fc946c784ff7f7c3742310cc8a57c5c6dc31631269876a88b809dbeff3d3"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:87701167f2a5c930b403e9756fab1d31d4d4da52856143b609e30a1ce7160f3c"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:14e76c0f23218b8f46c4d87018ca2e441535aed3632ca134b10239dfb6dadd6b"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0c0a590235ccd933d9892c627dec5bc7511ce6ad6c1011fdf5b11363022746c1"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8c7fe7afa480e3e82eed58e0ca89f751cd14d767638e2550c77a92a9e749c317"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:79909e27e8e4fcc9db4addea88aa63f6423ebb171db091fb4373e3312cb6d603"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8ac7b6a045b814cf0c47f3623d21ebd88b3e8cf216a14790b455ea7ff0135d18"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:72966d1b297c741541ca8cf1223ff262a6febe52481af742036a0b296e35fa5a"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:f9d0c5c045a3ca9bedfc35dca8526798eb91a07aa7a2c0fee134c6c6f321cbd7"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:5995f0164fa7df59db4746112fec3f49c461dd6b31b841873443bdb077c13cfc"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:4a8fcf28c05c1f6d7e177a9a46a1c52798bfe2ad80681d275b10dcf317deaf0b"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:761e8904c07ad053d285670f36dd94e1b6ab7f16ce62b9805c475b7aa1cffde6"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-win32.whl", hash = "sha256:71140351489970dfe5e60fc621ada3e0f41104a5eddaca47a7acb3c1b851d6d3"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:9ab77acb98eba3fd2a85cd160851816bfce6871d944d885febf012713f06659c"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:84c3990934bae40ea69a82034912ffe5a62c60bbf6ec5bc9691419641d7d5c9a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:74292fc76c905c0ef095fe11e188a32ebd03bc38f3f3e9bcb85e4e6db177b7ea"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c95a03c79bbe30eec3ec2b7f076074f4281526724c8685a42872974ef4d36b72"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f4c39b0e3eac288fedc2b43055cfc2ca7a60362d0e5e87a637beac5d801ef478"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:df2c707231459e8a4028eabcd3cfc827befd635b3ef72eada84ab13b52e1574d"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93ad6d87ac18e2a90b0fe89df7c65263b9a99a0eb98f0a3d2e079f12a0735837"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:59e5686dd847347e55dffcc191a96622f016bc0ad89105e24c14e0d6305acbc6"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:cd6056167405314a4dc3c173943f11249fa0f1b204f8b51ed4bde1a9cd1834dc"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_ppc64le.whl", hash = "sha256:083c8d17153ecb403e5e1eb76a7ef4babfc2c48d58899c98fcaa04833e7a2f9a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_s390x.whl", hash = "sha256:f5057856d21e7586765171eac8b9fc3f7d44ef39425f85dbcccb13b3ebea806c"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:7eb33a30d75562222b64f569c642ff3dc6689e09adda43a082208397f016c39a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-win32.whl", hash = "sha256:95dea361dd73757c6f1c0a1480ac499952c16ac83f7f5f4f84f0658a01b8ef41"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:eaa379fcd227ca235d04152ca6704c7cb55564116f8bc52545ff357628e10602"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:3e45867f1f2ab0711d60c6c71746ac53537f1684baa699f4f668d4c6f6ce8e14"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cadaeaba78750d58d3cc6ac4d1fd867da6fc73c88156b7a3212a3cd4819d679d"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:911d8a40b2bef5b8bbae2e36a0b103f142ac53557ab421dc16ac4aafee6f53dc"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:503e65837c71b875ecdd733877d852adbc465bd82c768a067badd953bf1bc5a3"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a60332922359f920193b1d4826953c507a877b523b2395ad7bc716ddd386d866"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:16a8663d6e281208d78806dbe14ee9903715361cf81f6d4309944e4d1e59ac5b"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:a16418ecf1329f71df119e8a65f3aa68004a3f9383821edcb20f0702934d8087"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:9d9153257a3f70d5f69edf2325357251ed20f772b12e593f3b3377b5f78e7ef8"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:02a51034802cbf38db3f89c66fb5d2ec57e6fe7ef2f4a44d070a593c3688667b"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:2e396d70bc4ef5325b72b593a72c8979999aa52fb8bcf03f701c1b03e1166918"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:11b53acf2411c3b09e6af37e4b9005cba376c872503c8f28218c7243582df45d"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-win32.whl", hash = "sha256:0bf2dae5291758b6f84cf923bfaa285632816007db0330002fa1de38bfcb7154"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:2c03cc56021a4bd59be889c2b9257dae13bf55041a3372d3295416f86b295fb5"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:024e606be3ed92216e2b6952ed859d86b4cfa52cd5bc5f050e7dc28f9b43ec42"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:4b0d02d7102dd0f997580b51edc4cebcf2ab6397a7edf89f1c73b586c614272c"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:358a7c4cb8ba9b46c453b1dd8d9e431452d5249072e4f56cfda3149f6ab1405e"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:81d6741ab457d14fdedc215516665050f3822d3e56508921cc7239f8c8e66a58"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8b8af03d2e37866d023ad0ddea594edefc31e827fee64f8de5611a1dbc373174"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9cf4e8ad252f7c38dd1f676b46514f92dc0ebeb0db5552f5f403509705e24753"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e696f0dd336161fca9adbb846875d40752e6eba585843c768935ba5c9960722b"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c22d3fe05ce11d3671297dc8973267daa0f938b93ec716e12e0f6dee81591dc1"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:109487860ef6a328f3eec66f2bf78b0b72400280d8f8ea05f69c51644ba6521a"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:37f8febc8ec50c14f3ec9637505f28e58d4f66752207ea177c1d67df25da5aed"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:f97e83fa6c25693c7a35de154681fcc257c1c41b38beb0304b9c4d2d9e164479"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:a152f5f33d64a6be73f1d30c9cc82dfc73cec6477ec268e7c6e4c7d23c2d2291"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:39049da0ffb96c8cbb65cbf5c5f3ca3168990adf3551bd1dee10c48fce8ae820"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-win32.whl", hash = "sha256:4457ea6774b5611f4bed5eaa5df55f70abde42364d498c5134b7ef4c6958e20e"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:e62164b50f84e20601c1ff8eb55620d2ad25fb81b59e3cd776a1902527a788af"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8eade758719add78ec36dc13201483f8e9b5d940329285edcd5f70c0a9edbd7f"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8499ca8f4502af841f68135133d8258f7b32a53a1d594aa98cc52013fff55678"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3fc1c4a2ffd64890aebdb3f97e1278b0cc72579a08ca4de8cd2c04799a3a22be"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:00d3ffdaafe92a5dc603cb9bd5111aaa36dfa187c8285c543be562e61b755f6b"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c2ac1b08635a8cd4e0cbeaf6f5e922085908d48eb05d44c5ae9eabab148512ca"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f6f45710b4459401609ebebdbcfb34515da4fc2aa886f95107f556ac69a9147e"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3ae1de54a77dc0d6d5fcf623290af4266412a7c4be0b1ff7444394f03f5c54e3"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3b590df687e3c5ee0deef9fc8c547d81986d9a1b56073d82de008744452d6541"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ab5de034a886f616a5668aa5d098af2b5385ed70142090e2a31bcbd0af0fdb3d"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:9cb3032517f1627cc012dbc80a8ec976ae76d93ea2b5feaa9d2a5b8882597579"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:608862a7bf6957f2333fc54ab4399e405baad0163dc9f8d99cb236816db169d4"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:0f438ae3532723fb6ead77e7c604be7c8374094ef4ee2c5e03a3a17f1fca256c"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:356541bf4381fa35856dafa6a965916e54bed415ad8a24ee6de6e37deccf2786"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-win32.whl", hash = "sha256:39cf9ed17fe3b1bc81f33c9ceb6ce67683ee7526e65fde1447c772afc54a1bb8"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:0a11e971ed097d24c534c037d298ad32c6ce81a45736d31e0ff0ad37ab437d59"},
+    {file = "charset_normalizer-3.0.1-py3-none-any.whl", hash = "sha256:7e189e2e1d3ed2f4aebabd2d5b0f931e883676e51c7624826e0a4e5fe8a0bf24"},
+]
+
+[[package]]
+name = "colorama"
+version = "0.4.6"
+description = "Cross-platform colored terminal text."
+category = "dev"
+optional = false
+python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,!=3.6.*,>=2.7"
+files = [
+    {file = "colorama-0.4.6-py2.py3-none-any.whl", hash = "sha256:4f1d9991f5acc0ca119f9d443620b77f9d6b33703e51011c16baf57afb285fc6"},
+    {file = "colorama-0.4.6.tar.gz", hash = "sha256:08695f5cb7ed6e0531a20572697297273c47b8cae5a63ffc6d6ed5c201be6e44"},
+]
+
+[[package]]
+name = "exceptiongroup"
+version = "1.1.0"
+description = "Backport of PEP 654 (exception groups)"
+category = "dev"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "exceptiongroup-1.1.0-py3-none-any.whl", hash = "sha256:327cbda3da756e2de031a3107b81ab7b3770a602c4d16ca618298c526f4bec1e"},
+    {file = "exceptiongroup-1.1.0.tar.gz", hash = "sha256:bcb67d800a4497e1b404c2dd44fca47d3b7a5e5433dbab67f96c1a685cdfdf23"},
+]
+
+[package.extras]
+test = ["pytest (>=6)"]
+
+[[package]]
+name = "fire"
+version = "0.4.0"
+description = "A library for automatically generating command line interfaces."
+category = "main"
+optional = false
+python-versions = "*"
+files = [
+    {file = "fire-0.4.0.tar.gz", hash = "sha256:c5e2b8763699d1142393a46d0e3e790c5eb2f0706082df8f647878842c216a62"},
+]
+
+[package.dependencies]
+six = "*"
+termcolor = "*"
+
+[[package]]
+name = "greenlet"
+version = "2.0.2"
+description = "Lightweight in-process concurrent programming"
+category = "main"
+optional = false
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
+files = [
+    {file = "greenlet-2.0.2-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:bdfea8c661e80d3c1c99ad7c3ff74e6e87184895bbaca6ee8cc61209f8b9b85d"},
+    {file = "greenlet-2.0.2-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:9d14b83fab60d5e8abe587d51c75b252bcc21683f24699ada8fb275d7712f5a9"},
+    {file = "greenlet-2.0.2-cp27-cp27m-win32.whl", hash = "sha256:6c3acb79b0bfd4fe733dff8bc62695283b57949ebcca05ae5c129eb606ff2d74"},
+    {file = "greenlet-2.0.2-cp27-cp27m-win_amd64.whl", hash = "sha256:283737e0da3f08bd637b5ad058507e578dd462db259f7f6e4c5c365ba4ee9343"},
+    {file = "greenlet-2.0.2-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:d27ec7509b9c18b6d73f2f5ede2622441de812e7b1a80bbd446cb0633bd3d5ae"},
+    {file = "greenlet-2.0.2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:30bcf80dda7f15ac77ba5af2b961bdd9dbc77fd4ac6105cee85b0d0a5fcf74df"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26fbfce90728d82bc9e6c38ea4d038cba20b7faf8a0ca53a9c07b67318d46088"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9190f09060ea4debddd24665d6804b995a9c122ef5917ab26e1566dcc712ceeb"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d75209eed723105f9596807495d58d10b3470fa6732dd6756595e89925ce2470"},
+    {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:3a51c9751078733d88e013587b108f1b7a1fb106d402fb390740f002b6f6551a"},
+    {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:76ae285c8104046b3a7f06b42f29c7b73f77683df18c49ab5af7983994c2dd91"},
+    {file = "greenlet-2.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:2d4686f195e32d36b4d7cf2d166857dbd0ee9f3d20ae349b6bf8afc8485b3645"},
+    {file = "greenlet-2.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:c4302695ad8027363e96311df24ee28978162cdcdd2006476c43970b384a244c"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c48f54ef8e05f04d6eff74b8233f6063cb1ed960243eacc474ee73a2ea8573ca"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a1846f1b999e78e13837c93c778dcfc3365902cfb8d1bdb7dd73ead37059f0d0"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a06ad5312349fec0ab944664b01d26f8d1f05009566339ac6f63f56589bc1a2"},
+    {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:eff4eb9b7eb3e4d0cae3d28c283dc16d9bed6b193c2e1ace3ed86ce48ea8df19"},
+    {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5454276c07d27a740c5892f4907c86327b632127dd9abec42ee62e12427ff7e3"},
+    {file = "greenlet-2.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:7cafd1208fdbe93b67c7086876f061f660cfddc44f404279c1585bbf3cdc64c5"},
+    {file = "greenlet-2.0.2-cp35-cp35m-macosx_10_14_x86_64.whl", hash = "sha256:910841381caba4f744a44bf81bfd573c94e10b3045ee00de0cbf436fe50673a6"},
+    {file = "greenlet-2.0.2-cp35-cp35m-manylinux2010_x86_64.whl", hash = "sha256:18a7f18b82b52ee85322d7a7874e676f34ab319b9f8cce5de06067384aa8ff43"},
+    {file = "greenlet-2.0.2-cp35-cp35m-win32.whl", hash = "sha256:03a8f4f3430c3b3ff8d10a2a86028c660355ab637cee9333d63d66b56f09d52a"},
+    {file = "greenlet-2.0.2-cp35-cp35m-win_amd64.whl", hash = "sha256:4b58adb399c4d61d912c4c331984d60eb66565175cdf4a34792cd9600f21b394"},
+    {file = "greenlet-2.0.2-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:703f18f3fda276b9a916f0934d2fb6d989bf0b4fb5a64825260eb9bfd52d78f0"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux2010_x86_64.whl", hash = "sha256:32e5b64b148966d9cccc2c8d35a671409e45f195864560829f395a54226408d3"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2dd11f291565a81d71dab10b7033395b7a3a5456e637cf997a6f33ebdf06f8db"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e0f72c9ddb8cd28532185f54cc1453f2c16fb417a08b53a855c4e6a418edd099"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cd021c754b162c0fb55ad5d6b9d960db667faad0fa2ff25bb6e1301b0b6e6a75"},
+    {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:3c9b12575734155d0c09d6c3e10dbd81665d5c18e1a7c6597df72fd05990c8cf"},
+    {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:b9ec052b06a0524f0e35bd8790686a1da006bd911dd1ef7d50b77bfbad74e292"},
+    {file = "greenlet-2.0.2-cp36-cp36m-win32.whl", hash = "sha256:dbfcfc0218093a19c252ca8eb9aee3d29cfdcb586df21049b9d777fd32c14fd9"},
+    {file = "greenlet-2.0.2-cp36-cp36m-win_amd64.whl", hash = "sha256:9f35ec95538f50292f6d8f2c9c9f8a3c6540bbfec21c9e5b4b751e0a7c20864f"},
+    {file = "greenlet-2.0.2-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:d5508f0b173e6aa47273bdc0a0b5ba055b59662ba7c7ee5119528f466585526b"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux2010_x86_64.whl", hash = "sha256:f82d4d717d8ef19188687aa32b8363e96062911e63ba22a0cff7802a8e58e5f1"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9c59a2120b55788e800d82dfa99b9e156ff8f2227f07c5e3012a45a399620b7"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2780572ec463d44c1d3ae850239508dbeb9fed38e294c68d19a24d925d9223ca"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:937e9020b514ceedb9c830c55d5c9872abc90f4b5862f89c0887033ae33c6f73"},
+    {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:36abbf031e1c0f79dd5d596bfaf8e921c41df2bdf54ee1eed921ce1f52999a86"},
+    {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:18e98fb3de7dba1c0a852731c3070cf022d14f0d68b4c87a19cc1016f3bb8b33"},
+    {file = "greenlet-2.0.2-cp37-cp37m-win32.whl", hash = "sha256:3f6ea9bd35eb450837a3d80e77b517ea5bc56b4647f5502cd28de13675ee12f7"},
+    {file = "greenlet-2.0.2-cp37-cp37m-win_amd64.whl", hash = "sha256:7492e2b7bd7c9b9916388d9df23fa49d9b88ac0640db0a5b4ecc2b653bf451e3"},
+    {file = "greenlet-2.0.2-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:b864ba53912b6c3ab6bcb2beb19f19edd01a6bfcbdfe1f37ddd1778abfe75a30"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux2010_x86_64.whl", hash = "sha256:ba2956617f1c42598a308a84c6cf021a90ff3862eddafd20c3333d50f0edb45b"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc3a569657468b6f3fb60587e48356fe512c1754ca05a564f11366ac9e306526"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8eab883b3b2a38cc1e050819ef06a7e6344d4a990d24d45bc6f2cf959045a45b"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:acd2162a36d3de67ee896c43effcd5ee3de247eb00354db411feb025aa319857"},
+    {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:0bf60faf0bc2468089bdc5edd10555bab6e85152191df713e2ab1fcc86382b5a"},
+    {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b0ef99cdbe2b682b9ccbb964743a6aca37905fda5e0452e5ee239b1654d37f2a"},
+    {file = "greenlet-2.0.2-cp38-cp38-win32.whl", hash = "sha256:b80f600eddddce72320dbbc8e3784d16bd3fb7b517e82476d8da921f27d4b249"},
+    {file = "greenlet-2.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:4d2e11331fc0c02b6e84b0d28ece3a36e0548ee1a1ce9ddde03752d9b79bba40"},
+    {file = "greenlet-2.0.2-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:88d9ab96491d38a5ab7c56dd7a3cc37d83336ecc564e4e8816dbed12e5aaefc8"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux2010_x86_64.whl", hash = "sha256:561091a7be172ab497a3527602d467e2b3fbe75f9e783d8b8ce403fa414f71a6"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:971ce5e14dc5e73715755d0ca2975ac88cfdaefcaab078a284fea6cfabf866df"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be4ed120b52ae4d974aa40215fcdfde9194d63541c7ded40ee12eb4dda57b76b"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94c817e84245513926588caf1152e3b559ff794d505555211ca041f032abbb6b"},
+    {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:1a819eef4b0e0b96bb0d98d797bef17dc1b4a10e8d7446be32d1da33e095dbb8"},
+    {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:7efde645ca1cc441d6dc4b48c0f7101e8d86b54c8530141b09fd31cef5149ec9"},
+    {file = "greenlet-2.0.2-cp39-cp39-win32.whl", hash = "sha256:ea9872c80c132f4663822dd2a08d404073a5a9b5ba6155bea72fb2a79d1093b5"},
+    {file = "greenlet-2.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:db1a39669102a1d8d12b57de2bb7e2ec9066a6f2b3da35ae511ff93b01b5d564"},
+    {file = "greenlet-2.0.2.tar.gz", hash = "sha256:e7c8dc13af7db097bed64a051d2dd49e9f0af495c26995c00a9ee842690d34c0"},
+]
+
+[package.extras]
+docs = ["Sphinx", "docutils (<0.18)"]
+test = ["objgraph", "psutil"]
+
+[[package]]
+name = "idna"
+version = "3.4"
+description = "Internationalized Domain Names in Applications (IDNA)"
+category = "main"
+optional = false
+python-versions = ">=3.5"
+files = [
+    {file = "idna-3.4-py3-none-any.whl", hash = "sha256:90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2"},
+    {file = "idna-3.4.tar.gz", hash = "sha256:814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4"},
+]
+
+[[package]]
+name = "inflect"
+version = "6.0.2"
+description = "Correctly generate plurals, singular nouns, ordinals, indefinite articles; convert numbers to words"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "inflect-6.0.2-py3-none-any.whl", hash = "sha256:182741ec7e9e4c8f7f55b01fa6d80bcd3c4a183d349dfa6d9abbff0a1279e98f"},
+    {file = "inflect-6.0.2.tar.gz", hash = "sha256:f1a6bcb0105046f89619fde1a7d044c612c614c2d85ef182582d9dc9b86d309a"},
+]
+
+[package.dependencies]
+pydantic = ">=1.9.1"
+
+[package.extras]
+docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)"]
+testing = ["flake8 (<5)", "pygments", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"]
+
+[[package]]
+name = "iniconfig"
+version = "2.0.0"
+description = "brain-dead simple config-ini parsing"
+category = "dev"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "iniconfig-2.0.0-py3-none-any.whl", hash = "sha256:b6a85871a79d2e3b22d2d1b94ac2824226a63c6b741c88f7ae975f18b6778374"},
+    {file = "iniconfig-2.0.0.tar.gz", hash = "sha256:2d91e135bf72d31a410b17c16da610a82cb55f6b0477d1a902134b24a455b8b3"},
+]
+
+[[package]]
+name = "mysqlclient"
+version = "2.1.1"
+description = "Python interface to MySQL"
+category = "main"
+optional = false
+python-versions = ">=3.5"
+files = [
+    {file = "mysqlclient-2.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:c1ed71bd6244993b526113cca3df66428609f90e4652f37eb51c33496d478b37"},
+    {file = "mysqlclient-2.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:c812b67e90082a840efb82a8978369e6e69fc62ce1bda4ca8f3084a9d862308b"},
+    {file = "mysqlclient-2.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:0d1cd3a5a4d28c222fa199002810e8146cffd821410b67851af4cc80aeccd97c"},
+    {file = "mysqlclient-2.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:b355c8b5a7d58f2e909acdbb050858390ee1b0e13672ae759e5e784110022994"},
+    {file = "mysqlclient-2.1.1-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:996924f3483fd36a34a5812210c69e71dea5a3d5978d01199b78b7f6d485c855"},
+    {file = "mysqlclient-2.1.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:dea88c8d3f5a5d9293dfe7f087c16dd350ceb175f2f6631c9cf4caf3e19b7a96"},
+    {file = "mysqlclient-2.1.1.tar.gz", hash = "sha256:828757e419fb11dd6c5ed2576ec92c3efaa93a0f7c39e263586d1ee779c3d782"},
+]
+
+[[package]]
+name = "packaging"
+version = "23.0"
+description = "Core utilities for Python packages"
+category = "dev"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "packaging-23.0-py3-none-any.whl", hash = "sha256:714ac14496c3e68c99c29b00845f7a2b85f3bb6f1078fd9f72fd20f0570002b2"},
+    {file = "packaging-23.0.tar.gz", hash = "sha256:b6ad297f8907de0fa2fe1ccbd26fdaf387f5f47c7275fedf8cce89f99446cf97"},
+]
+
+[[package]]
+name = "pluggy"
+version = "1.0.0"
+description = "plugin and hook calling mechanisms for python"
+category = "dev"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "pluggy-1.0.0-py2.py3-none-any.whl", hash = "sha256:74134bbf457f031a36d68416e1509f34bd5ccc019f0bcc952c7b909d06b37bd3"},
+    {file = "pluggy-1.0.0.tar.gz", hash = "sha256:4224373bacce55f955a878bf9cfa763c1e360858e330072059e10bad68531159"},
+]
+
+[package.extras]
+dev = ["pre-commit", "tox"]
+testing = ["pytest", "pytest-benchmark"]
+
+[[package]]
+name = "pydantic"
+version = "1.10.4"
+description = "Data validation and settings management using python type hints"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "pydantic-1.10.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b5635de53e6686fe7a44b5cf25fcc419a0d5e5c1a1efe73d49d48fe7586db854"},
+    {file = "pydantic-1.10.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6dc1cc241440ed7ca9ab59d9929075445da6b7c94ced281b3dd4cfe6c8cff817"},
+    {file = "pydantic-1.10.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:51bdeb10d2db0f288e71d49c9cefa609bca271720ecd0c58009bd7504a0c464c"},
+    {file = "pydantic-1.10.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:78cec42b95dbb500a1f7120bdf95c401f6abb616bbe8785ef09887306792e66e"},
+    {file = "pydantic-1.10.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:8775d4ef5e7299a2f4699501077a0defdaac5b6c4321173bcb0f3c496fbadf85"},
+    {file = "pydantic-1.10.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:572066051eeac73d23f95ba9a71349c42a3e05999d0ee1572b7860235b850cc6"},
+    {file = "pydantic-1.10.4-cp310-cp310-win_amd64.whl", hash = "sha256:7feb6a2d401f4d6863050f58325b8d99c1e56f4512d98b11ac64ad1751dc647d"},
+    {file = "pydantic-1.10.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:39f4a73e5342b25c2959529f07f026ef58147249f9b7431e1ba8414a36761f53"},
+    {file = "pydantic-1.10.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:983e720704431a6573d626b00662eb78a07148c9115129f9b4351091ec95ecc3"},
+    {file = "pydantic-1.10.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75d52162fe6b2b55964fbb0af2ee58e99791a3138588c482572bb6087953113a"},
+    {file = "pydantic-1.10.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fdf8d759ef326962b4678d89e275ffc55b7ce59d917d9f72233762061fd04a2d"},
+    {file = "pydantic-1.10.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:05a81b006be15655b2a1bae5faa4280cf7c81d0e09fcb49b342ebf826abe5a72"},
+    {file = "pydantic-1.10.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d88c4c0e5c5dfd05092a4b271282ef0588e5f4aaf345778056fc5259ba098857"},
+    {file = "pydantic-1.10.4-cp311-cp311-win_amd64.whl", hash = "sha256:6a05a9db1ef5be0fe63e988f9617ca2551013f55000289c671f71ec16f4985e3"},
+    {file = "pydantic-1.10.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:887ca463c3bc47103c123bc06919c86720e80e1214aab79e9b779cda0ff92a00"},
+    {file = "pydantic-1.10.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fdf88ab63c3ee282c76d652fc86518aacb737ff35796023fae56a65ced1a5978"},
+    {file = "pydantic-1.10.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a48f1953c4a1d9bd0b5167ac50da9a79f6072c63c4cef4cf2a3736994903583e"},
+    {file = "pydantic-1.10.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a9f2de23bec87ff306aef658384b02aa7c32389766af3c5dee9ce33e80222dfa"},
+    {file = "pydantic-1.10.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:cd8702c5142afda03dc2b1ee6bc358b62b3735b2cce53fc77b31ca9f728e4bc8"},
+    {file = "pydantic-1.10.4-cp37-cp37m-win_amd64.whl", hash = "sha256:6e7124d6855b2780611d9f5e1e145e86667eaa3bd9459192c8dc1a097f5e9903"},
+    {file = "pydantic-1.10.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0b53e1d41e97063d51a02821b80538053ee4608b9a181c1005441f1673c55423"},
+    {file = "pydantic-1.10.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:55b1625899acd33229c4352ce0ae54038529b412bd51c4915349b49ca575258f"},
+    {file = "pydantic-1.10.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:301d626a59edbe5dfb48fcae245896379a450d04baeed50ef40d8199f2733b06"},
+    {file = "pydantic-1.10.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b6f9d649892a6f54a39ed56b8dfd5e08b5f3be5f893da430bed76975f3735d15"},
+    {file = "pydantic-1.10.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:d7b5a3821225f5c43496c324b0d6875fde910a1c2933d726a743ce328fbb2a8c"},
+    {file = "pydantic-1.10.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:f2f7eb6273dd12472d7f218e1fef6f7c7c2f00ac2e1ecde4db8824c457300416"},
+    {file = "pydantic-1.10.4-cp38-cp38-win_amd64.whl", hash = "sha256:4b05697738e7d2040696b0a66d9f0a10bec0efa1883ca75ee9e55baf511909d6"},
+    {file = "pydantic-1.10.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a9a6747cac06c2beb466064dda999a13176b23535e4c496c9d48e6406f92d42d"},
+    {file = "pydantic-1.10.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eb992a1ef739cc7b543576337bebfc62c0e6567434e522e97291b251a41dad7f"},
+    {file = "pydantic-1.10.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:990406d226dea0e8f25f643b370224771878142155b879784ce89f633541a024"},
+    {file = "pydantic-1.10.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2e82a6d37a95e0b1b42b82ab340ada3963aea1317fd7f888bb6b9dfbf4fff57c"},
+    {file = "pydantic-1.10.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:9193d4f4ee8feca58bc56c8306bcb820f5c7905fd919e0750acdeeeef0615b28"},
+    {file = "pydantic-1.10.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2b3ce5f16deb45c472dde1a0ee05619298c864a20cded09c4edd820e1454129f"},
+    {file = "pydantic-1.10.4-cp39-cp39-win_amd64.whl", hash = "sha256:9cbdc268a62d9a98c56e2452d6c41c0263d64a2009aac69246486f01b4f594c4"},
+    {file = "pydantic-1.10.4-py3-none-any.whl", hash = "sha256:4948f264678c703f3877d1c8877c4e3b2e12e549c57795107f08cf70c6ec7774"},
+    {file = "pydantic-1.10.4.tar.gz", hash = "sha256:b9a3859f24eb4e097502a3be1fb4b2abb79b6103dd9e2e0edb70613a4459a648"},
+]
+
+[package.dependencies]
+typing-extensions = ">=4.2.0"
+
+[package.extras]
+dotenv = ["python-dotenv (>=0.10.4)"]
+email = ["email-validator (>=1.0.3)"]
+
+[[package]]
+name = "pydevd-pycharm"
+version = "231.6471.3"
+description = "PyCharm Debugger (used in PyCharm and PyDev)"
+category = "main"
+optional = false
+python-versions = "*"
+files = [
+    {file = "pydevd-pycharm-231.6471.3.tar.gz", hash = "sha256:3c77340fd1f7e984790081e0a9e89e0284450ce242df7d01aecabbd24e4e46ae"},
+]
+
+[[package]]
+name = "pytest"
+version = "7.2.1"
+description = "pytest: simple powerful testing with Python"
+category = "dev"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "pytest-7.2.1-py3-none-any.whl", hash = "sha256:c7c6ca206e93355074ae32f7403e8ea12163b1163c976fee7d4d84027c162be5"},
+    {file = "pytest-7.2.1.tar.gz", hash = "sha256:d45e0952f3727241918b8fd0f376f5ff6b301cc0777c6f9a556935c92d8a7d42"},
+]
+
+[package.dependencies]
+attrs = ">=19.2.0"
+colorama = {version = "*", markers = "sys_platform == \"win32\""}
+exceptiongroup = {version = ">=1.0.0rc8", markers = "python_version < \"3.11\""}
+iniconfig = "*"
+packaging = "*"
+pluggy = ">=0.12,<2.0"
+tomli = {version = ">=1.0.0", markers = "python_version < \"3.11\""}
+
+[package.extras]
+testing = ["argcomplete", "hypothesis (>=3.56)", "mock", "nose", "pygments (>=2.7.2)", "requests", "xmlschema"]
+
+[[package]]
+name = "requests"
+version = "2.28.2"
+description = "Python HTTP for Humans."
+category = "main"
+optional = false
+python-versions = ">=3.7, <4"
+files = [
+    {file = "requests-2.28.2-py3-none-any.whl", hash = "sha256:64299f4909223da747622c030b781c0d7811e359c37124b4bd368fb8c6518baa"},
+    {file = "requests-2.28.2.tar.gz", hash = "sha256:98b1b2782e3c6c4904938b84c0eb932721069dfdb9134313beff7c83c2df24bf"},
+]
+
+[package.dependencies]
+certifi = ">=2017.4.17"
+charset-normalizer = ">=2,<4"
+idna = ">=2.5,<4"
+urllib3 = ">=1.21.1,<1.27"
+
+[package.extras]
+socks = ["PySocks (>=1.5.6,!=1.5.7)"]
+use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"]
+
+[[package]]
+name = "six"
+version = "1.16.0"
+description = "Python 2 and 3 compatibility utilities"
+category = "main"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*"
+files = [
+    {file = "six-1.16.0-py2.py3-none-any.whl", hash = "sha256:8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254"},
+    {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"},
+]
+
+[[package]]
+name = "sqlalchemy"
+version = "1.4.41"
+description = "Database Abstraction Library"
+category = "main"
+optional = false
+python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,>=2.7"
+files = [
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:13e397a9371ecd25573a7b90bd037db604331cf403f5318038c46ee44908c44d"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:2d6495f84c4fd11584f34e62f9feec81bf373787b3942270487074e35cbe5330"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-win32.whl", hash = "sha256:e570cfc40a29d6ad46c9aeaddbdcee687880940a3a327f2c668dd0e4ef0a441d"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-win_amd64.whl", hash = "sha256:5facb7fd6fa8a7353bbe88b95695e555338fb038ad19ceb29c82d94f62775a05"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27mu-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:f37fa70d95658763254941ddd30ecb23fc4ec0c5a788a7c21034fc2305dab7cc"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:361f6b5e3f659e3c56ea3518cf85fbdae1b9e788ade0219a67eeaaea8a4e4d2a"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0990932f7cca97fece8017414f57fdd80db506a045869d7ddf2dda1d7cf69ecc"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:cd767cf5d7252b1c88fcfb58426a32d7bd14a7e4942497e15b68ff5d822b41ad"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5102fb9ee2c258a2218281adcb3e1918b793c51d6c2b4666ce38c35101bb940e"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-win32.whl", hash = "sha256:2082a2d2fca363a3ce21cfa3d068c5a1ce4bf720cf6497fb3a9fc643a8ee4ddd"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-win_amd64.whl", hash = "sha256:e4b12e3d88a8fffd0b4ca559f6d4957ed91bd4c0613a4e13846ab8729dc5c251"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:90484a2b00baedad361402c257895b13faa3f01780f18f4a104a2f5c413e4536"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b67fc780cfe2b306180e56daaa411dd3186bf979d50a6a7c2a5b5036575cbdbb"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2ad2b727fc41c7f8757098903f85fafb4bf587ca6605f82d9bf5604bd9c7cded"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-win32.whl", hash = "sha256:59bdc291165b6119fc6cdbc287c36f7f2859e6051dd923bdf47b4c55fd2f8bd0"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-win_amd64.whl", hash = "sha256:d2e054aed4645f9b755db85bc69fc4ed2c9020c19c8027976f66576b906a74f1"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:4ba7e122510bbc07258dc42be6ed45997efdf38129bde3e3f12649be70683546"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c0dcf127bb99458a9d211e6e1f0f3edb96c874dd12f2503d4d8e4f1fd103790b"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:e16c2be5cb19e2c08da7bd3a87fed2a0d4e90065ee553a940c4fc1a0fb1ab72b"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f5ebeeec5c14533221eb30bad716bc1fd32f509196318fb9caa7002c4a364e4c"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-win32.whl", hash = "sha256:3e2ef592ac3693c65210f8b53d0edcf9f4405925adcfc031ff495e8d18169682"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-win_amd64.whl", hash = "sha256:eb30cf008850c0a26b72bd1b9be6730830165ce049d239cfdccd906f2685f892"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:c23d64a0b28fc78c96289ffbd0d9d1abd48d267269b27f2d34e430ea73ce4b26"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8eb8897367a21b578b26f5713833836f886817ee2ffba1177d446fa3f77e67c8"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:14576238a5f89bcf504c5f0a388d0ca78df61fb42cb2af0efe239dc965d4f5c9"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:639e1ae8d48b3c86ffe59c0daa9a02e2bfe17ca3d2b41611b30a0073937d4497"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-win32.whl", hash = "sha256:0005bd73026cd239fc1e8ccdf54db58b6193be9a02b3f0c5983808f84862c767"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-win_amd64.whl", hash = "sha256:5323252be2bd261e0aa3f33cb3a64c45d76829989fa3ce90652838397d84197d"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:05f0de3a1dc3810a776275763764bb0015a02ae0f698a794646ebc5fb06fad33"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0002e829142b2af00b4eaa26c51728f3ea68235f232a2e72a9508a3116bd6ed0"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:22ff16cedab5b16a0db79f1bc99e46a6ddececb60c396562e50aab58ddb2871c"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ccfd238f766a5bb5ee5545a62dd03f316ac67966a6a658efb63eeff8158a4bbf"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-win32.whl", hash = "sha256:58bb65b3274b0c8a02cea9f91d6f44d0da79abc993b33bdedbfec98c8440175a"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-win_amd64.whl", hash = "sha256:ce8feaa52c1640de9541eeaaa8b5fb632d9d66249c947bb0d89dd01f87c7c288"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:199a73c31ac8ea59937cc0bf3dfc04392e81afe2ec8a74f26f489d268867846c"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4676d51c9f6f6226ae8f26dc83ec291c088fe7633269757d333978df78d931ab"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:036d8472356e1d5f096c5e0e1a7e0f9182140ada3602f8fff6b7329e9e7cfbcd"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2307495d9e0ea00d0c726be97a5b96615035854972cc538f6e7eaed23a35886c"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-win32.whl", hash = "sha256:9c56e19780cd1344fcd362fd6265a15f48aa8d365996a37fab1495cae8fcd97d"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-win_amd64.whl", hash = "sha256:f5fa526d027d804b1f85cdda1eb091f70bde6fb7d87892f6dd5a48925bc88898"},
+    {file = "SQLAlchemy-1.4.41.tar.gz", hash = "sha256:0292f70d1797e3c54e862e6f30ae474014648bc9c723e14a2fda730adb0a9791"},
+]
+
+[package.dependencies]
+greenlet = {version = "!=0.4.17", markers = "python_version >= \"3\" and (platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\")"}
+
+[package.extras]
+aiomysql = ["aiomysql", "greenlet (!=0.4.17)"]
+aiosqlite = ["aiosqlite", "greenlet (!=0.4.17)", "typing_extensions (!=3.10.0.1)"]
+asyncio = ["greenlet (!=0.4.17)"]
+asyncmy = ["asyncmy (>=0.2.3,!=0.2.4)", "greenlet (!=0.4.17)"]
+mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2)"]
+mssql = ["pyodbc"]
+mssql-pymssql = ["pymssql"]
+mssql-pyodbc = ["pyodbc"]
+mypy = ["mypy (>=0.910)", "sqlalchemy2-stubs"]
+mysql = ["mysqlclient (>=1.4.0)", "mysqlclient (>=1.4.0,<2)"]
+mysql-connector = ["mysql-connector-python"]
+oracle = ["cx_oracle (>=7)", "cx_oracle (>=7,<8)"]
+postgresql = ["psycopg2 (>=2.7)"]
+postgresql-asyncpg = ["asyncpg", "greenlet (!=0.4.17)"]
+postgresql-pg8000 = ["pg8000 (>=1.16.6,!=1.29.0)"]
+postgresql-psycopg2binary = ["psycopg2-binary"]
+postgresql-psycopg2cffi = ["psycopg2cffi"]
+pymysql = ["pymysql", "pymysql (<1)"]
+sqlcipher = ["sqlcipher3_binary"]
+
+[[package]]
+name = "sqlalchemy2-stubs"
+version = "0.0.2a32"
+description = "Typing Stubs for SQLAlchemy 1.4"
+category = "main"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "sqlalchemy2-stubs-0.0.2a32.tar.gz", hash = "sha256:2a2cfab71d35ac63bf21ad841d8610cd93a3bd4c6562848c538fa975585c2739"},
+    {file = "sqlalchemy2_stubs-0.0.2a32-py3-none-any.whl", hash = "sha256:7f5fb30b0cf7c6b74c50c1d94df77ff32007afee8d80499752eb3fedffdbdfb8"},
+]
+
+[package.dependencies]
+typing-extensions = ">=3.7.4"
+
+[[package]]
+name = "sqlmodel"
+version = "0.0.8"
+description = "SQLModel, SQL databases in Python, designed for simplicity, compatibility, and robustness."
+category = "main"
+optional = false
+python-versions = ">=3.6.1,<4.0.0"
+files = [
+    {file = "sqlmodel-0.0.8-py3-none-any.whl", hash = "sha256:0fd805719e0c5d4f22be32eb3ffc856eca3f7f20e8c7aa3e117ad91684b518ee"},
+    {file = "sqlmodel-0.0.8.tar.gz", hash = "sha256:3371b4d1ad59d2ffd0c530582c2140b6c06b090b32af9b9c6412986d7b117036"},
+]
+
+[package.dependencies]
+pydantic = ">=1.8.2,<2.0.0"
+SQLAlchemy = ">=1.4.17,<=1.4.41"
+sqlalchemy2-stubs = "*"
+
+[[package]]
+name = "termcolor"
+version = "2.2.0"
+description = "ANSI color formatting for output in terminal"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "termcolor-2.2.0-py3-none-any.whl", hash = "sha256:91ddd848e7251200eac969846cbae2dacd7d71c2871e92733289e7e3666f48e7"},
+    {file = "termcolor-2.2.0.tar.gz", hash = "sha256:dfc8ac3f350788f23b2947b3e6cfa5a53b630b612e6cd8965a015a776020b99a"},
+]
+
+[package.extras]
+tests = ["pytest", "pytest-cov"]
+
+[[package]]
+name = "tomli"
+version = "2.0.1"
+description = "A lil' TOML parser"
+category = "dev"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "tomli-2.0.1-py3-none-any.whl", hash = "sha256:939de3e7a6161af0c887ef91b7d41a53e7c5a1ca976325f429cb46ea9bc30ecc"},
+    {file = "tomli-2.0.1.tar.gz", hash = "sha256:de526c12914f0c550d15924c62d72abc48d6fe7364aa87328337a31007fe8a4f"},
+]
+
+[[package]]
+name = "typing-extensions"
+version = "4.4.0"
+description = "Backported and Experimental Type Hints for Python 3.7+"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "typing_extensions-4.4.0-py3-none-any.whl", hash = "sha256:16fa4864408f655d35ec496218b85f79b3437c829e93320c7c9215ccfd92489e"},
+    {file = "typing_extensions-4.4.0.tar.gz", hash = "sha256:1511434bb92bf8dd198c12b1cc812e800d4181cfcb867674e0f8279cc93087aa"},
+]
+
+[[package]]
+name = "urllib3"
+version = "1.26.14"
+description = "HTTP library with thread-safe connection pooling, file post, and more."
+category = "main"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*"
+files = [
+    {file = "urllib3-1.26.14-py2.py3-none-any.whl", hash = "sha256:75edcdc2f7d85b137124a6c3c9fc3933cdeaa12ecb9a6a959f22797a0feca7e1"},
+    {file = "urllib3-1.26.14.tar.gz", hash = "sha256:076907bf8fd355cde77728471316625a4d2f7e713c125f51953bb5b3eecf4f72"},
+]
+
+[package.extras]
+brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)", "brotlipy (>=0.6.0)"]
+secure = ["certifi", "cryptography (>=1.3.4)", "idna (>=2.0.0)", "ipaddress", "pyOpenSSL (>=0.14)", "urllib3-secure-extra"]
+socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"]
+
+[metadata]
+lock-version = "2.0"
+python-versions = "^3.10"
+content-hash = "e4293f87235f9c63a69951ac8ea600dd64719fa7f28c059decad45974adc5222"
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/__init__.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/__init__.py
index b11fd8254..edabf45f0 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/__init__.py
@@ -4,55 +4,19 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
 
 
-  comment: on-failure
+from .model import ToolModel
+from .logger import logger
+from .message import Connection
+from .plugin import Plugin
+from .stream import Stream, Substream
+from .context import Context
diff --git a/backend/python/pydevlake/pydevlake/api.py b/backend/python/pydevlake/pydevlake/api.py
new file mode 100644
index 000000000..cc2a930cd
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/api.py
@@ -0,0 +1,290 @@
+# 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
+
+#     http://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.
+
+
+from __future__ import annotations
+
+from typing import Optional, Union
+import json
+
+import requests as req
+from urllib.parse import urljoin
+
+
+RouteArgs = Union[list[str], dict[str, str]]
+QueryArgs = dict[str, str]
+Headers = dict[str, str]
+
+
+class Request:
+    def __init__(self,
+                 url: str,
+                 query_args: Optional[QueryArgs] = None,
+                 headers: Optional[Headers] = None):
+        self.url = url
+        self.query_args = query_args or {}
+        self.headers = headers or {}
+
+    def copy(self):
+        return Request(self.url, self.query_args, self.headers)
+
+    def __str__(self):
+        if self.query_args:
+            query_str = '&'.join(f'{k}={v}' for k, v in self.query_args.items())
+            return f'{self.url}?{query_str}'
+        return self.url
+
+
+class Response:
+    def __init__(self,
+                 request: Request,
+                 status: int,
+                 body: bytes = None,
+                 headers: Headers = None):
+        self.request = request
+        self.status = status
+        self.body = body or bytes()
+        self.headers = headers or {}
+
+    @property
+    def json(self):
+        if not hasattr(self, '_json'):
+            self._json = json.loads(self.body)
+        return self._json
+
+    def __str__(self):
+        return f'{self.request}: {self.status}'
+
+
+# Sentinel value to abort processing of requests/responses in hooks
+ABORT = object()
+
+
+class APIBase:
+    """
+    The base class for defining APIs.
+    It implements a hook system to preprocess requests before sending them and postprocess response 
+    before returning them.
+    Hooks are declared by decorating methods with `@request_hook` and `@response_hook`.
+    Hooks are executed in the order they are declared.
+    """
+    @property
+    def session(self):
+        if not hasattr(self, '_session'):
+            self._session = req.Session()
+        return self._session
+
+    @property
+    def base_url(self):
+        pass
+
+    def send(self, request: Request):
+        request = self._apply_hooks(request, self.request_hooks())
+        if request is ABORT:
+            return ABORT
+
+        res = self.session.get(
+            url=request.url,
+            headers=request.headers,
+            params=request.query_args
+        )
+
+        response = Response(
+            request=request,
+            status=res.status_code,
+            body=res.content,
+            headers=res.headers
+        )
+
+        response = self._apply_hooks(response, self.response_hooks())
+
+        return response
+
+    def _apply_hooks(self, target, hooks):
+        for hook in hooks:
+            result = hook.apply(target, self)
+            if result is ABORT:
+                return ABORT
+
+            if isinstance(result, type(target)):
+                target = result
+        return target
+                
+    def get(self, path, **query_args):
+        req = Request(urljoin(self.base_url, path), query_args)
+        return self.send(req)
+
+    def request_hooks(self):
+        if not hasattr(self, '_request_hooks'):
+            self._request_hooks = [h for h in self._iter_members() if isinstance(h, RequestHook)]
+        return self._request_hooks
+
+    def response_hooks(self):
+        if not hasattr(self, '_response_hooks'):
+            self._response_hooks = [h for h in self._iter_members() if isinstance(h, ResponseHook)]
+        return self._response_hooks
+ 
+    def _iter_members(self):
+        for c in reversed(type(self).__mro__):
+            for m in c.__dict__.values():
+                yield m
+
+
+class RequestHook:
+    """
+    Preprocess a request before sending it.
+    """
+    def apply(self, request: Request, api: APIBase):
+        pass
+
+
+class CustomRequestHook(RequestHook):
+    def __init__(self, fn):
+        self.fn = fn
+
+    def apply(self, request: Request, api: APIBase):
+        return self.fn(api, request)
+
+request_hook = CustomRequestHook
+
+
+class ResponseHook:
+    def apply(self, response: Response, api: APIBase):
+        pass
+
+
+class CustomResponseHook(ResponseHook):
+    def __init__(self, fn):
+        self.fn = fn
+
+    def apply(self, response: Response, api: APIBase):
+        return self.fn(api, response)
+
+response_hook = CustomResponseHook
+
+
+class Paginator:
+    """
+    Encapsulate logic for handling paginated responses.
+    """
+    def get_items(self, response) -> Optional[list[object]]:
+        """
+        Extracts the items from a response, e.g. returning the
+        `items` attribute of a JSON body.
+        Returning None indicates that the response is not paginated.
+        """
+        pass
+
+    def get_next_page_id(self, response) -> Optional[int | str]:
+        """
+        Extracts or compute the id of the next page from the response,
+        e.g. incrementing the value of `page` of a JSON body.
+        This id will be suplied to the next request via `set_next_page_param`.
+        Returning None indicates that the response is the last page.
+        """
+        pass
+
+    def set_next_page_param(self, request, next_page_id: int | str):
+        """
+        Modify the request to set the parameter for fetching next page,
+        e.g. set the `page` query parameter. 
+        """
+        pass
+
+
+class PagedResponse(Response):
+    """
+    Decorate requests.Response to add iteration of items
+    within the page and fetching next pages.
+    """
+    def __init__(self, response, paginator, api):
+        self.response = response
+        self.paginator = paginator
+        self.api = api
+
+    def __iter__(self):
+        current = self.response
+
+        while True:
+            items = self.paginator.get_items(current)
+            for item in items:
+                yield item
+
+            next_page_id = self.paginator.get_next_page_id(current)
+            if not next_page_id:
+                # No next page
+                return
+
+            # Fetch next page
+            next_request = current.request.copy()
+            self.paginator.set_next_page_param(next_request, next_page_id)
+            current = self.api.send(next_request)
+
+    def __getattr__(self, attr_name):
+        # Delegate everything to Response
+        return getattr(self.response, attr_name)
+
+
+class TokenPaginator(Paginator):
+    def __init__(self, items_attr: str, next_page_token_attr: str, next_page_token_param: str):
+        self.items_attr = items_attr
+        self.next_page_token_attr = next_page_token_attr
+        self.next_page_token_param = next_page_token_param
+
+    def get_items(self, response) -> Optional[list[object]]:
+        return response.json[self.items_attr]
+
+    def get_next_page_id(self, response) -> Optional[int | str]:
+        return response.json[self.next_page_token_attr]
+
+    def set_next_page_param(self, request, next_page_id):
+        request.query_args[self.next_page_token_param] = next_page_id
+        
+
+class APIException(Exception):
+    def __init__(self, response):
+        self.response = response
+    
+    def __str__(self):
+        return f'APIException: {self.response}'
+
+
+class API(APIBase):
+    """
+    Provides hooks for:
+    - pagination: define the `paginator` property in subclasses
+
+    # TODO:
+    - Error handling response hook: retries, 
+    - Rate limitation
+    """
+    @property
+    def paginator(self) -> Paginator:
+        """
+        Redefine in subclass to handle pagination
+        """
+        return None
+
+    @response_hook
+    def handle_error(self, response):
+        if response.status != 200:
+            raise APIException(response)
+
+    @response_hook
+    def paginate(self, response):
+        paginator = self.paginator
+        if not paginator:
+            return
+        return PagedResponse(response, paginator, self)
diff --git a/backend/python/pydevlake/pydevlake/context.py b/backend/python/pydevlake/pydevlake/context.py
new file mode 100644
index 000000000..a5c97b5ff
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/context.py
@@ -0,0 +1,52 @@
+# 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
+
+#     http://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.
+
+
+from urllib.parse import urlparse, parse_qsl
+from sqlmodel import SQLModel, create_engine
+
+from pydevlake.message import Connection
+
+
+class Context:
+    def __init__(self, db_url: str, connection_id: int, connection: Connection, options: dict):
+        self.db_url = db_url
+        self.connection_id = connection_id
+        self.connection = connection
+        self.options = options
+        self._engine = None
+
+    @property
+    def engine(self):
+        if not self._engine:
+            db_url = self.db_url
+            if not db_url:
+                raise Exception("Missing db_url setting")
+
+            # `parseTime` parameter is not understood by MySQL driver
+            # so we have to parse query args to remove it
+            connect_args = dict(parse_qsl(urlparse(self.db_url).query))
+            db_url = self.db_url.split('?')[0]
+
+            if 'parseTime' in connect_args:
+                del connect_args['parseTime']
+
+            self._engine = create_engine(db_url, connect_args=connect_args)
+            SQLModel.metadata.create_all(self._engine)
+        return self._engine
+
+    @property
+    def incremental(self):
+        return self.options.get('incremental') is True
diff --git a/backend/python/pydevlake/pydevlake/doc.template.json b/backend/python/pydevlake/pydevlake/doc.template.json
new file mode 100644
index 000000000..b152a5cf6
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/doc.template.json
@@ -0,0 +1,155 @@
+{
+    "info": {
+        "title": "$plugin_name plugin documentation",
+        "version": "1.0.0"
+    },
+    "openapi": "3.0.2",
+    "paths": {
+        "/plugins/$plugin_name/connections/{connectionId}": {
+            "get": {
+                "description": "Get a connection",
+                "parameters": [
+                    {
+                        "$$ref": "#/components/parameters/connectionId"
+                    }
+                ],
+                "responses": {
+                    "200": {
+                        "content": {
+                            "application/json": {
+                                "schema": {
+                                    "$$ref": "#/components/schemas/connection"
+                                }
+                            }
+                        }
+                    },
+                    "400": {
+                        "description": "There is no connection with given id"
+                    }
+                }
+            },
+            "patch": {
+                "description": "Update a connection",
+                "parameters": [
+                    {
+                        "$$ref": "#/components/parameters/connectionId"
+                    },
+                    {
+                        "name": "connection",
+                        "required": true,
+                        "in": "body",
+                        "schema": {
+                            "$$ref": "#/components/schemas/connection"
+                        }
+                    }
+                ],
+                "responses": {
+                    "200": {
+                        "content": {
+                            "application/json": {
+                                "schema": {
+                                    "$$ref": "#/components/schemas/connection"
+                                }
+                            }
+                        }
+                    }
+                }
+            },
+            "delete": {
+                "description": "Delete a connection",
+                "parameters": [
+                    {
+                        "$$ref": "#/components/parameters/connectionId"
+                    }
+                ],
+                "response": {
+                    "200": {
+                        "description": "The connection has been deleted"
+                    },
+                    "400": {
+                        "description": "There is no connection with given id"
+                    }
+                }
+            }
+        },
+        "/plugins/$plugin_name/connections": {
+            "get": {
+                "description": "Get all connections",
+                "responses": {
+                    "200": {
+                        "content": {
+                            "application/json": {
+                                "schema": {
+                                    "type": "array",
+                                    "items": {
+                                        "$$ref": "#/components/schemas/connection"
+                                    }
+                                }
+                            }
+                        }
+                    }
+                }
+            },
+            "post": {
+                "description": "Create a connection",
+                "parameters": [
+                    {
+                        "name": "connection",
+                        "required": true,
+                        "in": "body",
+                        "schema": {
+                            "$$ref": "#/components/schemas/connection"
+                        }
+                    }
+                ],
+                "responses": {
+                    "200": {
+                        "content": {
+                            "application/json": {
+                                "schema": {
+                                    "$$ref": "#/components/schemas/connection"
+                                }
+                            }
+                        }
+                    }
+                }
+            }
+        },
+        "/plugins/$plugin_name/test": {
+            "post": {
+                "description": "Test if a connection is valid",
+                "body": {
+                    "application/json": {
+                        "schema": {
+                            "$$ref": "#/components/schemas/connection"
+                        }
+                    }
+                }
+            },
+            "response": {
+                "200": {
+                    "description": "The connection is valid"
+                },
+                "400": {
+                    "description": "The connection is not valid"
+                }
+            }
+        }
+    },
+    "components": {
+        "schemas": {
+            "connection": $connection_schema
+        },
+        "parameters": {
+            "connectionId": {
+                "name": "connectionId",
+                "description": "Id of the connection",
+                "in": "path",
+                "required": true,
+                "schema": {
+                    "type": "int"
+                }
+            }
+        }
+    }
+}
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/docgen.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/docgen.py
index b11fd8254..acce0a39f 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/docgen.py
@@ -4,55 +4,30 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
-
-
-  comment: on-failure
+
+
+from typing import Type
+from pathlib import Path
+from string import Template
+import json
+
+from pydevlake.message import Connection
+
+
+# TODO: Move swagger documentation generation to GO side along with API implementation
+TEMPLATE_PATH = str(Path(__file__).parent / 'doc.template.json')
+
+def generate_doc(plugin_name: str, connection_type: Type[Connection]):
+    with open(TEMPLATE_PATH, 'r') as f:
+        doc_template = Template(f.read())
+        connection_schema = connection_type.schema_json()
+        doc = doc_template.substitute(plugin_name=plugin_name, connection_schema=connection_schema)
+        return json.loads(doc)
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/domain_layer/__init__.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/domain_layer/__init__.py
index b11fd8254..65d64ce95 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/domain_layer/__init__.py
@@ -4,55 +4,11 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
-
-
-  comment: on-failure
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/domain_layer/crossdomain.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/domain_layer/crossdomain.py
index b11fd8254..b001531df 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/domain_layer/crossdomain.py
@@ -4,55 +4,21 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
 
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
+
+from pydevlake.model import DomainModel
 
 
+class User(DomainModel, table=True):
+    __table_name__ = 'users'
 
-  comment: on-failure
+    name: str
+    email: str
diff --git a/backend/python/pydevlake/pydevlake/domain_layer/devops.py b/backend/python/pydevlake/pydevlake/domain_layer/devops.py
new file mode 100644
index 000000000..c15dd5c5d
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/domain_layer/devops.py
@@ -0,0 +1,68 @@
+# 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
+
+#     http://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.
+
+
+from typing import Optional
+from datetime import datetime
+from enum import Enum
+
+from sqlmodel import Field, Relationship
+
+from pydevlake.model import DomainModel, NoPKModel
+
+
+class CICDScope(DomainModel):
+	name: str
+	description: str
+	url: str
+	createdDate: datetime
+	updatedDate: datetime
+
+
+class CICDPipeline(DomainModel, table=True):
+    __table_name__ = 'cicd_pipelines'
+
+    class Result(Enum):
+        SUCCESS = "SUCCESS"
+        FAILURE = "FAILURE"
+        ABORT = "ABORT"
+        MANUAL = "MANUAL"
+
+    class Status(Enum):
+        IN_PROGRESS = "IN_PROGRESS"
+        DONE = "DONE"
+        MANUAL = "MANUAL"
+
+    class Type(Enum):
+        CI = "CI"
+        CD = "CD"
+        
+    name: str
+    status: Status
+    created_date: datetime
+    finished_date: Optional[datetime]
+    result: Optional[Result]
+    duration_sec: Optional[int]
+    environment: Optional[str]
+    type: Optional[Type] #Unused
+
+    # parent_pipelines: list["CICDPipeline"] = Relationship(back_populates="child_pipelines", link_model="CICDPipelineRelationship")
+    # child_pipelines: list["CICDPipeline"] = Relationship(back_populates="parent_pipelines", link_model="CICDPipelineRelationship")
+
+
+class CICDPipelineRelationship(NoPKModel):
+    __table_name__ = 'cicd_pipeline_relationships'
+    parent_pipeline_id: str = Field(primary_key=True, foreign_key=CICDPipeline.id)
+    child_pipeline_id: str = Field(primary_key=True, foreign_key=CICDPipeline.id)
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/extractor.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/extractor.py
index b11fd8254..8ee42914a 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/extractor.py
@@ -4,55 +4,26 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
 
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
 
+from typing import Type
+from pydevlake import ToolModel
 
 
-  comment: on-failure
+def autoextract(json: dict, model_cls: Type[ToolModel]) -> ToolModel:
+    annotations = dict(model_cls.__annotations__)
+    for key, value in json.items():
+        if key in annotations:
+            expected_type = annotations[key]
+            if isinstance(expected_type, type) and issubclass(expected_type, ToolModel):
+                # TODO: replace with actual foreign key
+                json[key] = value["id"]                    
+    return model_cls(**json)
diff --git a/backend/python/pydevlake/pydevlake/ipc.py b/backend/python/pydevlake/pydevlake/ipc.py
new file mode 100644
index 000000000..a457b9794
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/ipc.py
@@ -0,0 +1,93 @@
+# 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
+
+#     http://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.
+
+
+import os
+from functools import wraps
+from typing import Generator, TextIO
+
+from pydevlake.context import Context
+from pydevlake.message import Message
+
+
+def plugin_method(func):
+    def open_send_channel() -> TextIO:
+        fd = 3
+        return os.fdopen(fd, 'w')
+
+    def send_output(send_ch: TextIO, obj: object):
+        if not isinstance(obj, Message):
+            return
+        send_ch.write(obj.json())
+        send_ch.write('\n')
+        send_ch.flush()
+
+    @wraps(func)
+    def wrapper(self, *args):
+        ret = func(self, *args)
+        if ret is not None:
+            with open_send_channel() as send_ch:
+                if isinstance(ret, Generator):
+                    for each in ret:
+                        send_output(send_ch, each)
+                else:
+                    send_output(send_ch, ret)
+        return None
+
+    return wrapper
+
+
+class PluginCommands:
+    def __init__(self, plugin):
+        self._plugin = plugin
+
+    @plugin_method
+    def collect(self, ctx: dict, stream: str):
+        yield from self._plugin.collect(self._mk_context(ctx), stream)
+
+    @plugin_method
+    def extract(self, ctx: dict, stream: str):
+        yield from self._plugin.extract(self._mk_context(ctx), stream)
+
+    @plugin_method
+    def convert(self, ctx: dict, stream: str):
+        yield from self._plugin.convert(self._mk_context(ctx), stream)
+
+    @plugin_method
+    def test_connection(self, connection: dict):
+        connection = self._plugin.connection_type(**connection)
+        self._plugin.test_connection(connection)
+
+    @plugin_method
+    def make_pipeline(self, ctx: dict, scopes: list[dict]):
+        yield from self._plugin.make_pipeline(self._mk_context(ctx), scopes)
+
+    @plugin_method
+    def run_migrations(self, force: bool):
+        self._plugin.run_migrations(force)
+
+    @plugin_method
+    def plugin_info(self):
+        return self._plugin.plugin_info()
+
+    def startup(self, endpoint: str):
+        self._plugin.startup(endpoint)
+
+    def _mk_context(self, data: dict):
+        db_url = data['db_url']
+        connection_id = data['connection_id']
+        connection = self._plugin.connection_type(**data['connection'])
+        options = data.get('options', {})
+        return Context(db_url, connection_id, connection, options)
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pydevlake/logger.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pydevlake/logger.py
index b11fd8254..8f297ab8e 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pydevlake/logger.py
@@ -4,55 +4,38 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
-
-
-  comment: on-failure
+
+
+import sys
+import logging
+
+
+stdout_handler = logging.StreamHandler(sys.stdout)
+stdout_handler.addFilter(lambda rec: rec.levelno < logging.ERROR)
+
+
+stderr_handler = logging.StreamHandler(sys.stderr)
+stderr_handler.addFilter(lambda rec: rec.levelno >= logging.ERROR)
+
+logging.basicConfig(
+    level=logging.DEBUG,
+    format='%(levelname)s: %(message)s',
+    handlers=[stdout_handler, stderr_handler]
+)
+
+logger = logging.getLogger()
+
+
+if __name__ == "__main__":
+    logger.info('hey')
+    logger.debug('wut?')
+    logger.error('oops')
+    logger.critical('boom')
diff --git a/backend/python/pydevlake/pydevlake/message.py b/backend/python/pydevlake/pydevlake/message.py
new file mode 100644
index 000000000..de09167c4
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/message.py
@@ -0,0 +1,90 @@
+# 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
+
+#     http://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.
+
+
+from pydantic import BaseModel
+
+
+class Message(BaseModel):
+    pass
+
+
+class SubtaskMeta(BaseModel):
+    name: str
+    entry_point_name: str
+    required: bool
+    enabled_by_default: bool
+    description: str
+    domain_types: list[str]
+    arguments: list[str] = None
+
+
+class PluginInfo(Message):
+    name: str
+    description: str
+    connection_schema: dict
+    plugin_path: str
+    subtask_metas: list[SubtaskMeta]
+    extension: str = "datasource"
+    type: str = "python-poetry"
+
+
+class SwaggerDoc(Message):
+    name: str
+    resource: str
+    spec: dict
+
+
+class PluginDetails(Message):
+    plugin_info: PluginInfo
+    swagger: SwaggerDoc
+
+
+class RemoteProgress(Message):
+    increment: int = 0
+    current: int = 0
+    total: int = 0
+
+
+class Connection(Message):
+    pass
+
+
+class PipelineTask(Message):
+    plugin: str
+    # Do not snake_case this attribute,
+    # it must match the json tag name in PipelineTask go struct
+    skipOnFail: bool
+    subtasks: list[str]
+    options: dict[str, object]
+
+
+class PipelineStage(Message):
+    tasks: list[PipelineTask]
+
+
+class PipelinePlan(Message):
+    stages: list[PipelineStage]
+
+
+class PipelineScope(Message):
+    id: str
+    name: str
+    table_name: str
+
+
+class BlueprintScope(Message):
+    id: str
+    name: str
diff --git a/backend/python/pydevlake/pydevlake/model.py b/backend/python/pydevlake/pydevlake/model.py
new file mode 100644
index 000000000..3c8705010
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/model.py
@@ -0,0 +1,100 @@
+# 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
+
+#     http://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.
+
+
+import os
+from typing import Optional
+from inspect import getmodule
+
+from datetime import datetime
+from sqlalchemy import Column, String, DateTime, func
+from sqlalchemy.orm import declared_attr
+from sqlalchemy.inspection import inspect
+from sqlmodel import SQLModel, Field
+import inflect
+
+inflect_engine = inflect.engine()
+
+
+def get_plugin_name(cls):
+    """
+    Get the plugin name from a class by looking into
+    the file path of its module.
+    """
+    module = getmodule(cls)
+    path_segments = module.__file__.split(os.sep)
+    # Finds the name of the first enclosing folder
+    # that is not a python module 
+    depth = len(module.__name__.split('.')) + 1
+    return path_segments[-depth]
+
+
+class RawModel(SQLModel):
+    id: int = Field(primary_key=True)
+    params: str = b''
+    data: bytes
+    url: str = b''
+    input: bytes = b''
+    created_at: datetime = Field(default_factory=datetime.now)
+
+
+class RawDataOrigin(SQLModel):
+    # SQLModel doesn't like attributes starting with _
+    # so we change the names of the columns.
+    raw_data_params: str = Field(sa_column_kwargs={'name':'_raw_data_params'})
+    raw_data_table: str = Field(sa_column_kwargs={'name':'_raw_data_table'})
+    raw_data_id: Optional[str] = Field(sa_column_kwargs={'name':'_raw_data_id'})
+    raw_data_remark: Optional[str] = Field(sa_column_kwargs={'name':'_raw_data_remark'})
+
+    def set_origin(self, raw: RawModel):
+        self.raw_data_id = raw.id
+        self.raw_data_params = raw.params
+        self.raw_data_table = raw.__tablename__
+
+
+class ToolModel(RawDataOrigin):
+    @declared_attr
+    def __tablename__(cls) -> str:
+        plugin_name = get_plugin_name(cls)
+        plural_entity = inflect_engine.plural_noun(cls.__name__.lower())
+        return f'_tool_{plugin_name}_{plural_entity}'
+
+
+class NoPKModel(SQLModel):
+    created_at: datetime = Field(
+        sa_column=Column(DateTime(), default=func.now())
+    )
+    updated_at: datetime = Field(
+        sa_column=Column(DateTime(), default=func.now(), onupdate=func.now())
+    )
+
+
+class DomainModel(NoPKModel):
+    id: str = Field(primary_key=True)
+
+
+def generate_domain_id(tool_model: ToolModel, connection_id: str):
+    """
+    Generate an identifier for a domain entity
+    from the tool entity it originates from.
+    """
+    model_type = type(tool_model)
+    segments = [get_plugin_name(model_type), model_type.__name__, str(connection_id)]
+    mapper = inspect(model_type)
+    for primary_key_column in mapper.primary_key:
+        prop = mapper.get_property_by_column(primary_key_column)
+        attr_val = getattr(tool_model, prop.key)
+        segments.append(str(attr_val))
+    return ':'.join(segments)
diff --git a/backend/python/pydevlake/pydevlake/plugin.py b/backend/python/pydevlake/pydevlake/plugin.py
new file mode 100644
index 000000000..0e3c0a166
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/plugin.py
@@ -0,0 +1,174 @@
+# 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
+
+#     http://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.
+
+
+from typing import Type, Union, Iterable
+import sys
+from abc import abstractmethod
+import requests
+
+import fire
+
+import pydevlake.message as msg
+from pydevlake.subtasks import Subtask
+from pydevlake.docgen import generate_doc
+from pydevlake.ipc import PluginCommands
+from pydevlake.context import Context
+from pydevlake.stream import Stream
+from pydevlake.model import DomainModel
+
+
+class Plugin:
+    def __init__(self):
+        self._streams = dict()
+        for stream in self.streams:
+            if isinstance(stream, type):
+                stream = stream(self.name)
+            self._streams[stream.name] = stream
+
+    @property
+    def name(self) -> str:
+        """
+        The name of the plugin, defaults to the class name lowercased.
+        """
+        return type(self).__name__.lower().removesuffix('plugin')
+
+    @property
+    def description(self) -> str:
+        return f"{self.name} plugin"
+
+    @property
+    @abstractmethod
+    def connection_type(self) -> Type[msg.Connection]:
+        pass
+
+    @abstractmethod
+    def test_connection(self, connection: msg.Connection):
+        """
+        Test if the the connection with the datasource can be established with the given connection.
+        Must raise an exception if the connection can't be established.
+        """
+        pass
+
+    @property
+    def subtasks(self) -> list[Subtask]:
+        return [subtask for stream in self._streams.values() for subtask in stream.subtasks]
+
+    @abstractmethod
+    def get_scopes(self, scope_name: str, connection: msg.Connection) -> Iterable[DomainModel]:
+        pass
+
+    @property
+    def streams(self) -> list[Union[Stream, Type[Stream]]]:
+        pass
+
+    def collect(self, ctx: Context, stream: str):
+        yield from self.get_stream(stream).collector.run(ctx)
+
+    def extract(self, ctx: Context, stream: str):
+        yield from self.get_stream(stream).extractor.run(ctx)
+
+    def convert(self, ctx: Context, stream: str):
+        yield from self.get_stream(stream).convertor.run(ctx)
+
+    def run_migrations(self, force: bool):
+        # TODO: Create tables
+        pass
+
+    def make_pipeline(self, ctx: Context, scopes: list[msg.BlueprintScope]):
+        """
+        Make a simple pipeline using the scopes declared by the plugin.
+        """
+        stages = [
+            msg.PipelineStage(
+                tasks=[
+                    msg.PipelineTask(
+                        self.name,
+                        skipOnFail=False,
+                        subtasks=[t.name for t in self.subtasks],
+                        options={
+                            "scopeId": scope.id,
+                            "scopeName": scope.name}
+                    )
+                ]
+            )
+            for scope in scopes
+        ]
+
+        plan = msg.PipelinePlan(stages=stages)
+        yield plan
+
+        scopes = [ 
+            msg.PipelineScope(
+                id=':'.join([self.name, type(scope).__name__, ctx.connection_id, bp_scope.id]),
+                name=bp_scope.name,
+                table_name=scope.__tablename__
+            )
+            for bp_scope in scopes
+            for scope in self.get_scopes(bp_scope.name, ctx.connection)
+        ]
+        yield scopes
+
+    def get_stream(self, stream_name: str):
+        stream = self._streams.get(stream_name)
+        if stream is None:
+            raise Exception(f'Unkown stream {stream_name}')
+        return stream
+
+    def startup(self, endpoint: str):
+        details = msg.PluginDetails(
+            plugin_info=self.plugin_info(),
+            swagger=msg.SwaggerDoc(
+                name=self.name,
+                resource=self.name,
+                spec=generate_doc(self.name, self.connection_type)
+            )
+        )
+        resp = requests.post(f"{endpoint}/plugins/register", data=details.json())
+        if resp.status_code != 200:
+            raise Exception(f"unexpected http status code {resp.status_code}: {resp.content}")
+
+    def plugin_info(self):
+        subtask_metas = [
+            msg.SubtaskMeta(
+                name=subtask.name,
+                entry_point_name=subtask.verb,
+                arguments=[subtask.stream.name],
+                required=True,
+                enabled_by_default=True,
+                description=subtask.description,
+                domain_types=[dm.__name__ for dm in subtask.stream.domain_models]
+            )
+            for subtask in self.subtasks
+        ]
+
+        return msg.PluginInfo(
+            name=self.name,
+            description=self.description,
+            plugin_path=self._plugin_path(),
+            extension="datasource",
+            connection_schema=self.connection_type.schema(),
+            subtask_metas=subtask_metas
+        )
+
+    def _plugin_path(self):
+        module_name = type(self).__module__
+        module = sys.modules[module_name]
+        return module.__file__
+
+    @classmethod
+    def start(cls):
+        plugin = cls()
+        fire.Fire(PluginCommands(plugin))
diff --git a/backend/python/pydevlake/pydevlake/stream.py b/backend/python/pydevlake/pydevlake/stream.py
new file mode 100644
index 000000000..7422bd0de
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/stream.py
@@ -0,0 +1,99 @@
+# 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
+
+#     http://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.
+
+
+from typing import Iterable, Type
+from abc import abstractmethod
+
+from pydevlake.subtasks import Collector, Extractor, Convertor, SubstreamCollector
+from pydevlake.model import RawModel, ToolModel, DomainModel
+
+
+class Stream:
+    def __init__(self, plugin_name: str):
+        self.plugin_name = plugin_name
+        self.collector = Collector(self)
+        self.extractor = Extractor(self)
+        self.convertor = Convertor(self)
+        self._raw_model = None
+
+    @property
+    def subtasks(self):
+        return [self.collector, self.extractor, self.convertor]
+
+    @property
+    def name(self):
+        return type(self).__name__.lower()
+
+    @property
+    def qualified_name(self):
+        return f'{self.plugin_name}_{self.name}'
+
+    @property
+    def tool_model(self) -> Type[ToolModel]:
+        pass
+
+    @property
+    def domain_model(self) -> Type[DomainModel]:
+        pass
+
+    @property
+    def domain_models(self) -> Type[DomainModel]:
+        assert self.domain_model, "Streams must declare their domain_model or domain_models"
+        return [self.domain_model]
+
+    def raw_model(self, session) -> Type[RawModel]:
+        if self._raw_model is not None:
+            return self._raw_model
+
+        table_name = f'_raw_{self.plugin_name}_{self.name}'
+
+        # Look for existing raw model
+        for mapper in RawModel._sa_registry.mappers:
+            model = mapper.class_
+            if model.__tablename__ == table_name:
+                self._raw_model = model
+                return self._raw_model
+
+        # Create raw model
+        class StreamRawModel(RawModel, table=True):
+            __tablename__ = table_name
+
+        self._raw_model = StreamRawModel
+        RawModel.metadata.create_all(session.get_bind())
+        return self._raw_model
+
+    def collect(self, state, context) -> Iterable[tuple[object, dict]]:
+        pass
+
+    def extract(self, raw_data: dict) -> ToolModel:
+        return self.tool_model(**raw_data)
+
+    def convert(self, tool_model: ToolModel) -> DomainModel:
+        pass
+
+
+class Substream(Stream):
+    def __init__(self, plugin_name: str):
+        super().__init__(plugin_name)
+        self.collector = SubstreamCollector(self)
+
+    @property
+    @abstractmethod
+    def parent_stream(self):
+        pass
+
+    def collect(self, state, context, parent) -> Iterable[tuple[object, dict]]:
+        pass
diff --git a/backend/python/pydevlake/pydevlake/subtasks.py b/backend/python/pydevlake/pydevlake/subtasks.py
new file mode 100644
index 000000000..98591d331
--- /dev/null
+++ b/backend/python/pydevlake/pydevlake/subtasks.py
@@ -0,0 +1,215 @@
+# 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
+
+#     http://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.
+
+
+from abc import abstractmethod
+import json
+from datetime import datetime
+from typing import Tuple, Dict, Iterable, Optional, Generator
+
+
+import sqlalchemy.sql as sql
+from sqlmodel import Session, SQLModel, Field, select
+
+from pydevlake.model import RawModel, ToolModel, DomainModel, generate_domain_id
+from pydevlake.context import Context
+from pydevlake.message import RemoteProgress
+from pydevlake import logger
+
+
+class Subtask:
+    def __init__(self, stream):
+        self.stream = stream
+
+    @property
+    def name(self):
+        return f'{self.verb.lower()}{self.stream.plugin_name.capitalize()}{self.stream.name.capitalize()}'
+
+    @property
+    def description(self):
+        return f'{self.verb.capitalize()} {self.stream.plugin_name} {self.stream.name.lower()}'
+
+    @property
+    def verb(self) -> str:
+        pass
+
+    def run(self, ctx: Context, sync_point_interval=100):
+        with Session(ctx.engine) as session:
+            subtask_run = self._start_subtask(session, ctx.connection_id)
+            if ctx.incremental:
+                state = self._get_last_state(session, ctx.connection_id)
+            else:
+                self.delete(session, ctx)
+                state = dict()
+
+            try:
+                for i, (data, state) in enumerate(self.fetch(state, session, ctx)):
+                    self.process(data, session, ctx)
+
+                    if i % sync_point_interval == 0 and i != 0:
+                        # Save current state
+                        subtask_run.state = json.dumps(state)
+                        session.merge(subtask_run)
+                        session.commit()
+
+                        # Send progress
+                        yield RemoteProgress(
+                            increment=sync_point_interval,
+                            current=i
+                        )
+            except Exception as e:
+                logger.error(e)
+
+            subtask_run.state = json.dumps(state)
+            subtask_run.completed = datetime.now()
+            session.merge(subtask_run)
+            session.commit()
+
+    def _start_subtask(self, session, connection_id):
+        subtask_run = SubtaskRun(
+            subtask_name=self.name,
+            connection_id=connection_id,
+            started=datetime.now(),
+            state=json.dumps({})
+        )
+        session.add(subtask_run)
+        return subtask_run
+
+    @abstractmethod
+    def fetch(self, state: Dict, session: Session, ctx: Context) -> Iterable[Tuple[object, Dict]]:
+        """
+        Queries the data source and returns an iterable of (data, state) tuples.
+        The `data` can be any object.
+        The `state` is a dict with str keys.
+        `Fetch` is called with the last state of the last run of this subtask.
+        """
+        pass
+
+    @abstractmethod
+    def process(self, data: object, session: Session):
+        """
+        Called for all data entries returned by `fetch`.
+        """
+        pass
+
+    def _get_last_state(self, session, connection_id):
+        stmt = (
+            select(SubtaskRun)
+            .where(SubtaskRun.subtask_name == self.name)
+            .where(SubtaskRun.connection_id == connection_id)
+            .where(SubtaskRun.completed != None)
+            .order_by(SubtaskRun.started)
+        )
+        subtask_run = session.exec(stmt).first()
+        if subtask_run is not None:
+            return json.loads(subtask_run.state)
+        return {}
+
+
+class SubtaskRun(SQLModel, table=True):
+    """
+    Table storing information about the execution of subtasks.
+
+    #TODO: rework id uniqueness:
+    # sync with Keon about the table he created for Singer MR
+    """
+    id: Optional[int] = Field(primary_key=True)
+    subtask_name: str
+    connection_id: int
+    started: datetime
+    completed: Optional[datetime]
+    state: str # JSON encoded dict of atomic values
+
+
+class Collector(Subtask):
+    @property
+    def verb(self):
+        return 'collect'
+
+    def fetch(self, state: Dict, _, ctx: Context) -> Iterable[Tuple[object, Dict]]:
+        return self.stream.collect(state, ctx)
+
+    def process(self, data: object, session: Session, ctx: Context):
+        raw_model_class = self.stream.raw_model(session)
+        raw_model = raw_model_class(
+            params=self._params(ctx),
+            data=json.dumps(data).encode('utf8')
+        )
+        session.add(raw_model)
+
+    def _params(self, ctx: Context) -> str:
+        return json.dumps({
+            "connection_id": ctx.connection_id,
+            "scope_id": ctx.options['scopeId']
+        })
+
+    def delete(self, session, ctx):
+        raw_model = self.stream.raw_model(session)
+        stmt = sql.delete(raw_model).where(raw_model.params == self._params(ctx))
+        session.execute(stmt)
+
+
+class SubstreamCollector(Collector):
+    def fetch(self, state: Dict, session, ctx: Context):
+        for parent in session.exec(sql.select(self.stream.parent_stream.tool_model)).scalars():
+            yield from self.stream.collect(state, ctx, parent)
+
+
+class Extractor(Subtask):
+    @property
+    def verb(self):
+        return 'extract'
+
+    def fetch(self, state: Dict, session: Session, ctx: Context) -> Iterable[Tuple[object, dict]]:
+        raw_model = self.stream.raw_model(session)
+        # TODO: Should filter for same options?
+        for raw in session.query(raw_model).all():
+            yield raw, state
+
+    def process(self, raw: RawModel, session: Session, _):
+        tool_model = self.stream.extract(json.loads(raw.data))
+        tool_model.set_origin(raw)
+        session.merge(tool_model)
+
+    def delete(self, session, ctx):
+        pass
+
+class Convertor(Subtask):
+    @property
+    def verb(self):
+        return 'convert'
+
+    def fetch(self, state: Dict, session: Session, _) -> Iterable[Tuple[ToolModel, Dict]]:
+        for item in session.query(self.stream.tool_model).all():
+            yield item, state
+
+    def process(self, tool_model: ToolModel, session: Session, ctx: Context):
+        res = self.stream.convert(tool_model)
+        if isinstance(res, Generator):
+            for each in self.stream.convert(tool_model):
+                self._save(tool_model, each, session, ctx.connection_id)
+        else:
+            self._save(tool_model, res, session, ctx.connection_id)
+
+    def _save(self, tool_model: ToolModel, domain_model: DomainModel, session: Session, connection_id: int):
+        if not isinstance(domain_model, DomainModel):
+            logger.error(f'Expected a DomainModel but got a {type(domain_model)}: {domain_model}')
+            return
+
+        domain_model.id = generate_domain_id(tool_model, connection_id)
+        session.merge(domain_model)
+
+    def delete(self, session, ctx):
+        pass
diff --git a/.licenserc.yaml b/backend/python/pydevlake/pyproject.toml
similarity index 50%
copy from .licenserc.yaml
copy to backend/python/pydevlake/pyproject.toml
index b11fd8254..e46ee96e5 100644
--- a/.licenserc.yaml
+++ b/backend/python/pydevlake/pyproject.toml
@@ -4,55 +4,37 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
 
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
+[tool.poetry]
+name = "pydevlake"
+version = "0.1.0"
+description = "Devlake plugin framework"
+authors = ["Camille Teruel <ca...@meri.co>", "Keon Amini <ke...@merico.dev>"]
+license = "Apache-2.0"
+readme = "README.md"
+
+[tool.poetry.dependencies]
+python = "^3.10"
+sqlmodel = "^0.0.8"
+mysqlclient = "^2.1.1"
+requests = "^2.28.1"
+inflect = "^6.0.2"
+fire = "^0.4.0"
+pydantic = "^1.10.2"
+pydevd-pycharm = "^231.6471.3"
 
 
+[tool.poetry.group.dev.dependencies]
+pytest = "^7.2.0"
 
-  comment: on-failure
+[build-system]
+requires = ["poetry-core"]
+build-backend = "poetry.core.masonry.api"
diff --git a/backend/python/pydevlake/test/remote_test.go b/backend/python/pydevlake/test/remote_test.go
new file mode 100644
index 000000000..9aaeec877
--- /dev/null
+++ b/backend/python/pydevlake/test/remote_test.go
@@ -0,0 +1,100 @@
+/*
+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
+
+    http://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 test
+
+import (
+	"testing"
+
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/e2ehelper"
+	"github.com/apache/incubator-devlake/server/services/remote"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+	plg "github.com/apache/incubator-devlake/server/services/remote/plugin"
+)
+
+type CircleCIConnection struct {
+	ID    uint64 `json:"id"`
+	Token string `json:"token" encrypt:"yes"`
+}
+
+func CreateRemotePlugin(t *testing.T) models.RemotePlugin {
+	// TODO: Create a dummy plugin for tests instead of using CircleCI plugin
+	pluginCmdPath := "../../plugins/circle_ci/circle_ci/main.py"
+	invoker := bridge.NewPythonPoetryCmdInvoker(pluginCmdPath)
+
+	pluginInfo := models.PluginInfo{}
+	err := invoker.Call("plugin-info", bridge.DefaultContext).Get(&pluginInfo)
+
+	if err != nil {
+		t.Error("Cannot get plugin info", err)
+		return nil
+	}
+
+	remotePlugin, err := remote.NewRemotePlugin(&pluginInfo)
+	if err != nil {
+		t.Error("Cannot create remote plugin", err)
+		return nil
+	}
+
+	return remotePlugin
+}
+
+func TestCreateRemotePlugin(t *testing.T) {
+	_ = CreateRemotePlugin(t)
+}
+
+func TestRunSubTask(t *testing.T) {
+	remotePlugin := CreateRemotePlugin(t)
+	dataflowTester := e2ehelper.NewDataFlowTester(t, "circleci", remotePlugin)
+	subtask := remotePlugin.SubTaskMetas()[0]
+	options := make(map[string]interface{})
+	options["project_slug"] = "gh/circleci/bond"
+	options["scopeId"] = "1"
+	taskData := plg.RemotePluginTaskData{
+		DbUrl:      bridge.DefaultContext.GetConfig("db_url"),
+		Connection: CircleCIConnection{ID: 1},
+		Options:    options,
+	}
+	dataflowTester.Subtask(subtask, taskData)
+}
+
+func TestTestConnection(t *testing.T) {
+	remotePlugin := CreateRemotePlugin(t)
+
+	var handler plugin.ApiResourceHandler
+	for resource, endpoints := range remotePlugin.ApiResources() {
+		if resource == "test" {
+			handler = endpoints["POST"]
+		}
+	}
+
+	if handler == nil {
+		t.Error("Missing test connection API resource")
+	}
+
+	input := plugin.ApiResourceInput{
+		Body: map[string]interface{}{
+			"token": "secret",
+		},
+	}
+	_, err := handler(&input)
+	if err != nil {
+		t.Error(err)
+	}
+}
diff --git a/backend/python/pydevlake/test/stream_test.py b/backend/python/pydevlake/test/stream_test.py
new file mode 100644
index 000000000..db24465a2
--- /dev/null
+++ b/backend/python/pydevlake/test/stream_test.py
@@ -0,0 +1,141 @@
+# 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
+
+#     http://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.
+
+
+import json
+
+import pytest
+from sqlmodel import Session, Field
+
+from pydevlake import Stream, Connection, Context
+from pydevlake.model import ToolModel, DomainModel
+
+
+class DummyToolModel(ToolModel, table=True):
+    id: int = Field(primary_key=True)
+    name: str
+
+
+class DummyDomainModel(DomainModel, table=True):
+    Name: str
+
+
+class DummyStream(Stream):
+    tool_model=DummyToolModel
+    domain_model=DummyDomainModel
+
+    def collect(self, state, context):
+        for i, each in enumerate(context.connection.raw_data):
+            count = state.get("count", 0)
+            yield each, {"count": count+i}
+
+    def extract(self, raw) -> ToolModel:
+        return DummyToolModel(
+            id=raw["i"],
+            name=raw["n"]
+        )
+
+    def convert(self, tm):
+        return DummyDomainModel(
+            ID=tm.id,
+            Name=tm.name,
+        )
+
+
+class DummyConnection(Connection):
+    raw_data: list[dict]
+
+
+@pytest.fixture
+def raw_data():
+    return [
+        {"i": 1, "n": "alice"},
+        {"i": 2, "n": "bob"}
+    ]
+
+
+@pytest.fixture
+def connection(raw_data):
+    return DummyConnection(raw_data=raw_data)
+
+
+@pytest.fixture
+def ctx(connection):
+    return Context(
+        db_url="sqlite+pysqlite:///:memory:",
+        connection_id=11,
+        connection=connection,
+        options={"scopeId": 1, "scopeName": "foo"}
+    )
+
+
+@pytest.fixture
+def stream():
+    return DummyStream("test")
+
+def test_collect_data(stream, raw_data, ctx):
+    gen = stream.collector.run(ctx)
+    list(gen)
+
+    with Session(ctx.engine) as session:
+        raw_model = stream.raw_model(session)
+        all_raw = [json.loads(r.data) for r in session.query(raw_model).all()]
+        assert all_raw == raw_data
+
+
+def test_extract_data(stream, raw_data, ctx):
+    with Session(ctx.engine) as session:
+        for each in raw_data:
+            raw_model = stream.raw_model(session)
+            session.add(raw_model(data=json.dumps(each)))
+        session.commit()
+
+    gen = stream.extractor.run(ctx)
+    list(gen)
+
+    tool_models = session.query(DummyToolModel).all()
+    alice = tool_models[0]
+    bob = tool_models[1]
+    assert alice.name == 'alice'
+    assert alice.id == 1
+
+    assert bob.name == 'bob'
+    assert bob.id == 2
+
+
+def test_convert_data(stream, raw_data, ctx):
+    with Session(ctx.engine) as session:
+        for each in raw_data:
+            session.add(
+                DummyToolModel(
+                    id=each["i"],
+                    name=each["n"],
+                    raw_data_table="_raw_dummy_model",
+                    raw_data_params=json.dumps({"connection_id": ctx.connection_id, "scope_id": ctx.options.scopeId})
+                )
+            )
+        session.commit()
+
+    gen = stream.convertor.run(ctx)
+    list(gen)
+
+    tool_models = session.query(DummyDomainModel).all()
+    alice = tool_models[0]
+    bob = tool_models[1]
+    assert alice.Name == 'alice'
+    assert alice.id == 'test:DummyToolModel:11:1'
+
+    assert bob.Name == 'bob'
+    assert bob.id == 'test:DummyToolModel:11:2'
diff --git a/backend/python/requirements.txt b/backend/python/requirements.txt
index a33f6c928..6575706b4 100644
--- a/backend/python/requirements.txt
+++ b/backend/python/requirements.txt
@@ -1,3 +1,3 @@
+# runtime miscellaneous dependencies
 setuptools
-wheel
-tap-pagerduty==0.2.0
\ No newline at end of file
+wheel
\ No newline at end of file
diff --git a/backend/server/api/api.go b/backend/server/api/api.go
index 71a94c7de..46a2802ed 100644
--- a/backend/server/api/api.go
+++ b/backend/server/api/api.go
@@ -18,18 +18,23 @@ limitations under the License.
 package api
 
 import (
+	"net/http"
+	"strconv"
+	"strings"
+	"time"
+
 	"github.com/apache/incubator-devlake/core/config"
 	"github.com/apache/incubator-devlake/core/errors"
 	"github.com/apache/incubator-devlake/impls/logruslog"
+	_ "github.com/apache/incubator-devlake/server/api/docs"
+	"github.com/apache/incubator-devlake/server/api/remote"
 	"github.com/apache/incubator-devlake/server/api/shared"
 	"github.com/apache/incubator-devlake/server/services"
-	"net/http"
-	"time"
-
-	_ "github.com/apache/incubator-devlake/server/api/docs"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
 
 	"github.com/gin-contrib/cors"
 	"github.com/gin-gonic/gin"
+	"github.com/spf13/viper"
 	ginSwagger "github.com/swaggo/gin-swagger"
 	"github.com/swaggo/gin-swagger/swaggerFiles"
 )
@@ -52,7 +57,10 @@ func CreateApiService() {
 	v := config.GetConfig()
 	gin.SetMode(v.GetString("MODE"))
 	router := gin.Default()
-
+	remotePluginsEnabled := v.GetBool("ENABLE_REMOTE_PLUGINS")
+	if remotePluginsEnabled {
+		router.POST("/plugins/register", remote.RegisterPlugin(router, registerPluginEndpoints))
+	}
 	// Wait for user confirmation if db migration is needed
 	router.GET("/proceed-db-migration", func(ctx *gin.Context) {
 		if !services.MigrationRequireConfirmation() {
@@ -95,8 +103,24 @@ func CreateApiService() {
 	}))
 
 	RegisterRouter(router)
-	err := router.Run(v.GetString("PORT"))
+	port := v.GetString("PORT")
+	if remotePluginsEnabled {
+		go bootstrapRemotePlugins(v)
+	}
+	err := router.Run(port)
 	if err != nil {
 		panic(err)
 	}
 }
+
+func bootstrapRemotePlugins(v *viper.Viper) {
+	port := v.GetString("PORT")
+	portNum, err := strconv.Atoi(strings.Split(port, ":")[1])
+	if err != nil {
+		panic(err)
+	}
+	err = bridge.Bootstrap(v, portNum)
+	if err != nil {
+		logruslog.Global.Error(err, "")
+	}
+}
diff --git a/backend/server/api/remote/models.go b/backend/server/api/remote/models.go
new file mode 100644
index 000000000..079f27cdf
--- /dev/null
+++ b/backend/server/api/remote/models.go
@@ -0,0 +1,35 @@
+/*
+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
+
+    http://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 remote
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+)
+
+type SwaggerDoc struct {
+	Name     string          `json:"name" validate:"required"`
+	Resource string          `json:"resource" validate:"required"`
+	Spec     json.RawMessage `json:"spec" validate:"required"`
+}
+
+type PluginDetails struct {
+	PluginInfo models.PluginInfo `json:"plugin_info" validate:"required"`
+	Swagger    SwaggerDoc        `json:"swagger" validate:"required"`
+}
diff --git a/backend/server/api/remote/register.go b/backend/server/api/remote/register.go
new file mode 100644
index 000000000..1c886027b
--- /dev/null
+++ b/backend/server/api/remote/register.go
@@ -0,0 +1,107 @@
+/*
+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
+
+    http://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 remote
+
+import (
+	"fmt"
+	"net/http"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/server/api/shared"
+	"github.com/apache/incubator-devlake/server/services/remote"
+
+	"github.com/RaveNoX/go-jsonmerge"
+	"github.com/gin-gonic/gin"
+	"github.com/go-playground/validator/v10"
+	ginSwagger "github.com/swaggo/gin-swagger"
+	"github.com/swaggo/gin-swagger/swaggerFiles"
+	"github.com/swaggo/swag"
+)
+
+var (
+	vld        = validator.New()
+	cachedDocs = map[string]*swag.Spec{}
+)
+
+type ApiResource struct {
+	PluginName string
+	Resources  map[string]map[string]plugin.ApiResourceHandler
+}
+
+// TODO add swagger doc
+func RegisterPlugin(router *gin.Engine, registerEndpoints func(r *gin.Engine, pluginName string, apiResources map[string]map[string]plugin.ApiResourceHandler)) func(*gin.Context) {
+	return func(c *gin.Context) {
+		var details PluginDetails
+		if err := c.ShouldBindJSON(&details); err != nil {
+			shared.ApiOutputError(c, errors.BadInput.Wrap(err, shared.BadRequestBody))
+			return
+		}
+		if err := vld.Struct(&details); err != nil {
+			shared.ApiOutputError(c, errors.BadInput.Wrap(err, shared.BadRequestBody))
+			return
+		}
+		remotePlugin, err := remote.NewRemotePlugin(&details.PluginInfo)
+		if err != nil {
+			shared.ApiOutputError(c, errors.Default.Wrap(err, "plugin could not be initialized"))
+			return
+		}
+		resource := ApiResource{
+			PluginName: details.PluginInfo.Name,
+			Resources:  remotePlugin.ApiResources(),
+		}
+		registerEndpoints(router, resource.PluginName, resource.Resources)
+		registerSwagger(router, &details.Swagger)
+		shared.ApiOutputSuccess(c, nil, http.StatusOK)
+	}
+}
+
+func registerSwagger(router *gin.Engine, doc *SwaggerDoc) {
+	if spec, ok := cachedDocs[doc.Name]; ok {
+		spec.SwaggerTemplate = combineSpecs(spec.SwaggerTemplate, string(doc.Spec))
+	} else {
+		spec = &swag.Spec{
+			Version:          "",
+			Host:             "",
+			BasePath:         "",
+			Schemes:          nil,
+			Title:            "",
+			Description:      "",
+			InfoInstanceName: doc.Name,
+			SwaggerTemplate:  string(doc.Spec),
+		}
+		swag.Register(doc.Name, spec)
+		cachedDocs[doc.Name] = spec
+		router.GET(fmt.Sprintf("/plugins/swagger/%s/*any", doc.Resource), ginSwagger.CustomWrapHandler(
+			&ginSwagger.Config{
+				URL:                      "doc.json",
+				DocExpansion:             "list",
+				InstanceName:             doc.Name,
+				Title:                    "",
+				DefaultModelsExpandDepth: 1,
+				DeepLinking:              true,
+				PersistAuthorization:     false,
+			},
+			swaggerFiles.Handler))
+	}
+}
+
+func combineSpecs(spec1 string, spec2 string) string {
+	res, _ := jsonmerge.Merge(spec1, spec2)
+	return res.(string)
+}
diff --git a/backend/server/api/router.go b/backend/server/api/router.go
index 41bf887a8..8408a7af0 100644
--- a/backend/server/api/router.go
+++ b/backend/server/api/router.go
@@ -19,6 +19,9 @@ package api
 
 import (
 	"fmt"
+	"net/http"
+	"strings"
+
 	"github.com/apache/incubator-devlake/core/plugin"
 	"github.com/apache/incubator-devlake/server/api/blueprints"
 	"github.com/apache/incubator-devlake/server/api/domainlayer"
@@ -31,8 +34,6 @@ import (
 	"github.com/apache/incubator-devlake/server/api/task"
 	"github.com/apache/incubator-devlake/server/api/version"
 	"github.com/apache/incubator-devlake/server/services"
-	"net/http"
-	"strings"
 
 	"github.com/gin-gonic/gin"
 )
@@ -73,24 +74,29 @@ func RegisterRouter(r *gin.Engine) {
 	r.GET("/projects", project.GetProjects)
 
 	// mount all api resources for all plugins
-	pluginsApiResources, err := services.GetPluginsApiResources()
+	resources, err := services.GetPluginsApiResources()
 	if err != nil {
 		panic(err)
 	}
-	for pluginName, apiResources := range pluginsApiResources {
-		for resourcePath, resourceHandlers := range apiResources {
-			for method, h := range resourceHandlers {
-				r.Handle(
-					method,
-					fmt.Sprintf("/plugins/%s/%s", pluginName, resourcePath),
-					handlePluginCall(pluginName, h),
-				)
-			}
+	// mount all api resources for all plugins
+	for pluginName, apiResources := range resources {
+		registerPluginEndpoints(r, pluginName, apiResources)
+	}
+}
+
+func registerPluginEndpoints(r *gin.Engine, pluginName string, apiResources map[string]map[string]plugin.ApiResourceHandler) {
+	for resourcePath, resourceHandlers := range apiResources {
+		for method, h := range resourceHandlers {
+			r.Handle(
+				method,
+				fmt.Sprintf("/plugins/%s/%s", pluginName, resourcePath),
+				handlePluginCall(h),
+			)
 		}
 	}
 }
 
-func handlePluginCall(pluginName string, handler plugin.ApiResourceHandler) func(c *gin.Context) {
+func handlePluginCall(handler plugin.ApiResourceHandler) func(c *gin.Context) {
 	return func(c *gin.Context) {
 		var err error
 		input := &plugin.ApiResourceInput{}
diff --git a/backend/server/services/remote/bridge/bootstrap.go b/backend/server/services/remote/bridge/bootstrap.go
new file mode 100644
index 000000000..f686243f9
--- /dev/null
+++ b/backend/server/services/remote/bridge/bootstrap.go
@@ -0,0 +1,59 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"fmt"
+	"os"
+	"os/exec"
+	"path/filepath"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/utils"
+	"github.com/apache/incubator-devlake/impls/logruslog"
+	"github.com/spf13/viper"
+)
+
+func Bootstrap(cfg *viper.Viper, port int) errors.Error {
+	scriptPath := cfg.GetString("REMOTE_PLUGINS_STARTUP_PATH")
+	if scriptPath == "" {
+		return errors.BadInput.New(fmt.Sprintf("missing env key: %s", "REMOTE_PLUGINS_STARTUP_PATH"))
+	}
+	workingDir, err := errors.Convert01(os.Getwd())
+	if err != nil {
+		return err
+	}
+	absScriptPath := filepath.Join(workingDir, scriptPath)
+	cmd := exec.Command(absScriptPath, fmt.Sprintf("http://127.0.0.1:%d", port)) //expects the plugins to live on the same host
+	cmd.Dir = filepath.Dir(absScriptPath)
+	result, err := utils.RunProcess(cmd, &utils.RunProcessOptions{
+		OnStdout: func(b []byte) {
+			logruslog.Global.Info(string(b))
+		},
+		OnStderr: func(b []byte) {
+			logruslog.Global.Error(nil, string(b))
+		},
+	})
+	if err != nil {
+		return err
+	}
+	if result.GetError() != nil {
+		logruslog.Global.Error(result.GetError(), "error occurred bootstrapping remote plugins")
+	}
+	return nil
+}
diff --git a/backend/server/services/remote/bridge/bridge.go b/backend/server/services/remote/bridge/bridge.go
new file mode 100644
index 000000000..2d80685dd
--- /dev/null
+++ b/backend/server/services/remote/bridge/bridge.go
@@ -0,0 +1,64 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+)
+
+type (
+	Bridge struct {
+		invoker Invoker
+	}
+	Invoker interface {
+		Call(methodName string, ctx plugin.ExecContext, args ...any) *CallResult
+		Stream(methodName string, ctx plugin.ExecContext, args ...any) *MethodStream
+	}
+)
+
+func NewBridge(invoker Invoker) *Bridge {
+	return &Bridge{invoker: invoker}
+}
+
+func (b *Bridge) RemoteSubtaskEntrypointHandler(subtaskMeta models.SubtaskMeta) plugin.SubTaskEntryPoint {
+	return func(ctx plugin.SubTaskContext) errors.Error {
+		args := []interface{}{ctx.GetData()}
+		for _, arg := range subtaskMeta.Arguments {
+			args = append(args, arg)
+		}
+		stream := b.invoker.Stream(subtaskMeta.EntryPointName, NewChildRemoteContext(ctx), args...)
+		for recv := range stream.Receive() {
+			if recv.err != nil {
+				return recv.err
+			}
+			progress := RemoteProgress{}
+			err := recv.Get(&progress)
+			if err != nil {
+				return err
+			}
+			if progress.Current != 0 {
+				ctx.SetProgress(progress.Current, progress.Total)
+			} else if progress.Increment != 0 {
+				ctx.IncProgress(progress.Increment)
+			}
+		}
+		return nil
+	}
+}
diff --git a/backend/server/services/remote/bridge/cmd.go b/backend/server/services/remote/bridge/cmd.go
new file mode 100644
index 000000000..b3a14371a
--- /dev/null
+++ b/backend/server/services/remote/bridge/cmd.go
@@ -0,0 +1,156 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"fmt"
+	"os/exec"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/log"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/core/utils"
+)
+
+type CmdInvoker struct {
+	resolveCmd  func(methodName string, args ...string) (string, []string)
+	cancelled   bool
+	workingPath string
+}
+
+func NewCmdInvoker(workingPath string, resolveCmd func(methodName string, args ...string) (string, []string)) *CmdInvoker {
+	return &CmdInvoker{
+		resolveCmd:  resolveCmd,
+		workingPath: workingPath,
+	}
+}
+
+func (c *CmdInvoker) Call(methodName string, ctx plugin.ExecContext, args ...any) *CallResult {
+	serializedArgs, err := serialize(args...)
+	if err != nil {
+		return &CallResult{
+			err: err,
+		}
+	}
+	executable, inputArgs := c.resolveCmd(methodName, serializedArgs...)
+	cmdCtx := DefaultContext.GetContext()
+	cmd := exec.CommandContext(cmdCtx, executable, inputArgs...)
+	if c.workingPath != "" {
+		cmd.Dir = c.workingPath
+	}
+	response, err := utils.RunProcess(cmd, &utils.RunProcessOptions{
+		OnStdout: func(b []byte) {
+			msg := string(b)
+			c.logRemoteMessage(ctx.GetLogger(), msg)
+		},
+		OnStderr: func(b []byte) {
+			msg := string(b)
+			c.logRemoteError(ctx.GetLogger(), msg)
+		},
+		UseFdOut: true,
+	})
+	if err != nil {
+		return NewCallResult(nil, err)
+	}
+	err = response.GetError()
+	if err != nil {
+		return &CallResult{
+			err: errors.Default.Wrap(err, fmt.Sprintf("failed to invoke remote function \"%s\"", methodName)),
+		}
+	}
+	results, err := deserialize(response.GetFdOut().([]byte))
+	return NewCallResult(results, errors.Convert(err))
+}
+
+func (c *CmdInvoker) Stream(methodName string, ctx plugin.ExecContext, args ...any) *MethodStream {
+	recvChannel := make(chan *StreamResult)
+	stream := &MethodStream{
+		outbound: nil,
+		inbound:  recvChannel,
+	}
+	serializedArgs, err := serialize(args...)
+	if err != nil {
+		recvChannel <- NewStreamResult(nil, err)
+		return stream
+	}
+	executable, inputArgs := c.resolveCmd(methodName, serializedArgs...)
+	cmdCtx := DefaultContext.GetContext() // grabbing context off of ctx kills the cmd after a couple of seconds... why?
+	cmd := exec.CommandContext(cmdCtx, executable, inputArgs...)
+	if c.workingPath != "" {
+		cmd.Dir = c.workingPath
+	}
+	processHandle, err := utils.StreamProcess(cmd, &utils.StreamProcessOptions{
+		OnStdout: func(b []byte) (any, errors.Error) {
+			msg := string(b)
+			c.logRemoteMessage(ctx.GetLogger(), msg)
+			return b, nil
+		},
+		OnStderr: func(b []byte) (any, errors.Error) {
+			msg := string(b)
+			c.logRemoteError(ctx.GetLogger(), msg)
+			return b, nil
+		},
+		UseFdOut: true,
+		OnFdOut:  utils.NoopConverter,
+	})
+	if err != nil {
+		recvChannel <- NewStreamResult(nil, err)
+		return stream
+	}
+	go func() {
+		defer close(recvChannel)
+		for msg := range processHandle.Receive() {
+			if err = msg.GetError(); err != nil {
+				recvChannel <- NewStreamResult(nil, err)
+			}
+			if !c.cancelled {
+				select {
+				case <-ctx.GetContext().Done():
+					err = processHandle.Cancel()
+					if err != nil {
+						recvChannel <- NewStreamResult(nil, errors.Default.Wrap(err, "error cancelling python target"))
+						return
+					}
+					c.cancelled = true
+					// continue until the stream gets closed by the child
+				default:
+				}
+			}
+			response := msg.GetFdOut()
+			if response != nil {
+				results, err := deserialize(response.([]byte))
+				if err != nil {
+					recvChannel <- NewStreamResult(nil, err)
+				} else {
+					recvChannel <- NewStreamResult(results, nil)
+				}
+			}
+		}
+	}()
+	return stream
+}
+
+func (c *CmdInvoker) logRemoteMessage(logger log.Logger, msg string) {
+	logger.Info(msg)
+}
+
+func (c *CmdInvoker) logRemoteError(logger log.Logger, msg string) {
+	logger.Error(nil, msg)
+}
+
+var _ Invoker = (*CmdInvoker)(nil)
diff --git a/backend/server/services/remote/bridge/context.go b/backend/server/services/remote/bridge/context.go
new file mode 100644
index 000000000..57dd78dae
--- /dev/null
+++ b/backend/server/services/remote/bridge/context.go
@@ -0,0 +1,140 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"context"
+
+	"github.com/apache/incubator-devlake/core/config"
+	ctx "github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/dal"
+	"github.com/apache/incubator-devlake/core/log"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/impls/logruslog"
+	"github.com/spf13/cast"
+	"github.com/spf13/viper"
+)
+
+var DefaultContext = NewRemoteContext(logruslog.Global, config.GetConfig())
+
+type RemoteProgress struct {
+	Current   int `json:"current"`
+	Total     int `json:"total"`
+	Increment int `json:"increment"`
+}
+
+type RemoteContext interface {
+	plugin.ExecContext
+	GetSettings() map[string]any
+}
+
+type remoteContextImpl struct {
+	parent   plugin.ExecContext
+	logger   log.Logger
+	ctx      context.Context
+	Settings map[string]any `json:"settings"`
+}
+
+func (r remoteContextImpl) GetSettings() map[string]any {
+	return r.Settings
+}
+
+func (r remoteContextImpl) GetConfigReader() config.ConfigReader {
+	return r.parent.GetConfigReader()
+}
+
+func (r remoteContextImpl) ReplaceLogger(logger log.Logger) ctx.BasicRes {
+	return &remoteContextImpl{
+		parent:   r.parent,
+		logger:   logger,
+		ctx:      r.ctx,
+		Settings: r.Settings,
+	}
+}
+
+func (r remoteContextImpl) NestedLogger(name string) ctx.BasicRes {
+	return r.ReplaceLogger(r.logger.Nested(name))
+}
+
+func NewRemoteContext(logger log.Logger, cfg *viper.Viper) RemoteContext {
+	return &remoteContextImpl{
+		logger:   logger,
+		Settings: cfg.AllSettings(),
+		ctx:      context.Background(),
+	}
+}
+
+func NewChildRemoteContext(ec plugin.ExecContext) RemoteContext {
+	return &remoteContextImpl{
+		parent:   ec,
+		logger:   ec.GetLogger(),
+		ctx:      ec.GetContext(),
+		Settings: DefaultContext.GetSettings(),
+	}
+}
+
+func (r remoteContextImpl) GetConfig(name string) string {
+	val, ok := r.Settings[name]
+	if !ok {
+		return ""
+	}
+	return cast.ToString(val)
+}
+
+func (r remoteContextImpl) GetLogger() log.Logger {
+	return r.logger
+}
+
+func (r remoteContextImpl) GetDal() dal.Dal {
+	if r.parent != nil {
+		return r.parent.GetDal()
+	}
+	return nil
+}
+
+func (r remoteContextImpl) GetName() string {
+	if r.parent != nil {
+		return r.parent.GetName()
+	}
+	return "default_remote"
+}
+
+func (r remoteContextImpl) GetContext() context.Context {
+	return r.ctx
+}
+
+func (r remoteContextImpl) GetData() interface{} {
+	if r.parent != nil {
+		return r.parent.GetData()
+	}
+	return nil
+}
+
+func (r remoteContextImpl) SetProgress(current int, total int) {
+	if r.parent != nil {
+		r.parent.SetProgress(current, total)
+	}
+}
+
+func (r remoteContextImpl) IncProgress(quantity int) {
+	if r.parent != nil {
+		r.parent.IncProgress(quantity)
+	}
+}
+
+var _ RemoteContext = (*remoteContextImpl)(nil)
diff --git a/backend/server/services/remote/bridge/python_cmd.go b/backend/server/services/remote/bridge/python_cmd.go
new file mode 100644
index 000000000..bdcebcd82
--- /dev/null
+++ b/backend/server/services/remote/bridge/python_cmd.go
@@ -0,0 +1,38 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"path/filepath"
+	"strings"
+)
+
+const (
+	poetryExec = "poetry"
+	pythonExec = "python"
+)
+
+func NewPythonPoetryCmdInvoker(scriptPath string) *CmdInvoker {
+	tomlPath := filepath.Dir(filepath.Dir(scriptPath)) //the main entrypoint expected to be at toplevel
+	scriptPath = strings.TrimPrefix(scriptPath, tomlPath+"/")
+	return NewCmdInvoker(tomlPath, func(methodName string, args ...string) (string, []string) {
+		allArgs := []string{"run", pythonExec, scriptPath, methodName}
+		allArgs = append(allArgs, args...)
+		return poetryExec, allArgs
+	})
+}
diff --git a/backend/server/services/remote/bridge/returns.go b/backend/server/services/remote/bridge/returns.go
new file mode 100644
index 000000000..b7aa46fb6
--- /dev/null
+++ b/backend/server/services/remote/bridge/returns.go
@@ -0,0 +1,88 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/mitchellh/mapstructure"
+)
+
+type (
+	CallResult struct {
+		results []map[string]any
+		err     errors.Error
+	}
+	StreamResult = CallResult
+	MethodStream struct {
+		// send (not supported for now)
+		outbound chan<- any
+		// receive
+		inbound <-chan *StreamResult
+	}
+)
+
+func NewCallResult(results []map[string]any, err errors.Error) *CallResult {
+	return &CallResult{
+		results: results,
+		err:     err,
+	}
+}
+
+func (m *CallResult) Get(targets ...any) errors.Error {
+	if m.err != nil {
+		return m.err
+	}
+	if len(targets) != len(m.results) {
+		// if everything came back as nil, consider it good
+		for _, result := range m.results {
+			if result != nil {
+				return errors.Default.New("unequal results and targets length")
+			}
+		}
+		return nil
+	}
+
+	for i, target := range targets {
+		config := &mapstructure.DecoderConfig{
+			TagName: "json",
+			Result:  target,
+		}
+
+		decoder, err := mapstructure.NewDecoder(config)
+		if err != nil {
+			return errors.Convert(err)
+		}
+
+		err = decoder.Decode(m.results[i])
+		if err != nil {
+			return errors.Convert(err)
+		}
+	}
+	return nil
+}
+
+func NewStreamResult(results []map[string]any, err errors.Error) *StreamResult {
+	return &StreamResult{
+		results: results,
+		err:     err,
+	}
+}
+
+func (m *MethodStream) Receive() <-chan *StreamResult {
+	return m.inbound
+}
diff --git a/backend/server/services/remote/bridge/utils.go b/backend/server/services/remote/bridge/utils.go
new file mode 100644
index 000000000..542074e87
--- /dev/null
+++ b/backend/server/services/remote/bridge/utils.go
@@ -0,0 +1,57 @@
+/*
+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
+
+    http://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 bridge
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/core/errors"
+)
+
+func serialize(args ...any) ([]string, errors.Error) {
+	var serializedArgs []string
+	for _, arg := range args {
+		serializedArg, err := json.Marshal(arg)
+		if err != nil {
+			return nil, errors.Convert(err)
+		}
+		serializedArgs = append(serializedArgs, string(serializedArg))
+	}
+	return serializedArgs, nil
+}
+
+func deserialize(bytes json.RawMessage) ([]map[string]any, errors.Error) {
+	if len(bytes) == 0 {
+		return nil, nil
+	}
+	var result []map[string]any
+	if bytes[0] == '{' {
+		single := make(map[string]any)
+		if err := json.Unmarshal(bytes, &single); err != nil {
+			return nil, errors.Convert(err)
+		}
+		result = append(result, single)
+	} else if bytes[0] == '[' {
+		if err := json.Unmarshal(bytes, &result); err != nil {
+			return nil, errors.Convert(err)
+		}
+	} else {
+		return nil, errors.Default.New("malformed JSON from remote call")
+	}
+	return result, nil
+}
diff --git a/backend/server/services/remote/init.go b/backend/server/services/remote/init.go
new file mode 100644
index 000000000..98d46770d
--- /dev/null
+++ b/backend/server/services/remote/init.go
@@ -0,0 +1,56 @@
+/*
+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
+
+    http://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 remote
+
+import (
+	"github.com/apache/incubator-devlake/core/config"
+	"github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/errors"
+	pluginCore "github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+	remote "github.com/apache/incubator-devlake/server/services/remote/plugin"
+)
+
+var (
+	remotePlugins = make(map[string]models.RemotePlugin)
+)
+
+func Init(br context.BasicRes) {
+	remote.Init(br)
+}
+
+func NewRemotePlugin(info *models.PluginInfo) (models.RemotePlugin, errors.Error) {
+	if _, ok := remotePlugins[info.Name]; ok {
+		return nil, errors.BadInput.New("plugin already registered")
+	}
+	plugin, err := remote.NewRemotePlugin(info)
+	if err != nil {
+		return nil, errors.BadInput.New("unsupported plugin type")
+	}
+	forceMigration := config.GetConfig().GetBool("FORCE_MIGRATION")
+	err = plugin.RunMigrations(forceMigration)
+	if err != nil {
+		return nil, err
+	}
+	err = pluginCore.RegisterPlugin(info.Name, plugin)
+	if err != nil {
+		return nil, err
+	}
+	remotePlugins[info.Name] = plugin
+	return plugin, nil
+}
diff --git a/backend/server/services/remote/models/conversion.go b/backend/server/services/remote/models/conversion.go
new file mode 100644
index 000000000..5b07efdae
--- /dev/null
+++ b/backend/server/services/remote/models/conversion.go
@@ -0,0 +1,100 @@
+/*
+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
+
+    http://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 models
+
+import (
+	"fmt"
+	"reflect"
+	"strings"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/models/common"
+)
+
+func LoadTableModel(tableName string, schema map[string]any) (*models.DynamicTabler, errors.Error) {
+	structType, err := GenerateStructType(schema, reflect.TypeOf(common.Model{}))
+	if err != nil {
+		return nil, err
+	}
+	return models.NewDynamicTabler(tableName, structType), nil
+}
+
+func GenerateStructType(schema map[string]any, baseType reflect.Type) (reflect.Type, errors.Error) {
+	var structFields []reflect.StructField
+	propsRaw, ok := schema["properties"]
+	if !ok {
+		return nil, errors.BadInput.New("Missing properties in JSON schema")
+	}
+	props, ok := propsRaw.(map[string]any)
+	if !ok {
+		return nil, errors.BadInput.New("JSON schema properties must be an object")
+	}
+	if baseType != nil {
+		anonymousField := reflect.StructField{
+			Name:      baseType.Name(),
+			Type:      baseType,
+			Tag:       reflect.StructTag("mapstructure:\",squash\""),
+			Anonymous: true,
+		}
+		structFields = append(structFields, anonymousField)
+	}
+	for k, v := range props {
+		spec := v.(map[string]any)
+		field, err := generateStructField(k, spec)
+		if err != nil {
+			return nil, err
+		}
+		structFields = append(structFields, *field)
+	}
+	return reflect.StructOf(structFields), nil
+}
+
+func generateStructField(name string, schema map[string]any) (*reflect.StructField, errors.Error) {
+	tag := reflect.StructTag(fmt.Sprintf("json:\"%s\" "+
+		"gorm:\"serializer:encdec\"", //just encrypt everything for GORM operations - makes things easy
+		name))
+	goType, err := getGoType(schema)
+	if err != nil {
+		return nil, err
+	}
+	return &reflect.StructField{
+		Name: strings.Title(name), //nolint:staticcheck
+		Type: goType,
+		Tag:  tag,
+	}, nil
+}
+
+func getGoType(schema map[string]any) (reflect.Type, errors.Error) {
+	var goType reflect.Type
+	jsonType, ok := schema["type"].(string)
+	if !ok {
+		return nil, errors.BadInput.New("\"type\" property must be a string")
+	}
+	switch jsonType {
+	//TODO: support more types
+	case "integer":
+		goType = reflect.TypeOf(uint64(0))
+	case "string":
+		//TODO: distinguish stypes based on string format
+		goType = reflect.TypeOf("")
+	default:
+		return nil, errors.BadInput.New(fmt.Sprintf("Unsupported type %s", jsonType))
+	}
+	return goType, nil
+}
diff --git a/backend/server/services/remote/models/models.go b/backend/server/services/remote/models/models.go
new file mode 100644
index 000000000..d19a26fe7
--- /dev/null
+++ b/backend/server/services/remote/models/models.go
@@ -0,0 +1,58 @@
+/*
+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
+
+    http://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 models
+
+const (
+	PythonPoetryCmd PluginType      = "python-poetry"
+	PythonCmd       PluginType      = "python"
+	None            PluginExtension = ""
+	Metric          PluginExtension = "metric"
+	Datasource      PluginExtension = "datasource"
+)
+
+type (
+	PluginType      string
+	PluginExtension string
+)
+
+type PluginInfo struct {
+	Type             PluginType      `json:"type" validate:"required"`
+	Name             string          `json:"name" validate:"required"`
+	Extension        PluginExtension `json:"extension"`
+	ConnectionSchema map[string]any  `json:"connection_schema" validate:"required"`
+	Description      string          `json:"description"`
+	PluginPath       string          `json:"plugin_path" validate:"required"`
+	ApiEndpoints     []Endpoint      `json:"api_endpoints" validate:"dive"`
+	SubtaskMetas     []SubtaskMeta   `json:"subtask_metas" validate:"dive"`
+}
+
+type SubtaskMeta struct {
+	Name             string   `json:"name" validate:"required"`
+	EntryPointName   string   `json:"entry_point_name" validate:"required"`
+	Arguments        []string `json:"arguments"`
+	Required         bool     `json:"required"`
+	EnabledByDefault bool     `json:"enabled_by_default"`
+	Description      string   `json:"description" validate:"required"`
+	DomainTypes      []string `json:"domain_types" validate:"required"`
+}
+
+type Endpoint struct {
+	Resource string `json:"resource" validate:"required"`
+	Handler  string `json:"handler" validate:"required"`
+	Method   string `json:"method" validate:"required"`
+}
diff --git a/backend/server/services/remote/models/plugin_models.go b/backend/server/services/remote/models/plugin_models.go
new file mode 100644
index 000000000..4284112d6
--- /dev/null
+++ b/backend/server/services/remote/models/plugin_models.go
@@ -0,0 +1,41 @@
+/*
+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
+
+    http://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 models
+
+type (
+	PipelineScope struct {
+		ScopeId   string
+		ScopeName string
+		TableName string
+	}
+	WrappedPipelineScope struct {
+		Scope PipelineScope
+	}
+)
+
+func (p *WrappedPipelineScope) ScopeId() string {
+	return p.Scope.ScopeId
+}
+
+func (p *WrappedPipelineScope) ScopeName() string {
+	return p.Scope.ScopeName
+}
+
+func (p *WrappedPipelineScope) TableName() string {
+	return p.Scope.TableName
+}
diff --git a/backend/server/services/remote/models/plugin_remote.go b/backend/server/services/remote/models/plugin_remote.go
new file mode 100644
index 000000000..72f4a45bf
--- /dev/null
+++ b/backend/server/services/remote/models/plugin_remote.go
@@ -0,0 +1,31 @@
+/*
+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
+
+    http://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 models
+
+import (
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+)
+
+// RemotePlugin API supported by plugins running in different/remote processes
+type RemotePlugin interface {
+	plugin.PluginApi
+	plugin.PluginTask
+	plugin.PluginMeta
+	RunMigrations(forceMigrate bool) errors.Error
+}
diff --git a/backend/server/services/remote/plugin/connection_api.go b/backend/server/services/remote/plugin/connection_api.go
new file mode 100644
index 000000000..94b09fbee
--- /dev/null
+++ b/backend/server/services/remote/plugin/connection_api.go
@@ -0,0 +1,92 @@
+/*
+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
+
+    http://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 plugin
+
+import (
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"net/http"
+)
+
+type ConnectionAPI struct {
+	invoker  bridge.Invoker
+	connType *models.DynamicTabler
+	helper   *api.ConnectionApiHelper
+}
+
+func (c *ConnectionAPI) TestConnection(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	err := c.invoker.Call("test-connection", bridge.DefaultContext, input.Body).Get()
+	if err != nil {
+		return nil, err
+	}
+	return nil, nil
+}
+
+func (c *ConnectionAPI) PostConnections(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connection := c.connType.New()
+	err := c.helper.Create(connection, input)
+	if err != nil {
+		return nil, err
+	}
+	conn := connection.Unwrap()
+	return &plugin.ApiResourceOutput{Body: conn, Status: http.StatusOK}, nil
+}
+
+func (c *ConnectionAPI) ListConnections(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connections := c.connType.NewSlice()
+	err := c.helper.List(connections)
+	if err != nil {
+		return nil, err
+	}
+	conns := connections.Unwrap()
+	return &plugin.ApiResourceOutput{Body: conns}, nil
+}
+
+func (c *ConnectionAPI) GetConnection(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connection := c.connType.New()
+	err := c.helper.First(connection, input.Params)
+	if err != nil {
+		return nil, err
+	}
+	conn := connection.Unwrap()
+	return &plugin.ApiResourceOutput{Body: conn}, nil
+}
+
+func (c *ConnectionAPI) PatchConnection(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connection := c.connType.New()
+	err := c.helper.Patch(connection, input)
+	if err != nil {
+		return nil, err
+	}
+	conn := connection.Unwrap()
+	return &plugin.ApiResourceOutput{Body: conn, Status: http.StatusOK}, nil
+}
+
+func (c *ConnectionAPI) DeleteConnection(input *plugin.ApiResourceInput) (*plugin.ApiResourceOutput, errors.Error) {
+	connection := c.connType.New()
+	err := c.helper.First(connection, input.Params)
+	if err != nil {
+		return nil, err
+	}
+	err = c.helper.Delete(connection)
+	conn := connection.Unwrap()
+	return &plugin.ApiResourceOutput{Body: conn}, err
+}
diff --git a/backend/server/services/remote/plugin/default_api.go b/backend/server/services/remote/plugin/default_api.go
new file mode 100644
index 000000000..a2d500b28
--- /dev/null
+++ b/backend/server/services/remote/plugin/default_api.go
@@ -0,0 +1,47 @@
+/*
+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
+
+    http://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 plugin
+
+import (
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+)
+
+func GetDefaultAPI(invoker bridge.Invoker, connType *models.DynamicTabler, helper *api.ConnectionApiHelper) map[string]map[string]plugin.ApiResourceHandler {
+	connectionApi := &ConnectionAPI{
+		invoker:  invoker,
+		connType: connType,
+		helper:   helper,
+	}
+	return map[string]map[string]plugin.ApiResourceHandler{
+		"test": {
+			"POST": connectionApi.TestConnection,
+		},
+		"connections": {
+			"POST": connectionApi.PostConnections,
+			"GET":  connectionApi.ListConnections,
+		},
+		"connections/:connectionId": {
+			"GET":    connectionApi.GetConnection,
+			"PATCH":  connectionApi.PatchConnection,
+			"DELETE": connectionApi.DeleteConnection,
+		},
+	}
+}
diff --git a/backend/server/services/remote/plugin/init.go b/backend/server/services/remote/plugin/init.go
new file mode 100644
index 000000000..1ddd03001
--- /dev/null
+++ b/backend/server/services/remote/plugin/init.go
@@ -0,0 +1,61 @@
+/*
+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
+
+    http://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 plugin
+
+import (
+	"github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+	"github.com/go-playground/validator/v10"
+)
+
+var (
+	connectionHelper *api.ConnectionApiHelper
+	basicRes         context.BasicRes
+)
+
+func Init(br context.BasicRes) {
+	vld := validator.New()
+	basicRes = br
+	connectionHelper = api.NewConnectionHelper(
+		br,
+		vld,
+	)
+}
+
+func NewRemotePlugin(info *models.PluginInfo) (models.RemotePlugin, errors.Error) {
+	invoker := bridge.NewPythonPoetryCmdInvoker(info.PluginPath)
+	plugin, err := newPlugin(info, invoker)
+
+	if err != nil {
+		return nil, err
+	}
+
+	switch info.Extension {
+	case models.None:
+		return plugin, nil
+	case models.Metric:
+		return remoteMetricPlugin{plugin}, nil
+	case models.Datasource:
+		return remoteDatasourcePlugin{plugin}, nil
+	default:
+		return nil, errors.BadInput.New("unsupported plugin extension")
+	}
+}
diff --git a/backend/server/services/remote/plugin/plugin_extensions.go b/backend/server/services/remote/plugin/plugin_extensions.go
new file mode 100644
index 000000000..537e4640f
--- /dev/null
+++ b/backend/server/services/remote/plugin/plugin_extensions.go
@@ -0,0 +1,64 @@
+/*
+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
+
+    http://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 plugin
+
+import (
+	"encoding/json"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+)
+
+type (
+	remoteMetricPlugin struct {
+		*remotePluginImpl
+	}
+	remoteDatasourcePlugin struct {
+		*remotePluginImpl
+	}
+)
+
+func (p *remoteMetricPlugin) MakeMetricPluginPipelinePlanV200(projectName string, options json.RawMessage) (plugin.PipelinePlan, errors.Error) {
+	plan := plugin.PipelinePlan{}
+	err := p.invoker.Call("MakeMetricPluginPipelinePlanV200", bridge.DefaultContext, projectName, options).Get(&plan)
+	if err != nil {
+		return nil, err
+	}
+	return plan, err
+}
+
+func (p *remoteDatasourcePlugin) MakeDataSourcePipelinePlanV200(connectionId uint64, bpScopes []*plugin.BlueprintScopeV200, syncPolicy plugin.BlueprintSyncPolicy) (plugin.PipelinePlan, []plugin.Scope, errors.Error) {
+	plan := plugin.PipelinePlan{}
+	var scopes []models.PipelineScope
+	err := p.invoker.Call("make-pipeline", bridge.DefaultContext, connectionId, bpScopes).Get(&plan, &scopes)
+	if err != nil {
+		return nil, nil, err
+	}
+	var castedScopes []plugin.Scope
+	for _, scope := range scopes {
+		castedScopes = append(castedScopes, &models.WrappedPipelineScope{Scope: scope})
+	}
+	return plan, castedScopes, nil
+}
+
+var _ models.RemotePlugin = (*remoteMetricPlugin)(nil)
+var _ plugin.MetricPluginBlueprintV200 = (*remoteMetricPlugin)(nil)
+var _ models.RemotePlugin = (*remoteDatasourcePlugin)(nil)
+var _ plugin.DataSourcePluginBlueprintV200 = (*remoteDatasourcePlugin)(nil)
diff --git a/backend/server/services/remote/plugin/plugin_impl.go b/backend/server/services/remote/plugin/plugin_impl.go
new file mode 100644
index 000000000..7e611b283
--- /dev/null
+++ b/backend/server/services/remote/plugin/plugin_impl.go
@@ -0,0 +1,131 @@
+/*
+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
+
+    http://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 plugin
+
+import (
+	"fmt"
+
+	"github.com/apache/incubator-devlake/core/errors"
+	coreModels "github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+)
+
+type (
+	remotePluginImpl struct {
+		name             string
+		subtaskMetas     []plugin.SubTaskMeta
+		pluginPath       string
+		description      string
+		invoker          bridge.Invoker
+		connectionTabler *coreModels.DynamicTabler
+		resources        map[string]map[string]plugin.ApiResourceHandler
+	}
+	RemotePluginTaskData struct {
+		DbUrl        string                 `json:"db_url"`
+		ConnectionId uint64                 `json:"connection_id"`
+		Connection   interface{}            `json:"connection"`
+		Options      map[string]interface{} `json:"options"`
+	}
+)
+
+func newPlugin(info *models.PluginInfo, invoker bridge.Invoker) (*remotePluginImpl, errors.Error) {
+	connectionTableName := fmt.Sprintf("_tool_%s_connections", info.Name)
+	dynamicTabler, err := models.LoadTableModel(connectionTableName, info.ConnectionSchema)
+	if err != nil {
+		return nil, err
+	}
+	p := remotePluginImpl{
+		name:             info.Name,
+		invoker:          invoker,
+		pluginPath:       info.PluginPath,
+		description:      info.Description,
+		connectionTabler: dynamicTabler,
+		resources:        GetDefaultAPI(invoker, dynamicTabler, connectionHelper),
+	}
+	remoteBridge := bridge.NewBridge(invoker)
+	for _, subtask := range info.SubtaskMetas {
+		p.subtaskMetas = append(p.subtaskMetas, plugin.SubTaskMeta{
+			Name:             subtask.Name,
+			EntryPoint:       remoteBridge.RemoteSubtaskEntrypointHandler(subtask),
+			Required:         subtask.Required,
+			EnabledByDefault: subtask.EnabledByDefault,
+			Description:      subtask.Description,
+			DomainTypes:      subtask.DomainTypes,
+		})
+	}
+	return &p, nil
+}
+
+func (p *remotePluginImpl) SubTaskMetas() []plugin.SubTaskMeta {
+	return p.subtaskMetas
+}
+
+func (p *remotePluginImpl) PrepareTaskData(taskCtx plugin.TaskContext, options map[string]interface{}) (interface{}, errors.Error) {
+	dbUrl := taskCtx.GetConfig("db_url")
+	connectionId := uint64(options["connectionId"].(float64))
+
+	connectionHelper := api.NewConnectionHelper(
+		taskCtx,
+		nil,
+	)
+
+	connection := p.connectionTabler.New()
+	err := connectionHelper.FirstById(connection, connectionId)
+	if err != nil {
+		return nil, errors.Convert(err)
+	}
+
+	return RemotePluginTaskData{
+		DbUrl:        dbUrl,
+		ConnectionId: connectionId,
+		Connection:   connection.Unwrap(),
+		Options:      options,
+	}, nil
+}
+
+func (p *remotePluginImpl) Description() string {
+	return p.description
+}
+
+func (p *remotePluginImpl) RootPkgPath() string {
+	// RootPkgPath is only used to find to which plugin a given type belongs.
+	// This in turn is only used by DomainIdGenerator.
+	// Remote plugins define tool layer types in another language,
+	// so the reflective implementation of NewDomainIdGenerator cannot work.
+	return ""
+}
+
+func (p *remotePluginImpl) ApiResources() map[string]map[string]plugin.ApiResourceHandler {
+	return p.resources
+}
+
+func (p *remotePluginImpl) RunMigrations(forceMigrate bool) errors.Error {
+	err := api.CallDB(basicRes.GetDal().AutoMigrate, p.connectionTabler.New())
+	if err != nil {
+		return err
+	}
+
+	err = p.invoker.Call("run-migrations", bridge.DefaultContext, forceMigrate).Get()
+	return err
+}
+
+var _ models.RemotePlugin = (*remotePluginImpl)(nil)
+var _ plugin.Scope = (*models.WrappedPipelineScope)(nil)
diff --git a/backend/server/services/remote/run/run.go b/backend/server/services/remote/run/run.go
new file mode 100644
index 000000000..b774ab1f8
--- /dev/null
+++ b/backend/server/services/remote/run/run.go
@@ -0,0 +1,65 @@
+/*
+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
+
+    http://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 (
+	"encoding/json"
+	"fmt"
+
+	"github.com/apache/incubator-devlake/core/runner"
+	"github.com/apache/incubator-devlake/server/services/remote"
+	"github.com/apache/incubator-devlake/server/services/remote/bridge"
+	"github.com/apache/incubator-devlake/server/services/remote/models"
+	"github.com/spf13/cobra"
+)
+
+func main() {
+	cmd := &cobra.Command{Use: "run"}
+
+	pluginPath := cmd.Flags().StringP("path", "p", "", "path of the plugin directory")
+	connectionId := cmd.Flags().Uint64P("connectionId", "c", 0, "connection id")
+	optionsJSON := cmd.Flags().StringP("options", "o", "{}", "plugin options as a JSON object")
+	_ = cmd.MarkFlagRequired("path")
+	_ = cmd.MarkFlagRequired("connectionId")
+
+	cmd.Run = func(cmd *cobra.Command, args []string) {
+		invoker := bridge.NewPythonPoetryCmdInvoker(*pluginPath)
+
+		pluginInfo := models.PluginInfo{}
+		err := invoker.Call("plugin-info", bridge.DefaultContext).Get(&pluginInfo)
+
+		if err != nil {
+			panic(fmt.Sprintf("Cannot get plugin info: %s", err))
+		}
+
+		plugin, err := remote.NewRemotePlugin(&pluginInfo)
+		if err != nil {
+			panic(fmt.Sprintf("Cannot initialize plugin: %s", err))
+		}
+
+		var options map[string]interface{}
+		jsonErr := json.Unmarshal([]byte(*optionsJSON), &options)
+		if jsonErr != nil {
+			panic(fmt.Sprintf("Cannot parse options: %s", jsonErr))
+		}
+
+		options["connectionId"] = *connectionId
+		runner.DirectRun(cmd, args, plugin, options)
+	}
+	runner.RunCmd(cmd)
+}
diff --git a/backend/test/helper/api.go b/backend/test/helper/api.go
new file mode 100644
index 000000000..a29b987a7
--- /dev/null
+++ b/backend/test/helper/api.go
@@ -0,0 +1,253 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	apiProject "github.com/apache/incubator-devlake/server/api/project"
+	"net/http"
+	"reflect"
+	"time"
+)
+
+type Connection struct {
+	api.BaseConnection `mapstructure:",squash"`
+	api.RestConnection `mapstructure:",squash"`
+	api.AccessToken    `mapstructure:",squash"`
+}
+
+// CreateConnection FIXME
+func (d *DevlakeClient) TestConnection(pluginName string, connection any) {
+	d.testCtx.Helper()
+	_ = sendHttpRequest[Connection](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/plugins/%s/test", d.Endpoint, pluginName), connection)
+}
+
+// CreateConnection FIXME
+func (d *DevlakeClient) CreateConnection(pluginName string, connection any) *Connection {
+	d.testCtx.Helper()
+	created := sendHttpRequest[Connection](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/plugins/%s/connections", d.Endpoint, pluginName), connection)
+	return &created
+}
+
+// ListConnections FIXME
+func (d *DevlakeClient) ListConnections(pluginName string) []*Connection {
+	d.testCtx.Helper()
+	all := sendHttpRequest[[]*Connection](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodGet, fmt.Sprintf("%s/plugins/%s/connections", d.Endpoint, pluginName), nil)
+	return all
+}
+
+type BlueprintV2Config struct {
+	Connection       *plugin.BlueprintConnectionV200
+	CreatedDateAfter *time.Time
+	SkipOnFail       bool
+	ProjectName      string
+}
+
+// CreateBasicBlueprintV2 FIXME
+func (d *DevlakeClient) CreateBasicBlueprintV2(name string, config *BlueprintV2Config) models.Blueprint {
+	settings := &models.BlueprintSettings{
+		Version:          "2.0.0",
+		CreatedDateAfter: config.CreatedDateAfter,
+		Connections: ToJson([]*plugin.BlueprintConnectionV200{
+			config.Connection,
+		}),
+	}
+	blueprint := models.Blueprint{
+		Name:        name,
+		ProjectName: config.ProjectName,
+		Mode:        models.BLUEPRINT_MODE_NORMAL,
+		Plan:        nil,
+		Enable:      true,
+		CronConfig:  "manual",
+		IsManual:    true,
+		SkipOnFail:  config.SkipOnFail,
+		Labels:      []string{"test-label"},
+		Settings:    ToJson(settings),
+	}
+	d.testCtx.Helper()
+	blueprint = sendHttpRequest[models.Blueprint](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/blueprints", d.Endpoint), &blueprint)
+	return blueprint
+}
+
+type (
+	ProjectPlugin struct {
+		Name    string
+		Options any
+	}
+	ProjectConfig struct {
+		ProjectName        string
+		ProjectDescription string
+		EnableDora         bool
+		MetricPlugins      []ProjectPlugin
+	}
+)
+
+func (d *DevlakeClient) CreateProject(project *ProjectConfig) models.ApiOutputProject {
+	var metrics []models.BaseMetric
+	doraSeen := false
+	for _, p := range project.MetricPlugins {
+		if p.Name == "dora" {
+			doraSeen = true
+		}
+		metrics = append(metrics, models.BaseMetric{
+			PluginName:   p.Name,
+			PluginOption: string(ToJson(p.Options)),
+			Enable:       true,
+		})
+	}
+	if project.EnableDora && !doraSeen {
+		metrics = append(metrics, models.BaseMetric{
+			PluginName:   "dora",
+			PluginOption: string(ToJson(nil)),
+			Enable:       true,
+		})
+	}
+	return sendHttpRequest[models.ApiOutputProject](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/projects", d.Endpoint), &models.ApiInputProject{
+		BaseProject: models.BaseProject{
+			Name:        project.ProjectName,
+			Description: project.ProjectDescription,
+		},
+		Enable:  Val(true),
+		Metrics: &metrics,
+	})
+}
+
+func (d *DevlakeClient) GetProject(projectName string) models.ApiOutputProject {
+	return sendHttpRequest[models.ApiOutputProject](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodGet, fmt.Sprintf("%s/projects/%s", d.Endpoint, projectName), nil)
+}
+
+func (d *DevlakeClient) ListProjects() apiProject.PaginatedProjects {
+	return sendHttpRequest[apiProject.PaginatedProjects](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodGet, fmt.Sprintf("%s/projects", d.Endpoint), nil)
+}
+
+func (d *DevlakeClient) CreateScope(pluginName string, connectionId uint64, scope any) any {
+	return sendHttpRequest[any](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPut, fmt.Sprintf("%s/plugins/%s/connections/%d/scopes", d.Endpoint, pluginName, connectionId), scope)
+}
+
+func (d *DevlakeClient) ListScopes(pluginName string, connectionId uint64) []any {
+	return sendHttpRequest[[]any](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodGet, fmt.Sprintf("%s/plugins/%s/connections/%d/scopes", d.Endpoint, pluginName, connectionId), nil)
+}
+
+func (d *DevlakeClient) CreateTransformRule(pluginName string, rules any) any {
+	return sendHttpRequest[any](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/plugins/%s/transformation_rules", d.Endpoint, pluginName), rules)
+}
+
+func (d *DevlakeClient) ListTransformRules(pluginName string) []any {
+	return sendHttpRequest[[]any](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodGet, fmt.Sprintf("%s/plugins/%s/transformation_rules?pageSize=20?page=1", d.Endpoint, pluginName), nil)
+}
+
+// CreateBasicBlueprint FIXME
+func (d *DevlakeClient) CreateBasicBlueprint(name string, connection *plugin.BlueprintConnectionV100) models.Blueprint {
+	settings := &models.BlueprintSettings{
+		Version:     "1.0.0",
+		Connections: ToJson([]*plugin.BlueprintConnectionV100{connection}),
+	}
+	blueprint := models.Blueprint{
+		Name:       name,
+		Mode:       models.BLUEPRINT_MODE_NORMAL,
+		Plan:       nil,
+		Enable:     true,
+		CronConfig: "manual",
+		IsManual:   true,
+		Settings:   ToJson(settings),
+	}
+	d.testCtx.Helper()
+	blueprint = sendHttpRequest[models.Blueprint](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/blueprints", d.Endpoint), &blueprint)
+	return blueprint
+}
+
+// TriggerBlueprint FIXME
+func (d *DevlakeClient) TriggerBlueprint(blueprintId uint64) models.Pipeline {
+	d.testCtx.Helper()
+	pipeline := sendHttpRequest[models.Pipeline](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/blueprints/%d/trigger", d.Endpoint, blueprintId), nil)
+	return d.monitorPipeline(pipeline.ID)
+}
+
+// RunPipeline FIXME
+func (d *DevlakeClient) RunPipeline(pipeline models.NewPipeline) models.Pipeline {
+	d.testCtx.Helper()
+	pipelineResult := sendHttpRequest[models.Pipeline](d.testCtx, d.timeout, debugInfo{
+		print:      true,
+		inlineJson: false,
+	}, http.MethodPost, fmt.Sprintf("%s/pipelines", d.Endpoint), &pipeline)
+	return d.monitorPipeline(pipelineResult.ID)
+}
+
+// MonitorPipeline FIXME
+func (d *DevlakeClient) monitorPipeline(id uint64) models.Pipeline {
+	d.testCtx.Helper()
+	var previousResult models.Pipeline
+	endpoint := fmt.Sprintf("%s/pipelines/%d", d.Endpoint, id)
+	coloredPrintf("calling:\n\t%s %s\nwith:\n%s\n", http.MethodGet, endpoint, string(ToCleanJson(false, nil)))
+	for {
+		time.Sleep(1 * time.Second)
+		pipelineResult := sendHttpRequest[models.Pipeline](d.testCtx, d.timeout, debugInfo{
+			print: false,
+		}, http.MethodGet, fmt.Sprintf("%s/pipelines/%d", d.Endpoint, id), nil)
+		if pipelineResult.Status == models.TASK_COMPLETED || pipelineResult.Status == models.TASK_FAILED {
+			coloredPrintf("result: %s\n", ToCleanJson(true, &pipelineResult))
+			return pipelineResult
+		}
+		if !reflect.DeepEqual(pipelineResult, previousResult) {
+			coloredPrintf("result: %s\n", ToCleanJson(true, &pipelineResult))
+		}
+		previousResult = pipelineResult
+	}
+}
diff --git a/backend/test/helper/client.go b/backend/test/helper/client.go
new file mode 100644
index 000000000..fa3bb2428
--- /dev/null
+++ b/backend/test/helper/client.go
@@ -0,0 +1,307 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"bytes"
+	"context"
+	"encoding/json"
+	goerror "errors"
+	"fmt"
+	"io"
+	"net/http"
+	"os"
+	"syscall"
+	"testing"
+	"time"
+
+	"github.com/apache/incubator-devlake/core/config"
+	corectx "github.com/apache/incubator-devlake/core/context"
+	"github.com/apache/incubator-devlake/core/errors"
+	"github.com/apache/incubator-devlake/core/log"
+	"github.com/apache/incubator-devlake/core/migration"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/core/runner"
+	contextimpl "github.com/apache/incubator-devlake/impls/context"
+	"github.com/apache/incubator-devlake/impls/dalgorm"
+	"github.com/apache/incubator-devlake/impls/logruslog"
+	"github.com/apache/incubator-devlake/server/api"
+	remotePlugin "github.com/apache/incubator-devlake/server/services/remote/plugin"
+	"github.com/spf13/viper"
+	"github.com/stretchr/testify/require"
+	"gorm.io/gorm"
+)
+
+var throwawayDir string
+
+func init() {
+	tempDir, err := errors.Convert01(os.MkdirTemp("", "devlake_test"+"_*"))
+	if err != nil {
+		panic(err)
+	}
+	throwawayDir = tempDir
+}
+
+// DevlakeClient FIXME
+type (
+	DevlakeClient struct {
+		Endpoint string
+		db       *gorm.DB
+		log      log.Logger
+		cfg      *viper.Viper
+		testCtx  *testing.T
+		basicRes corectx.BasicRes
+		timeout  time.Duration
+	}
+	LocalClientConfig struct {
+		ServerPort           uint
+		DbURL                string
+		CreateServer         bool
+		DropDb               bool
+		Plugins              map[string]plugin.PluginMeta
+		AdditionalMigrations func() []plugin.MigrationScript
+		Timeout              time.Duration
+	}
+	RemoteClientConfig struct {
+		Endpoint string
+	}
+)
+
+// ConnectRemoteServer returns a client to an existing server based on the config
+func ConnectRemoteServer(t *testing.T, sbConfig *RemoteClientConfig) *DevlakeClient {
+	return &DevlakeClient{
+		Endpoint: sbConfig.Endpoint,
+		db:       nil,
+		log:      nil,
+		testCtx:  t,
+	}
+}
+
+// ConnectLocalServer spins up a local server from the config and returns a client connected to it
+func ConnectLocalServer(t *testing.T, sbConfig *LocalClientConfig) *DevlakeClient {
+	t.Helper()
+	fmt.Printf("Using test temp directory: %s\n", throwawayDir)
+	logger := logruslog.Global.Nested("test")
+	cfg := config.GetConfig()
+	cfg.Set("DB_URL", sbConfig.DbURL)
+	db, err := runner.NewGormDb(cfg, logger)
+	require.NoError(t, err)
+	addr := fmt.Sprintf("http://localhost:%d", sbConfig.ServerPort)
+	d := &DevlakeClient{
+		Endpoint: addr,
+		db:       db,
+		log:      logger,
+		cfg:      cfg,
+		basicRes: contextimpl.NewDefaultBasicRes(cfg, logger, dalgorm.NewDalgorm(db)),
+		testCtx:  t,
+		timeout:  sbConfig.Timeout,
+	}
+	d.configureEncryption()
+	d.initPlugins(sbConfig)
+	if sbConfig.DropDb {
+		d.dropDB()
+	}
+	if sbConfig.CreateServer {
+		cfg.Set("PORT", fmt.Sprintf(":%d", sbConfig.ServerPort))
+		cfg.Set("PLUGIN_DIR", throwawayDir)
+		cfg.Set("LOGGING_DIR", throwawayDir)
+		go func() {
+			api.CreateApiService()
+		}()
+	}
+	req, err2 := http.NewRequest(http.MethodGet, fmt.Sprintf("%s/proceed-db-migration", addr), nil)
+	require.NoError(t, err2)
+	d.forceSendHttpRequest(20, req, func(err errors.Error) bool {
+		e := err.Unwrap()
+		return goerror.Is(e, syscall.ECONNREFUSED)
+	})
+	d.runMigrations(sbConfig)
+	return d
+}
+
+// SetTimeout override the timeout of api requests
+func (d *DevlakeClient) SetTimeout(timeout time.Duration) {
+	d.timeout = timeout
+}
+
+// RunPlugin manually execute a plugin directly (local server only)
+func (d *DevlakeClient) RunPlugin(ctx context.Context, pluginName string, pluginTask plugin.PluginTask, options map[string]interface{}, subtaskNames ...string) errors.Error {
+	if len(subtaskNames) == 0 {
+		subtaskNames = GetSubtaskNames(pluginTask.SubTaskMetas()...)
+	}
+	optionsJson, err := json.Marshal(options)
+	if err != nil {
+		return errors.Convert(err)
+	}
+	subtasksJson, err := json.Marshal(subtaskNames)
+	if err != nil {
+		return errors.Convert(err)
+	}
+	task := &models.Task{
+		Plugin:   pluginName,
+		Options:  string(optionsJson),
+		Subtasks: subtasksJson,
+	}
+	return runner.RunPluginSubTasks(
+		ctx,
+		d.basicRes,
+		task,
+		pluginTask,
+		nil,
+	)
+}
+
+func (d *DevlakeClient) configureEncryption() {
+	v := config.GetConfig()
+	encKey := v.GetString(plugin.EncodeKeyEnvStr)
+	if encKey == "" {
+		// Randomly generate a bunch of encryption keys and set them to config
+		encKey = plugin.RandomEncKey()
+		v.Set(plugin.EncodeKeyEnvStr, encKey)
+		err := config.WriteConfig(v)
+		if err != nil {
+			panic(err)
+		}
+	}
+}
+
+func (d *DevlakeClient) forceSendHttpRequest(retries uint, req *http.Request, onError func(err errors.Error) bool) {
+	d.testCtx.Helper()
+	for {
+		res, err := http.DefaultClient.Do(req)
+		if err != nil {
+			if !onError(errors.Default.WrapRaw(err)) {
+				require.NoError(d.testCtx, err)
+			}
+		} else {
+			if res.StatusCode != http.StatusOK {
+				panic(fmt.Sprintf("received HTTP status %d", res.StatusCode))
+			}
+			return
+		}
+		retries--
+		if retries == 0 {
+			panic("retry limit exceeded")
+		}
+		fmt.Printf("retrying http call to %s\n", req.URL.String())
+		time.Sleep(1 * time.Second)
+	}
+}
+
+func (d *DevlakeClient) initPlugins(sbConfig *LocalClientConfig) {
+	remotePlugin.Init(d.basicRes)
+	d.testCtx.Helper()
+	if sbConfig.Plugins != nil {
+		for name, p := range sbConfig.Plugins {
+			require.NoError(d.testCtx, plugin.RegisterPlugin(name, p))
+		}
+	}
+	for _, p := range plugin.AllPlugins() {
+		if pi, ok := p.(plugin.PluginInit); ok {
+			err := pi.Init(d.basicRes)
+			require.NoError(d.testCtx, err)
+		}
+	}
+}
+
+func (d *DevlakeClient) runMigrations(sbConfig *LocalClientConfig) {
+	d.testCtx.Helper()
+	basicRes := contextimpl.NewDefaultBasicRes(d.cfg, d.log, dalgorm.NewDalgorm(d.db))
+	getMigrator := func() plugin.Migrator {
+		migrator, err := migration.NewMigrator(basicRes)
+		require.NoError(d.testCtx, err)
+		return migrator
+	}
+	{
+		migrator := getMigrator()
+		for pluginName, pluginInst := range sbConfig.Plugins {
+			if migratable, ok := pluginInst.(plugin.PluginMigration); ok {
+				migrator.Register(migratable.MigrationScripts(), pluginName)
+			}
+		}
+		require.NoError(d.testCtx, migrator.Execute())
+	}
+	{
+		migrator := getMigrator()
+		if sbConfig.AdditionalMigrations != nil {
+			scripts := sbConfig.AdditionalMigrations()
+			migrator.Register(scripts, "extra migrations")
+		}
+		require.NoError(d.testCtx, migrator.Execute())
+	}
+}
+
+func (d *DevlakeClient) dropDB() {
+	d.testCtx.Helper()
+	migrator := d.db.Migrator()
+	tables, err := migrator.GetTables()
+	require.NoError(d.testCtx, err)
+	var tablesRaw []any
+	for _, table := range tables {
+		tablesRaw = append(tablesRaw, table)
+	}
+	err = migrator.DropTable(tablesRaw...)
+	require.NoError(d.testCtx, err)
+}
+
+func sendHttpRequest[Res any](t *testing.T, timeout time.Duration, debug debugInfo, httpMethod string, endpoint string, body any) Res {
+	t.Helper()
+	b := ToJson(body)
+	if debug.print {
+		coloredPrintf("calling:\n\t%s %s\nwith:\n%s\n", httpMethod, endpoint, string(ToCleanJson(debug.inlineJson, body)))
+	}
+	timer := time.After(timeout)
+	for {
+		request, err := http.NewRequest(httpMethod, endpoint, bytes.NewReader(b))
+		require.NoError(t, err)
+		request.Header.Add("Content-Type", "application/json")
+		response, err := http.DefaultClient.Do(request)
+		require.NoError(t, err)
+		if timeout > 0 {
+			select {
+			case <-timer:
+			default:
+				if response.StatusCode >= 300 {
+					time.Sleep(1 * time.Second)
+					continue
+				}
+			}
+		}
+		require.True(t, response.StatusCode < 300, "unexpected http status code: %d", response.StatusCode)
+		var result Res
+		b, _ = io.ReadAll(response.Body)
+		require.NoError(t, json.Unmarshal(b, &result))
+		if debug.print {
+			coloredPrintf("result: %s\n", ToCleanJson(debug.inlineJson, b))
+		}
+		require.NoError(t, response.Body.Close())
+		return result
+	}
+}
+
+func coloredPrintf(msg string, args ...any) {
+	msg = fmt.Sprintf(msg, args...)
+	colorifier := "\033[1;33m%+v\033[0m" //yellow
+	fmt.Printf(colorifier, msg)
+}
+
+type debugInfo struct {
+	print      bool
+	inlineJson bool
+}
diff --git a/backend/test/helper/db_selector.go b/backend/test/helper/db_selector.go
new file mode 100644
index 000000000..9cbe23569
--- /dev/null
+++ b/backend/test/helper/db_selector.go
@@ -0,0 +1,41 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"fmt"
+	"os"
+
+	"github.com/apache/incubator-devlake/core/config"
+)
+
+// UseMySQL FIXME
+func UseMySQL(host string, port int) string {
+	conn := fmt.Sprintf("mysql://merico:merico@%s:%d/lake?charset=utf8mb4&parseTime=True", host, port)
+	_ = os.Setenv("E2E_DB_URL", conn)
+	config.GetConfig().Set("E2E_DB_URL", conn)
+	return conn
+}
+
+// UsePostgres FIXME
+func UsePostgres(host string, port int) string {
+	conn := fmt.Sprintf("postgres://merico:merico@%s:%d/lake", host, port)
+	_ = os.Setenv("E2E_DB_URL", conn)
+	config.GetConfig().Set("E2E_DB_URL", conn)
+	return conn
+}
diff --git a/backend/test/helper/init.go b/backend/test/helper/init.go
new file mode 100644
index 000000000..0ba99d91f
--- /dev/null
+++ b/backend/test/helper/init.go
@@ -0,0 +1,51 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"os"
+	"path/filepath"
+)
+
+var (
+	ProjectRoot = ""
+	Shell       = ""
+)
+
+func LocalInit() {
+	pwd, err := os.Getwd()
+	if err != nil {
+		panic(err)
+	}
+	for {
+		dir := filepath.Base(pwd)
+		if dir == "" {
+			panic("base repo directory not found")
+		}
+		if dir == "lake" || dir == "incubator-devlake" {
+			ProjectRoot = pwd
+			break
+		}
+		pwd = filepath.Dir(pwd)
+	}
+	Shell = "/bin/sh"
+	err = os.Chdir(ProjectRoot)
+	if err != nil {
+		panic(err)
+	}
+}
diff --git a/backend/test/helper/json_helper.go b/backend/test/helper/json_helper.go
new file mode 100644
index 000000000..2ce2cdd0c
--- /dev/null
+++ b/backend/test/helper/json_helper.go
@@ -0,0 +1,104 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"encoding/json"
+	"reflect"
+	"time"
+)
+
+// ToJson FIXME
+func ToJson(x any) json.RawMessage {
+	b, err := json.Marshal(x)
+	if err != nil {
+		panic(err)
+	}
+	return b
+}
+
+// ToCleanJson FIXME
+func ToCleanJson(inline bool, x any) json.RawMessage {
+	j, err := json.Marshal(x)
+	if err != nil {
+		panic(err)
+	}
+	var m any
+	if j[0] == '[' {
+		//it's a slice
+		m = x
+	} else {
+		m = map[string]any{}
+		err = json.Unmarshal(j, &m)
+		if err != nil {
+			panic(err)
+		}
+		if m != nil {
+			removeNullsFromMap(m.(map[string]any))
+		}
+	}
+	var b []byte
+	if inline {
+		b, err = json.Marshal(m)
+	} else {
+		b, err = json.MarshalIndent(m, "", "    ")
+	}
+	if err != nil {
+		panic(err)
+	}
+	return b
+}
+
+func removeNullsFromMap(m map[string]any) {
+	refMap := reflect.ValueOf(m)
+	for _, refKey := range refMap.MapKeys() {
+		key := refKey.String()
+		refValue := refMap.MapIndex(refKey)
+		if refValue.IsNil() || refValue.Elem().IsZero() {
+			delete(m, key)
+			continue
+		}
+		value := refValue.Interface()
+		if isNullTime(value) {
+			delete(m, key)
+			continue
+		}
+		switch valueCasted := value.(type) {
+		case map[string]any:
+			removeNullsFromMap(valueCasted)
+		case []any:
+			for _, arrayValue := range valueCasted {
+				if m, ok := arrayValue.(map[string]any); ok {
+					removeNullsFromMap(m)
+				}
+			}
+		}
+	}
+}
+
+func isNullTime(value any) bool {
+	if str, ok := value.(string); ok {
+		if t, err := time.Parse("2006-01-02T15:04:05Z", str); err == nil {
+			zeroTime := time.Time{}
+			if t.Equal(zeroTime) {
+				return true
+			}
+		}
+	}
+	return false
+}
diff --git a/backend/test/helper/utils.go b/backend/test/helper/utils.go
new file mode 100644
index 000000000..479fcd534
--- /dev/null
+++ b/backend/test/helper/utils.go
@@ -0,0 +1,91 @@
+/*
+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
+
+    http://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 helper
+
+import (
+	"bufio"
+	"encoding/json"
+	"fmt"
+	"os"
+	"path/filepath"
+	"strings"
+
+	"github.com/apache/incubator-devlake/core/plugin"
+)
+
+// IsWSL FIXME
+func IsWSL() bool {
+	lines, err := readFile("/proc/version")
+	if err != nil {
+		return false
+	}
+	for _, line := range lines {
+		l := strings.ToLower(line)
+		if strings.Contains(l, "microsoft") {
+			return true
+		}
+	}
+	return false
+}
+
+// GetSubtaskNames FIXME
+func GetSubtaskNames(metas ...plugin.SubTaskMeta) []string {
+	var names []string
+	for _, m := range metas {
+		names = append(names, m.Name)
+	}
+	return names
+}
+
+// AddToPath FIXME
+func AddToPath(newPaths ...string) {
+	path := os.ExpandEnv("$PATH")
+	for _, newPath := range newPaths {
+		newPath, _ = filepath.Abs(newPath)
+		path = fmt.Sprintf("%s:%s", newPath, path)
+	}
+	_ = os.Setenv("PATH", path)
+}
+
+func Val[T any](t T) *T {
+	return &t
+}
+
+func Cast[T any](m any) T {
+	j := ToJson(m)
+	t := new(T)
+	err := json.Unmarshal(j, t)
+	if err != nil {
+		panic(err)
+	}
+	return *t
+}
+
+func readFile(path string) ([]string, error) {
+	file, err := os.Open(path)
+	if err != nil {
+		return nil, err
+	}
+	defer file.Close()
+	var lines []string
+	scanner := bufio.NewScanner(file)
+	for scanner.Scan() {
+		lines = append(lines, scanner.Text())
+	}
+	return lines, scanner.Err()
+}
diff --git a/.licenserc.yaml b/backend/test/remote/docker-compose.test.yml
similarity index 50%
copy from .licenserc.yaml
copy to backend/test/remote/docker-compose.test.yml
index b11fd8254..897b59ad5 100644
--- a/.licenserc.yaml
+++ b/backend/test/remote/docker-compose.test.yml
@@ -13,46 +13,35 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
+version: "3"
+services:
 
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
+  mysql-test:
+    image: mysql:8.0.26
+    platform: linux/x86_64
+    volumes:
+      - mysql-test-storage:/var/lib/mysql
+    restart: always
+    ports:
+      - "3307:3306"
+    environment:
+      MYSQL_ROOT_PASSWORD: admin
+      MYSQL_DATABASE: lake
+      MYSQL_USER: merico
+      MYSQL_PASSWORD: merico
 
+  postgres-test:
+    image: postgres:14.2-alpine
+    restart: always
+    ports:
+      - "3308:5432"
+    volumes:
+      - postgres-test-storage:/var/lib/postgresql/data
+    environment:
+      POSTGRES_DB: lake
+      POSTGRES_USER: merico
+      POSTGRES_PASSWORD: merico
 
-
-  comment: on-failure
+volumes:
+  mysql-test-storage:
+  postgres-test-storage:
\ No newline at end of file
diff --git a/.licenserc.yaml b/backend/test/remote/fakeplugin/build.sh
old mode 100644
new mode 100755
similarity index 50%
copy from .licenserc.yaml
copy to backend/test/remote/fakeplugin/build.sh
index b11fd8254..1a4a9d605
--- a/.licenserc.yaml
+++ b/backend/test/remote/fakeplugin/build.sh
@@ -1,58 +1,25 @@
+#!/bin/sh
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
 
+cd "${0%/*}" # make sure we're in the correct dir
 
-  comment: on-failure
+poetry config virtualenvs.create true
+echo "Installing dependencies of fake python plugin" &&\
+poetry install &&\
+exit_code=$?
+if [ $exit_code != 0 ]; then
+  exit $exit_code
+fi
diff --git a/.licenserc.yaml b/backend/test/remote/fakeplugin/fakeplugin/__init__.py
similarity index 50%
copy from .licenserc.yaml
copy to backend/test/remote/fakeplugin/fakeplugin/__init__.py
index b11fd8254..65d64ce95 100644
--- a/.licenserc.yaml
+++ b/backend/test/remote/fakeplugin/fakeplugin/__init__.py
@@ -4,55 +4,11 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
-
-
-
-  comment: on-failure
diff --git a/backend/test/remote/fakeplugin/fakeplugin/main.py b/backend/test/remote/fakeplugin/fakeplugin/main.py
new file mode 100644
index 000000000..0c7c2db8e
--- /dev/null
+++ b/backend/test/remote/fakeplugin/fakeplugin/main.py
@@ -0,0 +1,122 @@
+# 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
+
+#     http://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.
+
+from enum import Enum
+from datetime import datetime
+from typing import Optional
+
+from sqlmodel import Field
+
+from pydevlake import Plugin, Connection, Stream, ToolModel
+from pydevlake.domain_layer.devops import CICDScope, CICDPipeline
+
+
+VALID_TOKEN = "this_is_a_valid_token"
+VALID_PROJECT = "this_is_a_valid_project"
+
+
+class FakePipeline(ToolModel, table=True):
+    class State(Enum):
+        PENDING = "pending"
+        RUNNING = "running"
+        FAILURE = "failure"
+        SUCCESS = "success"
+
+    id: str = Field(primary_key=True)
+    project: str
+    started_at: Optional[datetime]
+    finished_at: Optional[datetime]
+    state: State
+
+
+class FakeStream(Stream):
+    tool_model = FakePipeline
+    domain_model = CICDPipeline
+
+    fake_pipelines = [
+        FakePipeline(id=1, project=VALID_PROJECT, state=FakePipeline.State.SUCCESS, started_at=datetime(2023, 1, 10, 11, 0, 0), finished_at=datetime(2023, 1, 10, 11, 3, 0)),
+        FakePipeline(id=2, project=VALID_PROJECT, state=FakePipeline.State.FAILURE, started_at=datetime(2023, 1, 10, 12, 0, 0), finished_at=datetime(2023, 1, 10, 12, 1, 30)),
+        FakePipeline(id=1, project=VALID_PROJECT, state=FakePipeline.State.PENDING),
+    ]
+
+    def collect(self, state, context):
+        project = context.options['project']
+        if project == VALID_PROJECT:
+            for p in self.fake_pipelines:
+                yield dict(p)
+
+    def convert(self, pipeline: FakePipeline):
+        yield CICDPipeline(
+            name=pipeline.id,
+            status=self.convert_status(pipeline.state),
+            finished_date=pipeline.finished_at,
+            result=self.convert_result(pipeline.state),
+            duration_sec=self.duration(pipeline),
+            environment=[],
+            type=CICDPipeline.Type.CI
+        )
+
+    def convert_status(self, state: FakePipeline.State):
+        match state:
+            case FakePipeline.State.FAILURE | FakePipeline.State.SUCCESS:
+                return CICDPipeline.Status.DONE
+            case _:
+                return CICDPipeline.Status.IN_PROGRESS
+            
+    def convert_result(self, state: FakePipeline.State):
+        match state:
+            case FakePipeline.State.SUCCESS:
+                return CICDPipeline.Result.SUCCESS
+            case FakePipeline.State.FAILURE:
+                return CICDPipeline.Status.FAILURE
+            case _:
+                return None            
+
+    def duration(self, pipeline: FakePipeline):
+        if pipeline.finished_at:
+            return (pipeline.finished_at - pipeline.started_at).seconds
+        return None
+
+
+class FakeConnection(Connection):
+    token: str
+
+
+class FakePlugin(Plugin):
+    @property
+    def connection_type(self):
+        return FakeConnection
+
+    def get_scopes(self, scope_name: str, connection: FakeConnection):
+        assert connection
+        yield CICDScope(
+            id=1,
+            name=scope_name,
+            url=f"http://fake.org/api/project/{scope_name}"
+        )
+
+    def test_connection(self, connection: FakeConnection):
+        if connection.token != VALID_TOKEN:
+            raise Exception("Invalid token")
+        
+    @property
+    def streams(self):
+        return [
+            FakeStream
+        ]
+
+
+if __name__ == '__main__':
+    FakePlugin.start()
diff --git a/backend/test/remote/fakeplugin/poetry.lock b/backend/test/remote/fakeplugin/poetry.lock
new file mode 100644
index 000000000..ab0cac7b0
--- /dev/null
+++ b/backend/test/remote/fakeplugin/poetry.lock
@@ -0,0 +1,513 @@
+# This file is automatically @generated by Poetry and should not be changed by hand.
+
+[[package]]
+name = "certifi"
+version = "2022.12.7"
+description = "Python package for providing Mozilla's CA Bundle."
+category = "main"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "certifi-2022.12.7-py3-none-any.whl", hash = "sha256:4ad3232f5e926d6718ec31cfc1fcadfde020920e278684144551c91769c7bc18"},
+    {file = "certifi-2022.12.7.tar.gz", hash = "sha256:35824b4c3a97115964b408844d64aa14db1cc518f6562e8d7261699d1350a9e3"},
+]
+
+[[package]]
+name = "charset-normalizer"
+version = "3.0.1"
+description = "The Real First Universal Charset Detector. Open, modern and actively maintained alternative to Chardet."
+category = "main"
+optional = false
+python-versions = "*"
+files = [
+    {file = "charset-normalizer-3.0.1.tar.gz", hash = "sha256:ebea339af930f8ca5d7a699b921106c6e29c617fe9606fa7baa043c1cdae326f"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_10_9_universal2.whl", hash = "sha256:88600c72ef7587fe1708fd242b385b6ed4b8904976d5da0893e31df8b3480cb6"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:c75ffc45f25324e68ab238cb4b5c0a38cd1c3d7f1fb1f72b5541de469e2247db"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:db72b07027db150f468fbada4d85b3b2729a3db39178abf5c543b784c1254539"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:62595ab75873d50d57323a91dd03e6966eb79c41fa834b7a1661ed043b2d404d"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ff6f3db31555657f3163b15a6b7c6938d08df7adbfc9dd13d9d19edad678f1e8"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:772b87914ff1152b92a197ef4ea40efe27a378606c39446ded52c8f80f79702e"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:70990b9c51340e4044cfc394a81f614f3f90d41397104d226f21e66de668730d"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:292d5e8ba896bbfd6334b096e34bffb56161c81408d6d036a7dfa6929cff8783"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:2edb64ee7bf1ed524a1da60cdcd2e1f6e2b4f66ef7c077680739f1641f62f555"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:31a9ddf4718d10ae04d9b18801bd776693487cbb57d74cc3458a7673f6f34639"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_ppc64le.whl", hash = "sha256:44ba614de5361b3e5278e1241fda3dc1838deed864b50a10d7ce92983797fa76"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_s390x.whl", hash = "sha256:12db3b2c533c23ab812c2b25934f60383361f8a376ae272665f8e48b88e8e1c6"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:c512accbd6ff0270939b9ac214b84fb5ada5f0409c44298361b2f5e13f9aed9e"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-win32.whl", hash = "sha256:502218f52498a36d6bf5ea77081844017bf7982cdbe521ad85e64cabee1b608b"},
+    {file = "charset_normalizer-3.0.1-cp310-cp310-win_amd64.whl", hash = "sha256:601f36512f9e28f029d9481bdaf8e89e5148ac5d89cffd3b05cd533eeb423b59"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:0298eafff88c99982a4cf66ba2efa1128e4ddaca0b05eec4c456bbc7db691d8d"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:a8d0fc946c784ff7f7c3742310cc8a57c5c6dc31631269876a88b809dbeff3d3"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:87701167f2a5c930b403e9756fab1d31d4d4da52856143b609e30a1ce7160f3c"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:14e76c0f23218b8f46c4d87018ca2e441535aed3632ca134b10239dfb6dadd6b"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:0c0a590235ccd933d9892c627dec5bc7511ce6ad6c1011fdf5b11363022746c1"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:8c7fe7afa480e3e82eed58e0ca89f751cd14d767638e2550c77a92a9e749c317"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:79909e27e8e4fcc9db4addea88aa63f6423ebb171db091fb4373e3312cb6d603"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:8ac7b6a045b814cf0c47f3623d21ebd88b3e8cf216a14790b455ea7ff0135d18"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:72966d1b297c741541ca8cf1223ff262a6febe52481af742036a0b296e35fa5a"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:f9d0c5c045a3ca9bedfc35dca8526798eb91a07aa7a2c0fee134c6c6f321cbd7"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_ppc64le.whl", hash = "sha256:5995f0164fa7df59db4746112fec3f49c461dd6b31b841873443bdb077c13cfc"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_s390x.whl", hash = "sha256:4a8fcf28c05c1f6d7e177a9a46a1c52798bfe2ad80681d275b10dcf317deaf0b"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:761e8904c07ad053d285670f36dd94e1b6ab7f16ce62b9805c475b7aa1cffde6"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-win32.whl", hash = "sha256:71140351489970dfe5e60fc621ada3e0f41104a5eddaca47a7acb3c1b851d6d3"},
+    {file = "charset_normalizer-3.0.1-cp311-cp311-win_amd64.whl", hash = "sha256:9ab77acb98eba3fd2a85cd160851816bfce6871d944d885febf012713f06659c"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-macosx_10_9_x86_64.whl", hash = "sha256:84c3990934bae40ea69a82034912ffe5a62c60bbf6ec5bc9691419641d7d5c9a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:74292fc76c905c0ef095fe11e188a32ebd03bc38f3f3e9bcb85e4e6db177b7ea"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c95a03c79bbe30eec3ec2b7f076074f4281526724c8685a42872974ef4d36b72"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f4c39b0e3eac288fedc2b43055cfc2ca7a60362d0e5e87a637beac5d801ef478"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:df2c707231459e8a4028eabcd3cfc827befd635b3ef72eada84ab13b52e1574d"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:93ad6d87ac18e2a90b0fe89df7c65263b9a99a0eb98f0a3d2e079f12a0735837"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:59e5686dd847347e55dffcc191a96622f016bc0ad89105e24c14e0d6305acbc6"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_i686.whl", hash = "sha256:cd6056167405314a4dc3c173943f11249fa0f1b204f8b51ed4bde1a9cd1834dc"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_ppc64le.whl", hash = "sha256:083c8d17153ecb403e5e1eb76a7ef4babfc2c48d58899c98fcaa04833e7a2f9a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_s390x.whl", hash = "sha256:f5057856d21e7586765171eac8b9fc3f7d44ef39425f85dbcccb13b3ebea806c"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:7eb33a30d75562222b64f569c642ff3dc6689e09adda43a082208397f016c39a"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-win32.whl", hash = "sha256:95dea361dd73757c6f1c0a1480ac499952c16ac83f7f5f4f84f0658a01b8ef41"},
+    {file = "charset_normalizer-3.0.1-cp36-cp36m-win_amd64.whl", hash = "sha256:eaa379fcd227ca235d04152ca6704c7cb55564116f8bc52545ff357628e10602"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:3e45867f1f2ab0711d60c6c71746ac53537f1684baa699f4f668d4c6f6ce8e14"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:cadaeaba78750d58d3cc6ac4d1fd867da6fc73c88156b7a3212a3cd4819d679d"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:911d8a40b2bef5b8bbae2e36a0b103f142ac53557ab421dc16ac4aafee6f53dc"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:503e65837c71b875ecdd733877d852adbc465bd82c768a067badd953bf1bc5a3"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:a60332922359f920193b1d4826953c507a877b523b2395ad7bc716ddd386d866"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:16a8663d6e281208d78806dbe14ee9903715361cf81f6d4309944e4d1e59ac5b"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:a16418ecf1329f71df119e8a65f3aa68004a3f9383821edcb20f0702934d8087"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:9d9153257a3f70d5f69edf2325357251ed20f772b12e593f3b3377b5f78e7ef8"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_ppc64le.whl", hash = "sha256:02a51034802cbf38db3f89c66fb5d2ec57e6fe7ef2f4a44d070a593c3688667b"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_s390x.whl", hash = "sha256:2e396d70bc4ef5325b72b593a72c8979999aa52fb8bcf03f701c1b03e1166918"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:11b53acf2411c3b09e6af37e4b9005cba376c872503c8f28218c7243582df45d"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-win32.whl", hash = "sha256:0bf2dae5291758b6f84cf923bfaa285632816007db0330002fa1de38bfcb7154"},
+    {file = "charset_normalizer-3.0.1-cp37-cp37m-win_amd64.whl", hash = "sha256:2c03cc56021a4bd59be889c2b9257dae13bf55041a3372d3295416f86b295fb5"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_10_9_universal2.whl", hash = "sha256:024e606be3ed92216e2b6952ed859d86b4cfa52cd5bc5f050e7dc28f9b43ec42"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:4b0d02d7102dd0f997580b51edc4cebcf2ab6397a7edf89f1c73b586c614272c"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:358a7c4cb8ba9b46c453b1dd8d9e431452d5249072e4f56cfda3149f6ab1405e"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:81d6741ab457d14fdedc215516665050f3822d3e56508921cc7239f8c8e66a58"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8b8af03d2e37866d023ad0ddea594edefc31e827fee64f8de5611a1dbc373174"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:9cf4e8ad252f7c38dd1f676b46514f92dc0ebeb0db5552f5f403509705e24753"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e696f0dd336161fca9adbb846875d40752e6eba585843c768935ba5c9960722b"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c22d3fe05ce11d3671297dc8973267daa0f938b93ec716e12e0f6dee81591dc1"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:109487860ef6a328f3eec66f2bf78b0b72400280d8f8ea05f69c51644ba6521a"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:37f8febc8ec50c14f3ec9637505f28e58d4f66752207ea177c1d67df25da5aed"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_ppc64le.whl", hash = "sha256:f97e83fa6c25693c7a35de154681fcc257c1c41b38beb0304b9c4d2d9e164479"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_s390x.whl", hash = "sha256:a152f5f33d64a6be73f1d30c9cc82dfc73cec6477ec268e7c6e4c7d23c2d2291"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:39049da0ffb96c8cbb65cbf5c5f3ca3168990adf3551bd1dee10c48fce8ae820"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-win32.whl", hash = "sha256:4457ea6774b5611f4bed5eaa5df55f70abde42364d498c5134b7ef4c6958e20e"},
+    {file = "charset_normalizer-3.0.1-cp38-cp38-win_amd64.whl", hash = "sha256:e62164b50f84e20601c1ff8eb55620d2ad25fb81b59e3cd776a1902527a788af"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_10_9_universal2.whl", hash = "sha256:8eade758719add78ec36dc13201483f8e9b5d940329285edcd5f70c0a9edbd7f"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:8499ca8f4502af841f68135133d8258f7b32a53a1d594aa98cc52013fff55678"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:3fc1c4a2ffd64890aebdb3f97e1278b0cc72579a08ca4de8cd2c04799a3a22be"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:00d3ffdaafe92a5dc603cb9bd5111aaa36dfa187c8285c543be562e61b755f6b"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:c2ac1b08635a8cd4e0cbeaf6f5e922085908d48eb05d44c5ae9eabab148512ca"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:f6f45710b4459401609ebebdbcfb34515da4fc2aa886f95107f556ac69a9147e"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3ae1de54a77dc0d6d5fcf623290af4266412a7c4be0b1ff7444394f03f5c54e3"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:3b590df687e3c5ee0deef9fc8c547d81986d9a1b56073d82de008744452d6541"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:ab5de034a886f616a5668aa5d098af2b5385ed70142090e2a31bcbd0af0fdb3d"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:9cb3032517f1627cc012dbc80a8ec976ae76d93ea2b5feaa9d2a5b8882597579"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_ppc64le.whl", hash = "sha256:608862a7bf6957f2333fc54ab4399e405baad0163dc9f8d99cb236816db169d4"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_s390x.whl", hash = "sha256:0f438ae3532723fb6ead77e7c604be7c8374094ef4ee2c5e03a3a17f1fca256c"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:356541bf4381fa35856dafa6a965916e54bed415ad8a24ee6de6e37deccf2786"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-win32.whl", hash = "sha256:39cf9ed17fe3b1bc81f33c9ceb6ce67683ee7526e65fde1447c772afc54a1bb8"},
+    {file = "charset_normalizer-3.0.1-cp39-cp39-win_amd64.whl", hash = "sha256:0a11e971ed097d24c534c037d298ad32c6ce81a45736d31e0ff0ad37ab437d59"},
+    {file = "charset_normalizer-3.0.1-py3-none-any.whl", hash = "sha256:7e189e2e1d3ed2f4aebabd2d5b0f931e883676e51c7624826e0a4e5fe8a0bf24"},
+]
+
+[[package]]
+name = "fire"
+version = "0.4.0"
+description = "A library for automatically generating command line interfaces."
+category = "main"
+optional = false
+python-versions = "*"
+files = [
+    {file = "fire-0.4.0.tar.gz", hash = "sha256:c5e2b8763699d1142393a46d0e3e790c5eb2f0706082df8f647878842c216a62"},
+]
+
+[package.dependencies]
+six = "*"
+termcolor = "*"
+
+[[package]]
+name = "greenlet"
+version = "2.0.2"
+description = "Lightweight in-process concurrent programming"
+category = "main"
+optional = false
+python-versions = ">=2.7,!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*"
+files = [
+    {file = "greenlet-2.0.2-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:bdfea8c661e80d3c1c99ad7c3ff74e6e87184895bbaca6ee8cc61209f8b9b85d"},
+    {file = "greenlet-2.0.2-cp27-cp27m-manylinux2010_x86_64.whl", hash = "sha256:9d14b83fab60d5e8abe587d51c75b252bcc21683f24699ada8fb275d7712f5a9"},
+    {file = "greenlet-2.0.2-cp27-cp27m-win32.whl", hash = "sha256:6c3acb79b0bfd4fe733dff8bc62695283b57949ebcca05ae5c129eb606ff2d74"},
+    {file = "greenlet-2.0.2-cp27-cp27m-win_amd64.whl", hash = "sha256:283737e0da3f08bd637b5ad058507e578dd462db259f7f6e4c5c365ba4ee9343"},
+    {file = "greenlet-2.0.2-cp27-cp27mu-manylinux2010_x86_64.whl", hash = "sha256:d27ec7509b9c18b6d73f2f5ede2622441de812e7b1a80bbd446cb0633bd3d5ae"},
+    {file = "greenlet-2.0.2-cp310-cp310-macosx_11_0_x86_64.whl", hash = "sha256:30bcf80dda7f15ac77ba5af2b961bdd9dbc77fd4ac6105cee85b0d0a5fcf74df"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:26fbfce90728d82bc9e6c38ea4d038cba20b7faf8a0ca53a9c07b67318d46088"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9190f09060ea4debddd24665d6804b995a9c122ef5917ab26e1566dcc712ceeb"},
+    {file = "greenlet-2.0.2-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:d75209eed723105f9596807495d58d10b3470fa6732dd6756595e89925ce2470"},
+    {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_aarch64.whl", hash = "sha256:3a51c9751078733d88e013587b108f1b7a1fb106d402fb390740f002b6f6551a"},
+    {file = "greenlet-2.0.2-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:76ae285c8104046b3a7f06b42f29c7b73f77683df18c49ab5af7983994c2dd91"},
+    {file = "greenlet-2.0.2-cp310-cp310-win_amd64.whl", hash = "sha256:2d4686f195e32d36b4d7cf2d166857dbd0ee9f3d20ae349b6bf8afc8485b3645"},
+    {file = "greenlet-2.0.2-cp311-cp311-macosx_10_9_universal2.whl", hash = "sha256:c4302695ad8027363e96311df24ee28978162cdcdd2006476c43970b384a244c"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c48f54ef8e05f04d6eff74b8233f6063cb1ed960243eacc474ee73a2ea8573ca"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a1846f1b999e78e13837c93c778dcfc3365902cfb8d1bdb7dd73ead37059f0d0"},
+    {file = "greenlet-2.0.2-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:3a06ad5312349fec0ab944664b01d26f8d1f05009566339ac6f63f56589bc1a2"},
+    {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_aarch64.whl", hash = "sha256:eff4eb9b7eb3e4d0cae3d28c283dc16d9bed6b193c2e1ace3ed86ce48ea8df19"},
+    {file = "greenlet-2.0.2-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:5454276c07d27a740c5892f4907c86327b632127dd9abec42ee62e12427ff7e3"},
+    {file = "greenlet-2.0.2-cp311-cp311-win_amd64.whl", hash = "sha256:7cafd1208fdbe93b67c7086876f061f660cfddc44f404279c1585bbf3cdc64c5"},
+    {file = "greenlet-2.0.2-cp35-cp35m-macosx_10_14_x86_64.whl", hash = "sha256:910841381caba4f744a44bf81bfd573c94e10b3045ee00de0cbf436fe50673a6"},
+    {file = "greenlet-2.0.2-cp35-cp35m-manylinux2010_x86_64.whl", hash = "sha256:18a7f18b82b52ee85322d7a7874e676f34ab319b9f8cce5de06067384aa8ff43"},
+    {file = "greenlet-2.0.2-cp35-cp35m-win32.whl", hash = "sha256:03a8f4f3430c3b3ff8d10a2a86028c660355ab637cee9333d63d66b56f09d52a"},
+    {file = "greenlet-2.0.2-cp35-cp35m-win_amd64.whl", hash = "sha256:4b58adb399c4d61d912c4c331984d60eb66565175cdf4a34792cd9600f21b394"},
+    {file = "greenlet-2.0.2-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:703f18f3fda276b9a916f0934d2fb6d989bf0b4fb5a64825260eb9bfd52d78f0"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux2010_x86_64.whl", hash = "sha256:32e5b64b148966d9cccc2c8d35a671409e45f195864560829f395a54226408d3"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:2dd11f291565a81d71dab10b7033395b7a3a5456e637cf997a6f33ebdf06f8db"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e0f72c9ddb8cd28532185f54cc1453f2c16fb417a08b53a855c4e6a418edd099"},
+    {file = "greenlet-2.0.2-cp36-cp36m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:cd021c754b162c0fb55ad5d6b9d960db667faad0fa2ff25bb6e1301b0b6e6a75"},
+    {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_aarch64.whl", hash = "sha256:3c9b12575734155d0c09d6c3e10dbd81665d5c18e1a7c6597df72fd05990c8cf"},
+    {file = "greenlet-2.0.2-cp36-cp36m-musllinux_1_1_x86_64.whl", hash = "sha256:b9ec052b06a0524f0e35bd8790686a1da006bd911dd1ef7d50b77bfbad74e292"},
+    {file = "greenlet-2.0.2-cp36-cp36m-win32.whl", hash = "sha256:dbfcfc0218093a19c252ca8eb9aee3d29cfdcb586df21049b9d777fd32c14fd9"},
+    {file = "greenlet-2.0.2-cp36-cp36m-win_amd64.whl", hash = "sha256:9f35ec95538f50292f6d8f2c9c9f8a3c6540bbfec21c9e5b4b751e0a7c20864f"},
+    {file = "greenlet-2.0.2-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:d5508f0b173e6aa47273bdc0a0b5ba055b59662ba7c7ee5119528f466585526b"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux2010_x86_64.whl", hash = "sha256:f82d4d717d8ef19188687aa32b8363e96062911e63ba22a0cff7802a8e58e5f1"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c9c59a2120b55788e800d82dfa99b9e156ff8f2227f07c5e3012a45a399620b7"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:2780572ec463d44c1d3ae850239508dbeb9fed38e294c68d19a24d925d9223ca"},
+    {file = "greenlet-2.0.2-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:937e9020b514ceedb9c830c55d5c9872abc90f4b5862f89c0887033ae33c6f73"},
+    {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_aarch64.whl", hash = "sha256:36abbf031e1c0f79dd5d596bfaf8e921c41df2bdf54ee1eed921ce1f52999a86"},
+    {file = "greenlet-2.0.2-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:18e98fb3de7dba1c0a852731c3070cf022d14f0d68b4c87a19cc1016f3bb8b33"},
+    {file = "greenlet-2.0.2-cp37-cp37m-win32.whl", hash = "sha256:3f6ea9bd35eb450837a3d80e77b517ea5bc56b4647f5502cd28de13675ee12f7"},
+    {file = "greenlet-2.0.2-cp37-cp37m-win_amd64.whl", hash = "sha256:7492e2b7bd7c9b9916388d9df23fa49d9b88ac0640db0a5b4ecc2b653bf451e3"},
+    {file = "greenlet-2.0.2-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:b864ba53912b6c3ab6bcb2beb19f19edd01a6bfcbdfe1f37ddd1778abfe75a30"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux2010_x86_64.whl", hash = "sha256:ba2956617f1c42598a308a84c6cf021a90ff3862eddafd20c3333d50f0edb45b"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:fc3a569657468b6f3fb60587e48356fe512c1754ca05a564f11366ac9e306526"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:8eab883b3b2a38cc1e050819ef06a7e6344d4a990d24d45bc6f2cf959045a45b"},
+    {file = "greenlet-2.0.2-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:acd2162a36d3de67ee896c43effcd5ee3de247eb00354db411feb025aa319857"},
+    {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_aarch64.whl", hash = "sha256:0bf60faf0bc2468089bdc5edd10555bab6e85152191df713e2ab1fcc86382b5a"},
+    {file = "greenlet-2.0.2-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:b0ef99cdbe2b682b9ccbb964743a6aca37905fda5e0452e5ee239b1654d37f2a"},
+    {file = "greenlet-2.0.2-cp38-cp38-win32.whl", hash = "sha256:b80f600eddddce72320dbbc8e3784d16bd3fb7b517e82476d8da921f27d4b249"},
+    {file = "greenlet-2.0.2-cp38-cp38-win_amd64.whl", hash = "sha256:4d2e11331fc0c02b6e84b0d28ece3a36e0548ee1a1ce9ddde03752d9b79bba40"},
+    {file = "greenlet-2.0.2-cp39-cp39-macosx_11_0_x86_64.whl", hash = "sha256:88d9ab96491d38a5ab7c56dd7a3cc37d83336ecc564e4e8816dbed12e5aaefc8"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux2010_x86_64.whl", hash = "sha256:561091a7be172ab497a3527602d467e2b3fbe75f9e783d8b8ce403fa414f71a6"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:971ce5e14dc5e73715755d0ca2975ac88cfdaefcaab078a284fea6cfabf866df"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:be4ed120b52ae4d974aa40215fcdfde9194d63541c7ded40ee12eb4dda57b76b"},
+    {file = "greenlet-2.0.2-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:94c817e84245513926588caf1152e3b559ff794d505555211ca041f032abbb6b"},
+    {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_aarch64.whl", hash = "sha256:1a819eef4b0e0b96bb0d98d797bef17dc1b4a10e8d7446be32d1da33e095dbb8"},
+    {file = "greenlet-2.0.2-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:7efde645ca1cc441d6dc4b48c0f7101e8d86b54c8530141b09fd31cef5149ec9"},
+    {file = "greenlet-2.0.2-cp39-cp39-win32.whl", hash = "sha256:ea9872c80c132f4663822dd2a08d404073a5a9b5ba6155bea72fb2a79d1093b5"},
+    {file = "greenlet-2.0.2-cp39-cp39-win_amd64.whl", hash = "sha256:db1a39669102a1d8d12b57de2bb7e2ec9066a6f2b3da35ae511ff93b01b5d564"},
+    {file = "greenlet-2.0.2.tar.gz", hash = "sha256:e7c8dc13af7db097bed64a051d2dd49e9f0af495c26995c00a9ee842690d34c0"},
+]
+
+[package.extras]
+docs = ["Sphinx", "docutils (<0.18)"]
+test = ["objgraph", "psutil"]
+
+[[package]]
+name = "idna"
+version = "3.4"
+description = "Internationalized Domain Names in Applications (IDNA)"
+category = "main"
+optional = false
+python-versions = ">=3.5"
+files = [
+    {file = "idna-3.4-py3-none-any.whl", hash = "sha256:90b77e79eaa3eba6de819a0c442c0b4ceefc341a7a2ab77d7562bf49f425c5c2"},
+    {file = "idna-3.4.tar.gz", hash = "sha256:814f528e8dead7d329833b91c5faa87d60bf71824cd12a7530b5526063d02cb4"},
+]
+
+[[package]]
+name = "inflect"
+version = "6.0.2"
+description = "Correctly generate plurals, singular nouns, ordinals, indefinite articles; convert numbers to words"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "inflect-6.0.2-py3-none-any.whl", hash = "sha256:182741ec7e9e4c8f7f55b01fa6d80bcd3c4a183d349dfa6d9abbff0a1279e98f"},
+    {file = "inflect-6.0.2.tar.gz", hash = "sha256:f1a6bcb0105046f89619fde1a7d044c612c614c2d85ef182582d9dc9b86d309a"},
+]
+
+[package.dependencies]
+pydantic = ">=1.9.1"
+
+[package.extras]
+docs = ["furo", "jaraco.packaging (>=9)", "jaraco.tidelift (>=1.4)", "rst.linker (>=1.9)", "sphinx (>=3.5)"]
+testing = ["flake8 (<5)", "pygments", "pytest (>=6)", "pytest-black (>=0.3.7)", "pytest-checkdocs (>=2.4)", "pytest-cov", "pytest-enabler (>=1.3)", "pytest-flake8", "pytest-mypy (>=0.9.1)"]
+
+[[package]]
+name = "mysqlclient"
+version = "2.1.1"
+description = "Python interface to MySQL"
+category = "main"
+optional = false
+python-versions = ">=3.5"
+files = [
+    {file = "mysqlclient-2.1.1-cp310-cp310-win_amd64.whl", hash = "sha256:c1ed71bd6244993b526113cca3df66428609f90e4652f37eb51c33496d478b37"},
+    {file = "mysqlclient-2.1.1-cp311-cp311-win_amd64.whl", hash = "sha256:c812b67e90082a840efb82a8978369e6e69fc62ce1bda4ca8f3084a9d862308b"},
+    {file = "mysqlclient-2.1.1-cp38-cp38-win_amd64.whl", hash = "sha256:0d1cd3a5a4d28c222fa199002810e8146cffd821410b67851af4cc80aeccd97c"},
+    {file = "mysqlclient-2.1.1-cp39-cp39-win_amd64.whl", hash = "sha256:b355c8b5a7d58f2e909acdbb050858390ee1b0e13672ae759e5e784110022994"},
+    {file = "mysqlclient-2.1.1-pp38-pypy38_pp73-win_amd64.whl", hash = "sha256:996924f3483fd36a34a5812210c69e71dea5a3d5978d01199b78b7f6d485c855"},
+    {file = "mysqlclient-2.1.1-pp39-pypy39_pp73-win_amd64.whl", hash = "sha256:dea88c8d3f5a5d9293dfe7f087c16dd350ceb175f2f6631c9cf4caf3e19b7a96"},
+    {file = "mysqlclient-2.1.1.tar.gz", hash = "sha256:828757e419fb11dd6c5ed2576ec92c3efaa93a0f7c39e263586d1ee779c3d782"},
+]
+
+[[package]]
+name = "pydantic"
+version = "1.10.4"
+description = "Data validation and settings management using python type hints"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "pydantic-1.10.4-cp310-cp310-macosx_10_9_x86_64.whl", hash = "sha256:b5635de53e6686fe7a44b5cf25fcc419a0d5e5c1a1efe73d49d48fe7586db854"},
+    {file = "pydantic-1.10.4-cp310-cp310-macosx_11_0_arm64.whl", hash = "sha256:6dc1cc241440ed7ca9ab59d9929075445da6b7c94ced281b3dd4cfe6c8cff817"},
+    {file = "pydantic-1.10.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:51bdeb10d2db0f288e71d49c9cefa609bca271720ecd0c58009bd7504a0c464c"},
+    {file = "pydantic-1.10.4-cp310-cp310-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:78cec42b95dbb500a1f7120bdf95c401f6abb616bbe8785ef09887306792e66e"},
+    {file = "pydantic-1.10.4-cp310-cp310-musllinux_1_1_i686.whl", hash = "sha256:8775d4ef5e7299a2f4699501077a0defdaac5b6c4321173bcb0f3c496fbadf85"},
+    {file = "pydantic-1.10.4-cp310-cp310-musllinux_1_1_x86_64.whl", hash = "sha256:572066051eeac73d23f95ba9a71349c42a3e05999d0ee1572b7860235b850cc6"},
+    {file = "pydantic-1.10.4-cp310-cp310-win_amd64.whl", hash = "sha256:7feb6a2d401f4d6863050f58325b8d99c1e56f4512d98b11ac64ad1751dc647d"},
+    {file = "pydantic-1.10.4-cp311-cp311-macosx_10_9_x86_64.whl", hash = "sha256:39f4a73e5342b25c2959529f07f026ef58147249f9b7431e1ba8414a36761f53"},
+    {file = "pydantic-1.10.4-cp311-cp311-macosx_11_0_arm64.whl", hash = "sha256:983e720704431a6573d626b00662eb78a07148c9115129f9b4351091ec95ecc3"},
+    {file = "pydantic-1.10.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75d52162fe6b2b55964fbb0af2ee58e99791a3138588c482572bb6087953113a"},
+    {file = "pydantic-1.10.4-cp311-cp311-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:fdf8d759ef326962b4678d89e275ffc55b7ce59d917d9f72233762061fd04a2d"},
+    {file = "pydantic-1.10.4-cp311-cp311-musllinux_1_1_i686.whl", hash = "sha256:05a81b006be15655b2a1bae5faa4280cf7c81d0e09fcb49b342ebf826abe5a72"},
+    {file = "pydantic-1.10.4-cp311-cp311-musllinux_1_1_x86_64.whl", hash = "sha256:d88c4c0e5c5dfd05092a4b271282ef0588e5f4aaf345778056fc5259ba098857"},
+    {file = "pydantic-1.10.4-cp311-cp311-win_amd64.whl", hash = "sha256:6a05a9db1ef5be0fe63e988f9617ca2551013f55000289c671f71ec16f4985e3"},
+    {file = "pydantic-1.10.4-cp37-cp37m-macosx_10_9_x86_64.whl", hash = "sha256:887ca463c3bc47103c123bc06919c86720e80e1214aab79e9b779cda0ff92a00"},
+    {file = "pydantic-1.10.4-cp37-cp37m-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fdf88ab63c3ee282c76d652fc86518aacb737ff35796023fae56a65ced1a5978"},
+    {file = "pydantic-1.10.4-cp37-cp37m-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:a48f1953c4a1d9bd0b5167ac50da9a79f6072c63c4cef4cf2a3736994903583e"},
+    {file = "pydantic-1.10.4-cp37-cp37m-musllinux_1_1_i686.whl", hash = "sha256:a9f2de23bec87ff306aef658384b02aa7c32389766af3c5dee9ce33e80222dfa"},
+    {file = "pydantic-1.10.4-cp37-cp37m-musllinux_1_1_x86_64.whl", hash = "sha256:cd8702c5142afda03dc2b1ee6bc358b62b3735b2cce53fc77b31ca9f728e4bc8"},
+    {file = "pydantic-1.10.4-cp37-cp37m-win_amd64.whl", hash = "sha256:6e7124d6855b2780611d9f5e1e145e86667eaa3bd9459192c8dc1a097f5e9903"},
+    {file = "pydantic-1.10.4-cp38-cp38-macosx_10_9_x86_64.whl", hash = "sha256:0b53e1d41e97063d51a02821b80538053ee4608b9a181c1005441f1673c55423"},
+    {file = "pydantic-1.10.4-cp38-cp38-macosx_11_0_arm64.whl", hash = "sha256:55b1625899acd33229c4352ce0ae54038529b412bd51c4915349b49ca575258f"},
+    {file = "pydantic-1.10.4-cp38-cp38-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:301d626a59edbe5dfb48fcae245896379a450d04baeed50ef40d8199f2733b06"},
+    {file = "pydantic-1.10.4-cp38-cp38-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:b6f9d649892a6f54a39ed56b8dfd5e08b5f3be5f893da430bed76975f3735d15"},
+    {file = "pydantic-1.10.4-cp38-cp38-musllinux_1_1_i686.whl", hash = "sha256:d7b5a3821225f5c43496c324b0d6875fde910a1c2933d726a743ce328fbb2a8c"},
+    {file = "pydantic-1.10.4-cp38-cp38-musllinux_1_1_x86_64.whl", hash = "sha256:f2f7eb6273dd12472d7f218e1fef6f7c7c2f00ac2e1ecde4db8824c457300416"},
+    {file = "pydantic-1.10.4-cp38-cp38-win_amd64.whl", hash = "sha256:4b05697738e7d2040696b0a66d9f0a10bec0efa1883ca75ee9e55baf511909d6"},
+    {file = "pydantic-1.10.4-cp39-cp39-macosx_10_9_x86_64.whl", hash = "sha256:a9a6747cac06c2beb466064dda999a13176b23535e4c496c9d48e6406f92d42d"},
+    {file = "pydantic-1.10.4-cp39-cp39-macosx_11_0_arm64.whl", hash = "sha256:eb992a1ef739cc7b543576337bebfc62c0e6567434e522e97291b251a41dad7f"},
+    {file = "pydantic-1.10.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:990406d226dea0e8f25f643b370224771878142155b879784ce89f633541a024"},
+    {file = "pydantic-1.10.4-cp39-cp39-manylinux_2_5_i686.manylinux1_i686.manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:2e82a6d37a95e0b1b42b82ab340ada3963aea1317fd7f888bb6b9dfbf4fff57c"},
+    {file = "pydantic-1.10.4-cp39-cp39-musllinux_1_1_i686.whl", hash = "sha256:9193d4f4ee8feca58bc56c8306bcb820f5c7905fd919e0750acdeeeef0615b28"},
+    {file = "pydantic-1.10.4-cp39-cp39-musllinux_1_1_x86_64.whl", hash = "sha256:2b3ce5f16deb45c472dde1a0ee05619298c864a20cded09c4edd820e1454129f"},
+    {file = "pydantic-1.10.4-cp39-cp39-win_amd64.whl", hash = "sha256:9cbdc268a62d9a98c56e2452d6c41c0263d64a2009aac69246486f01b4f594c4"},
+    {file = "pydantic-1.10.4-py3-none-any.whl", hash = "sha256:4948f264678c703f3877d1c8877c4e3b2e12e549c57795107f08cf70c6ec7774"},
+    {file = "pydantic-1.10.4.tar.gz", hash = "sha256:b9a3859f24eb4e097502a3be1fb4b2abb79b6103dd9e2e0edb70613a4459a648"},
+]
+
+[package.dependencies]
+typing-extensions = ">=4.2.0"
+
+[package.extras]
+dotenv = ["python-dotenv (>=0.10.4)"]
+email = ["email-validator (>=1.0.3)"]
+
+[[package]]
+name = "pydevlake"
+version = "0.1.0"
+description = "Devlake plugin framework"
+category = "main"
+optional = false
+python-versions = "^3.10"
+files = []
+develop = false
+
+[package.dependencies]
+fire = "^0.4.0"
+inflect = "^6.0.2"
+mysqlclient = "^2.1.1"
+pydantic = "^1.10.2"
+requests = "^2.28.1"
+sqlmodel = "^0.0.8"
+
+[package.source]
+type = "directory"
+url = "../../../python/pydevlake"
+
+[[package]]
+name = "requests"
+version = "2.28.2"
+description = "Python HTTP for Humans."
+category = "main"
+optional = false
+python-versions = ">=3.7, <4"
+files = [
+    {file = "requests-2.28.2-py3-none-any.whl", hash = "sha256:64299f4909223da747622c030b781c0d7811e359c37124b4bd368fb8c6518baa"},
+    {file = "requests-2.28.2.tar.gz", hash = "sha256:98b1b2782e3c6c4904938b84c0eb932721069dfdb9134313beff7c83c2df24bf"},
+]
+
+[package.dependencies]
+certifi = ">=2017.4.17"
+charset-normalizer = ">=2,<4"
+idna = ">=2.5,<4"
+urllib3 = ">=1.21.1,<1.27"
+
+[package.extras]
+socks = ["PySocks (>=1.5.6,!=1.5.7)"]
+use-chardet-on-py3 = ["chardet (>=3.0.2,<6)"]
+
+[[package]]
+name = "six"
+version = "1.16.0"
+description = "Python 2 and 3 compatibility utilities"
+category = "main"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*"
+files = [
+    {file = "six-1.16.0-py2.py3-none-any.whl", hash = "sha256:8abb2f1d86890a2dfb989f9a77cfcfd3e47c2a354b01111771326f8aa26e0254"},
+    {file = "six-1.16.0.tar.gz", hash = "sha256:1e61c37477a1626458e36f7b1d82aa5c9b094fa4802892072e49de9c60c4c926"},
+]
+
+[[package]]
+name = "sqlalchemy"
+version = "1.4.41"
+description = "Database Abstraction Library"
+category = "main"
+optional = false
+python-versions = "!=3.0.*,!=3.1.*,!=3.2.*,!=3.3.*,!=3.4.*,!=3.5.*,>=2.7"
+files = [
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-macosx_10_14_x86_64.whl", hash = "sha256:13e397a9371ecd25573a7b90bd037db604331cf403f5318038c46ee44908c44d"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:2d6495f84c4fd11584f34e62f9feec81bf373787b3942270487074e35cbe5330"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-win32.whl", hash = "sha256:e570cfc40a29d6ad46c9aeaddbdcee687880940a3a327f2c668dd0e4ef0a441d"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27m-win_amd64.whl", hash = "sha256:5facb7fd6fa8a7353bbe88b95695e555338fb038ad19ceb29c82d94f62775a05"},
+    {file = "SQLAlchemy-1.4.41-cp27-cp27mu-manylinux_2_5_x86_64.manylinux1_x86_64.whl", hash = "sha256:f37fa70d95658763254941ddd30ecb23fc4ec0c5a788a7c21034fc2305dab7cc"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-macosx_10_15_x86_64.whl", hash = "sha256:361f6b5e3f659e3c56ea3518cf85fbdae1b9e788ade0219a67eeaaea8a4e4d2a"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0990932f7cca97fece8017414f57fdd80db506a045869d7ddf2dda1d7cf69ecc"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:cd767cf5d7252b1c88fcfb58426a32d7bd14a7e4942497e15b68ff5d822b41ad"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:5102fb9ee2c258a2218281adcb3e1918b793c51d6c2b4666ce38c35101bb940e"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-win32.whl", hash = "sha256:2082a2d2fca363a3ce21cfa3d068c5a1ce4bf720cf6497fb3a9fc643a8ee4ddd"},
+    {file = "SQLAlchemy-1.4.41-cp310-cp310-win_amd64.whl", hash = "sha256:e4b12e3d88a8fffd0b4ca559f6d4957ed91bd4c0613a4e13846ab8729dc5c251"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-macosx_10_15_x86_64.whl", hash = "sha256:90484a2b00baedad361402c257895b13faa3f01780f18f4a104a2f5c413e4536"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:b67fc780cfe2b306180e56daaa411dd3186bf979d50a6a7c2a5b5036575cbdbb"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2ad2b727fc41c7f8757098903f85fafb4bf587ca6605f82d9bf5604bd9c7cded"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-win32.whl", hash = "sha256:59bdc291165b6119fc6cdbc287c36f7f2859e6051dd923bdf47b4c55fd2f8bd0"},
+    {file = "SQLAlchemy-1.4.41-cp311-cp311-win_amd64.whl", hash = "sha256:d2e054aed4645f9b755db85bc69fc4ed2c9020c19c8027976f66576b906a74f1"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-macosx_10_14_x86_64.whl", hash = "sha256:4ba7e122510bbc07258dc42be6ed45997efdf38129bde3e3f12649be70683546"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:c0dcf127bb99458a9d211e6e1f0f3edb96c874dd12f2503d4d8e4f1fd103790b"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:e16c2be5cb19e2c08da7bd3a87fed2a0d4e90065ee553a940c4fc1a0fb1ab72b"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:f5ebeeec5c14533221eb30bad716bc1fd32f509196318fb9caa7002c4a364e4c"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-win32.whl", hash = "sha256:3e2ef592ac3693c65210f8b53d0edcf9f4405925adcfc031ff495e8d18169682"},
+    {file = "SQLAlchemy-1.4.41-cp36-cp36m-win_amd64.whl", hash = "sha256:eb30cf008850c0a26b72bd1b9be6730830165ce049d239cfdccd906f2685f892"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-macosx_10_15_x86_64.whl", hash = "sha256:c23d64a0b28fc78c96289ffbd0d9d1abd48d267269b27f2d34e430ea73ce4b26"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:8eb8897367a21b578b26f5713833836f886817ee2ffba1177d446fa3f77e67c8"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:14576238a5f89bcf504c5f0a388d0ca78df61fb42cb2af0efe239dc965d4f5c9"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:639e1ae8d48b3c86ffe59c0daa9a02e2bfe17ca3d2b41611b30a0073937d4497"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-win32.whl", hash = "sha256:0005bd73026cd239fc1e8ccdf54db58b6193be9a02b3f0c5983808f84862c767"},
+    {file = "SQLAlchemy-1.4.41-cp37-cp37m-win_amd64.whl", hash = "sha256:5323252be2bd261e0aa3f33cb3a64c45d76829989fa3ce90652838397d84197d"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-macosx_10_15_x86_64.whl", hash = "sha256:05f0de3a1dc3810a776275763764bb0015a02ae0f698a794646ebc5fb06fad33"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:0002e829142b2af00b4eaa26c51728f3ea68235f232a2e72a9508a3116bd6ed0"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:22ff16cedab5b16a0db79f1bc99e46a6ddececb60c396562e50aab58ddb2871c"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:ccfd238f766a5bb5ee5545a62dd03f316ac67966a6a658efb63eeff8158a4bbf"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-win32.whl", hash = "sha256:58bb65b3274b0c8a02cea9f91d6f44d0da79abc993b33bdedbfec98c8440175a"},
+    {file = "SQLAlchemy-1.4.41-cp38-cp38-win_amd64.whl", hash = "sha256:ce8feaa52c1640de9541eeaaa8b5fb632d9d66249c947bb0d89dd01f87c7c288"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-macosx_10_15_x86_64.whl", hash = "sha256:199a73c31ac8ea59937cc0bf3dfc04392e81afe2ec8a74f26f489d268867846c"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:4676d51c9f6f6226ae8f26dc83ec291c088fe7633269757d333978df78d931ab"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_12_x86_64.manylinux2010_x86_64.whl", hash = "sha256:036d8472356e1d5f096c5e0e1a7e0f9182140ada3602f8fff6b7329e9e7cfbcd"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-manylinux_2_5_x86_64.manylinux1_x86_64.manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:2307495d9e0ea00d0c726be97a5b96615035854972cc538f6e7eaed23a35886c"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-win32.whl", hash = "sha256:9c56e19780cd1344fcd362fd6265a15f48aa8d365996a37fab1495cae8fcd97d"},
+    {file = "SQLAlchemy-1.4.41-cp39-cp39-win_amd64.whl", hash = "sha256:f5fa526d027d804b1f85cdda1eb091f70bde6fb7d87892f6dd5a48925bc88898"},
+    {file = "SQLAlchemy-1.4.41.tar.gz", hash = "sha256:0292f70d1797e3c54e862e6f30ae474014648bc9c723e14a2fda730adb0a9791"},
+]
+
+[package.dependencies]
+greenlet = {version = "!=0.4.17", markers = "python_version >= \"3\" and (platform_machine == \"aarch64\" or platform_machine == \"ppc64le\" or platform_machine == \"x86_64\" or platform_machine == \"amd64\" or platform_machine == \"AMD64\" or platform_machine == \"win32\" or platform_machine == \"WIN32\")"}
+
+[package.extras]
+aiomysql = ["aiomysql", "greenlet (!=0.4.17)"]
+aiosqlite = ["aiosqlite", "greenlet (!=0.4.17)", "typing_extensions (!=3.10.0.1)"]
+asyncio = ["greenlet (!=0.4.17)"]
+asyncmy = ["asyncmy (>=0.2.3,!=0.2.4)", "greenlet (!=0.4.17)"]
+mariadb-connector = ["mariadb (>=1.0.1,!=1.1.2)"]
+mssql = ["pyodbc"]
+mssql-pymssql = ["pymssql"]
+mssql-pyodbc = ["pyodbc"]
+mypy = ["mypy (>=0.910)", "sqlalchemy2-stubs"]
+mysql = ["mysqlclient (>=1.4.0)", "mysqlclient (>=1.4.0,<2)"]
+mysql-connector = ["mysql-connector-python"]
+oracle = ["cx_oracle (>=7)", "cx_oracle (>=7,<8)"]
+postgresql = ["psycopg2 (>=2.7)"]
+postgresql-asyncpg = ["asyncpg", "greenlet (!=0.4.17)"]
+postgresql-pg8000 = ["pg8000 (>=1.16.6,!=1.29.0)"]
+postgresql-psycopg2binary = ["psycopg2-binary"]
+postgresql-psycopg2cffi = ["psycopg2cffi"]
+pymysql = ["pymysql", "pymysql (<1)"]
+sqlcipher = ["sqlcipher3_binary"]
+
+[[package]]
+name = "sqlalchemy2-stubs"
+version = "0.0.2a32"
+description = "Typing Stubs for SQLAlchemy 1.4"
+category = "main"
+optional = false
+python-versions = ">=3.6"
+files = [
+    {file = "sqlalchemy2-stubs-0.0.2a32.tar.gz", hash = "sha256:2a2cfab71d35ac63bf21ad841d8610cd93a3bd4c6562848c538fa975585c2739"},
+    {file = "sqlalchemy2_stubs-0.0.2a32-py3-none-any.whl", hash = "sha256:7f5fb30b0cf7c6b74c50c1d94df77ff32007afee8d80499752eb3fedffdbdfb8"},
+]
+
+[package.dependencies]
+typing-extensions = ">=3.7.4"
+
+[[package]]
+name = "sqlmodel"
+version = "0.0.8"
+description = "SQLModel, SQL databases in Python, designed for simplicity, compatibility, and robustness."
+category = "main"
+optional = false
+python-versions = ">=3.6.1,<4.0.0"
+files = [
+    {file = "sqlmodel-0.0.8-py3-none-any.whl", hash = "sha256:0fd805719e0c5d4f22be32eb3ffc856eca3f7f20e8c7aa3e117ad91684b518ee"},
+    {file = "sqlmodel-0.0.8.tar.gz", hash = "sha256:3371b4d1ad59d2ffd0c530582c2140b6c06b090b32af9b9c6412986d7b117036"},
+]
+
+[package.dependencies]
+pydantic = ">=1.8.2,<2.0.0"
+SQLAlchemy = ">=1.4.17,<=1.4.41"
+sqlalchemy2-stubs = "*"
+
+[[package]]
+name = "termcolor"
+version = "2.2.0"
+description = "ANSI color formatting for output in terminal"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "termcolor-2.2.0-py3-none-any.whl", hash = "sha256:91ddd848e7251200eac969846cbae2dacd7d71c2871e92733289e7e3666f48e7"},
+    {file = "termcolor-2.2.0.tar.gz", hash = "sha256:dfc8ac3f350788f23b2947b3e6cfa5a53b630b612e6cd8965a015a776020b99a"},
+]
+
+[package.extras]
+tests = ["pytest", "pytest-cov"]
+
+[[package]]
+name = "typing-extensions"
+version = "4.4.0"
+description = "Backported and Experimental Type Hints for Python 3.7+"
+category = "main"
+optional = false
+python-versions = ">=3.7"
+files = [
+    {file = "typing_extensions-4.4.0-py3-none-any.whl", hash = "sha256:16fa4864408f655d35ec496218b85f79b3437c829e93320c7c9215ccfd92489e"},
+    {file = "typing_extensions-4.4.0.tar.gz", hash = "sha256:1511434bb92bf8dd198c12b1cc812e800d4181cfcb867674e0f8279cc93087aa"},
+]
+
+[[package]]
+name = "urllib3"
+version = "1.26.14"
+description = "HTTP library with thread-safe connection pooling, file post, and more."
+category = "main"
+optional = false
+python-versions = ">=2.7, !=3.0.*, !=3.1.*, !=3.2.*, !=3.3.*, !=3.4.*, !=3.5.*"
+files = [
+    {file = "urllib3-1.26.14-py2.py3-none-any.whl", hash = "sha256:75edcdc2f7d85b137124a6c3c9fc3933cdeaa12ecb9a6a959f22797a0feca7e1"},
+    {file = "urllib3-1.26.14.tar.gz", hash = "sha256:076907bf8fd355cde77728471316625a4d2f7e713c125f51953bb5b3eecf4f72"},
+]
+
+[package.extras]
+brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)", "brotlipy (>=0.6.0)"]
+secure = ["certifi", "cryptography (>=1.3.4)", "idna (>=2.0.0)", "ipaddress", "pyOpenSSL (>=0.14)", "urllib3-secure-extra"]
+socks = ["PySocks (>=1.5.6,!=1.5.7,<2.0)"]
+
+[metadata]
+lock-version = "2.0"
+python-versions = "^3.10"
+content-hash = "ceb16988b541afeca0e4c6ecf1099ad7bb1c58ddafbf3884455c734e6494c283"
diff --git a/.licenserc.yaml b/backend/test/remote/fakeplugin/pyproject.toml
similarity index 50%
copy from .licenserc.yaml
copy to backend/test/remote/fakeplugin/pyproject.toml
index b11fd8254..c91fa4692 100644
--- a/.licenserc.yaml
+++ b/backend/test/remote/fakeplugin/pyproject.toml
@@ -4,55 +4,26 @@
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
 
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
+[tool.poetry]
+name = "fakeplugin"
+version = "0.1.0"
+description = "Fake python plugin used only in tests"
+authors = []
 
+[tool.poetry.dependencies]
+python = "^3.10"
+pydevlake = { path = "../../../python/pydevlake", develop = false }
 
 
-  comment: on-failure
+[build-system]
+requires = ["poetry-core"]
+build-backend = "poetry.core.masonry.api"
diff --git a/.licenserc.yaml b/backend/test/remote/fakeplugin/start.sh
old mode 100644
new mode 100755
similarity index 50%
copy from .licenserc.yaml
copy to backend/test/remote/fakeplugin/start.sh
index b11fd8254..6a9d005ea
--- a/.licenserc.yaml
+++ b/backend/test/remote/fakeplugin/start.sh
@@ -1,58 +1,26 @@
+#!/bin/sh
 # 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
-#
+
 #     http://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.
-#
-header:
-  license:
-    spdx-id: Apache-2.0
-    copyright-owner: Apache Software Foundation
-
-  paths-ignore:
-    - 'dist'
-    - 'licenses'
-    - '**/*.md'
-    - 'LICENSE'
-    - 'NOTICE'
-    - 'config-ui/node_modules'
-    - 'bin'
-    - 'logs'
-    - 'mocks'
-    - 'vendor'
-    - '.env.example'
-    - '**/*.log'
-    - '**/env.example'
-    - '**/*.csv'
-    - '**/*.json'
-    - '**/*.sql'
-    - '**/*.svg'
-    - '**/*.png'
-    - '.editorconfig'
-    - '**/.gitignore'
-    - '**/.helmignore'
-    - '**/.dockerignore'
-    - 'DISCLAIMER-WIP'
-    - 'DISCLAIMER'
-    - 'go.mod'
-    - 'go.sum'
-    - 'requirements.txt'
-    - '**/.babelrc'
-    - '**/empty'
-    - '**/*.conf'
-    - '**/.eslintrc'
-    - 'deployment/helm/templates/_helpers.tpl'
-    - '**/.nvmrc'
 
+endpoint="$1"
 
+cd "${0%/*}" # make sure we're in the correct dir
 
-  comment: on-failure
+echo "Registering fake python plugin" &&\
+poetry run python fakeplugin/main.py startup "$endpoint" &&\
+exit_code=$?
+if [ $exit_code != 0 ]; then
+  exit $exit_code
+fi
diff --git a/backend/test/remote/remote_test.go b/backend/test/remote/remote_test.go
new file mode 100644
index 000000000..4ecbc8768
--- /dev/null
+++ b/backend/test/remote/remote_test.go
@@ -0,0 +1,113 @@
+/*
+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
+
+    http://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 test
+
+import (
+	"fmt"
+	"github.com/apache/incubator-devlake/core/models"
+	"github.com/apache/incubator-devlake/core/plugin"
+	"github.com/apache/incubator-devlake/core/utils"
+	"github.com/apache/incubator-devlake/helpers/pluginhelper/api"
+	"github.com/apache/incubator-devlake/test/helper"
+	"github.com/stretchr/testify/require"
+	"os"
+	"os/exec"
+	"path/filepath"
+	"testing"
+	"time"
+)
+
+const PLUGIN_NAME = "fake"
+
+func setupEnv() {
+	fmt.Println("Setup test env")
+	helper.LocalInit()
+	_ = os.Setenv("REMOTE_PLUGINS_STARTUP_PATH", filepath.Join("backend/test/remote/fakeplugin/start.sh"))
+	_ = os.Setenv("ENABLE_REMOTE_PLUGINS", "true")
+}
+
+func buildPython(t *testing.T) {
+	fmt.Println("Build fake plugin")
+	path := filepath.Join(helper.ProjectRoot, "backend/test/remote/fakeplugin/build.sh")
+	cmd := exec.Command(helper.Shell, []string{path}...)
+	cmd.Dir = filepath.Dir(path)
+	cmd.Env = append(cmd.Env, os.Environ()...)
+	r, err := utils.RunProcess(cmd,
+		&utils.RunProcessOptions{
+			OnStdout: func(b []byte) {
+				fmt.Println(string(b))
+			},
+			OnStderr: func(b []byte) {
+				fmt.Println(string(b))
+			},
+		})
+	require.NoError(t, err)
+	require.NoError(t, r.GetError())
+}
+
+func connectLocalServer(t *testing.T) *helper.DevlakeClient {
+	fmt.Println("Connect to server")
+	client := helper.ConnectLocalServer(t, &helper.LocalClientConfig{
+		ServerPort:   8089,
+		DbURL:        helper.UseMySQL("127.0.0.1", 3307),
+		CreateServer: true,
+		DropDb:       true,
+		Plugins:      map[string]plugin.PluginMeta{},
+	})
+	client.SetTimeout(60 * time.Second)
+	// Wait for plugin registration
+	time.Sleep(3 * time.Second)
+	return client
+}
+
+func TestRunPipeline(t *testing.T) {
+	setupEnv()
+	buildPython(t)
+	client := connectLocalServer(t)
+	fmt.Println("Create new connection")
+	conn := client.CreateConnection(PLUGIN_NAME,
+		api.AccessToken{
+			Token: "this_is_a_valid_token",
+		},
+	)
+	client.SetTimeout(0)
+	conns := client.ListConnections(PLUGIN_NAME)
+	require.Equal(t, 1, len(conns))
+	require.Equal(t, "this_is_a_valid_token", conns[0].Token)
+	fmt.Println("Run pipeline")
+	t.Run("run_pipeline", func(t *testing.T) {
+		pipeline := client.RunPipeline(models.NewPipeline{
+			Name: "remote_test",
+			Plan: []plugin.PipelineStage{
+				{
+					{
+						Plugin:   PLUGIN_NAME,
+						Subtasks: nil,
+						Options: map[string]interface{}{
+							"connectionId": conn.ID,
+							"scopeId":      1,
+						},
+					},
+				},
+			},
+		})
+		require.Equal(t, models.TASK_COMPLETED, pipeline.Status)
+		require.Equal(t, 1, pipeline.FinishedTasks)
+		require.Equal(t, "", pipeline.ErrorName)
+	})
+}