diff --git a/cmd/mo-service/main.go b/cmd/mo-service/main.go
index 6ce4c450f830334ec3409d2918f6553e31e1cc7f..cc444fd631eb59deea5558f319f50d14ffd55827 100644
--- a/cmd/mo-service/main.go
+++ b/cmd/mo-service/main.go
@@ -19,6 +19,8 @@ import (
 	"errors"
 	"flag"
 	"fmt"
+	"github.com/matrixorigin/matrixone/pkg/cnservice/cnclient"
+	"github.com/matrixorigin/matrixone/pkg/sql/compile"
 	"math/rand"
 	"os"
 	"os/signal"
@@ -88,13 +90,17 @@ func startService(cfg *Config, stopper *stopper.Stopper) error {
 func startCNService(cfg *Config, stopper *stopper.Stopper) error {
 	return stopper.RunNamedTask("cn-service", func(ctx context.Context) {
 		c := cfg.getCNServiceConfig()
-		s, err := cnservice.NewService(&c, ctx)
+		s, err := cnservice.NewService(&c, ctx, cnservice.WithMessageHandle(compile.CnServerMessageHandler))
 		if err != nil {
 			panic(err)
 		}
 		if err := s.Start(); err != nil {
 			panic(err)
 		}
+		err = cnclient.NewCNClient(&cnclient.ClientConfig{})
+		if err != nil {
+			panic(err)
+		}
 
 		<-ctx.Done()
 		if err := s.Close(); err != nil {
diff --git a/pkg/cnservice/cnclient/client.go b/pkg/cnservice/cnclient/client.go
new file mode 100644
index 0000000000000000000000000000000000000000..ccc67278bf73fb957a6335803a6d741ae5edda0c
--- /dev/null
+++ b/pkg/cnservice/cnclient/client.go
@@ -0,0 +1,101 @@
+// Copyright 2021 - 2022 Matrix Origin
+//
+// Licensed 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 cnclient
+
+import (
+	"context"
+	"github.com/fagongzi/goetty/v2"
+	"github.com/matrixorigin/matrixone/pkg/common/morpc"
+	"github.com/matrixorigin/matrixone/pkg/pb/pipeline"
+	"sync"
+)
+
+var Client *CNClient
+
+type CNClient struct {
+	config *ClientConfig
+	client morpc.RPCClient
+
+	// pool for send message
+	requestPool *sync.Pool
+}
+
+func (c *CNClient) Send(ctx context.Context, backend string, request morpc.Message) (*morpc.Future, error) {
+	return c.client.Send(ctx, backend, request)
+}
+
+func (c *CNClient) NewStream(backend string) (morpc.Stream, error) {
+	return c.client.NewStream(backend)
+}
+
+func (c *CNClient) Close() error {
+	return c.client.Close()
+}
+
+const (
+	dfMaxSenderNumber       = 10
+	dfClientReadBufferSize  = 1 << 10
+	dfClientWriteBufferSize = 1 << 10
+	dfPayLoadCopyBufferSize = 1 << 20
+)
+
+// ClientConfig a config to init a CNClient
+type ClientConfig struct {
+	// MaxSenderNumber is the max number of backends per host for compute node service.
+	MaxSenderNumber int
+	// related buffer size.
+	PayLoadCopyBufferSize int
+	ReadBufferSize        int
+	WriteBufferSize       int
+}
+
+func NewCNClient(cfg *ClientConfig) error {
+	var err error
+	cfg.Fill()
+	Client = &CNClient{config: cfg}
+	Client.requestPool = &sync.Pool{New: func() any { return &pipeline.Message{} }}
+
+	codec := morpc.NewMessageCodec(Client.acquireMessage, cfg.PayLoadCopyBufferSize)
+	factory := morpc.NewGoettyBasedBackendFactory(codec,
+		morpc.WithBackendConnectWhenCreate(),
+		morpc.WithBackendGoettyOptions(goetty.WithSessionRWBUfferSize(
+			cfg.ReadBufferSize, cfg.WriteBufferSize)),
+	)
+
+	Client.client, err = morpc.NewClient(factory,
+		morpc.WithClientMaxBackendPerHost(cfg.MaxSenderNumber),
+	)
+	return err
+}
+
+func (c *CNClient) acquireMessage() morpc.Message {
+	return c.requestPool.Get().(*pipeline.Message)
+}
+
+// Fill set some default value for client config.
+func (cfg *ClientConfig) Fill() {
+	if cfg.PayLoadCopyBufferSize < 0 {
+		cfg.PayLoadCopyBufferSize = dfPayLoadCopyBufferSize
+	}
+	if cfg.MaxSenderNumber <= 0 {
+		cfg.MaxSenderNumber = dfMaxSenderNumber
+	}
+	if cfg.ReadBufferSize < 0 {
+		cfg.ReadBufferSize = dfClientReadBufferSize
+	}
+	if cfg.WriteBufferSize < 0 {
+		cfg.WriteBufferSize = dfClientWriteBufferSize
+	}
+}
diff --git a/pkg/cnservice/server.go b/pkg/cnservice/server.go
index a8c5b38338cf3b746f6d2ddc7669de568db974a6..25ea781d8fc50e0b2622c830d806fe40d6079dd5 100644
--- a/pkg/cnservice/server.go
+++ b/pkg/cnservice/server.go
@@ -36,7 +36,9 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/pb/pipeline"
 )
 
-func NewService(cfg *Config, ctx context.Context) (Service, error) {
+type Options func(*service)
+
+func NewService(cfg *Config, ctx context.Context, options ...Options) (Service, error) {
 
 	if err := cfg.Validate(); err != nil {
 		return nil, err
@@ -44,26 +46,31 @@ func NewService(cfg *Config, ctx context.Context) (Service, error) {
 
 	srv := &service{cfg: cfg}
 	srv.logger = logutil.Adjust(srv.logger)
-	srv.pool = &sync.Pool{
+	srv.responsePool = &sync.Pool{
 		New: func() any {
 			return &pipeline.Message{}
 		},
 	}
 
+	pu := config.NewParameterUnit(&cfg.Frontend, nil, nil, nil, nil, nil)
+	cfg.Frontend.SetDefaultValues()
+	err := srv.initMOServer(ctx, pu)
+	if err != nil {
+		return nil, err
+	}
+
 	server, err := morpc.NewRPCServer("cn-server", cfg.ListenAddress,
-		morpc.NewMessageCodec(srv.acquireMessage, 16<<20),
-		morpc.WithServerGoettyOptions(goetty.WithSessionRWBUfferSize(1<<20, 1<<20)))
+		morpc.NewMessageCodec(srv.acquireMessage, cfg.PayLoadCopyBufferSize),
+		morpc.WithServerGoettyOptions(goetty.WithSessionRWBUfferSize(cfg.ReadBufferSize, cfg.WriteBufferSize)))
 	if err != nil {
 		return nil, err
 	}
 	server.RegisterRequestHandler(compile.NewServer().HandleRequest)
 	srv.server = server
 
-	pu := config.NewParameterUnit(&cfg.Frontend, nil, nil, nil, nil, nil)
-	cfg.Frontend.SetDefaultValues()
-	err = srv.initMOServer(ctx, pu)
-	if err != nil {
-		return nil, err
+	srv.requestHandler = defaultRequestHandler
+	for _, opt := range options {
+		opt(srv)
 	}
 
 	return srv, nil
@@ -87,15 +94,16 @@ func (s *service) Close() error {
 }
 
 func (s *service) acquireMessage() morpc.Message {
-	return s.pool.Get().(*pipeline.Message)
+	return s.responsePool.Get().(*pipeline.Message)
 }
 
-/*
-func (s *service) releaseMessage(msg *pipeline.Message) {
-	msg.Reset()
-	s.pool.Put(msg)
+func defaultRequestHandler(ctx context.Context, message morpc.Message, cs morpc.ClientSession) error {
+	return nil
 }
-*/
+
+//func (s *service) handleRequest(ctx context.Context, req morpc.Message, _ uint64, cs morpc.ClientSession) error {
+//	return s.requestHandler(ctx, req, cs)
+//}
 
 func (s *service) initMOServer(ctx context.Context, pu *config.ParameterUnit) error {
 	var err error
@@ -234,3 +242,9 @@ func (s *service) getTxnClient() (c client.TxnClient, err error) {
 	c = s._txnClient
 	return
 }
+
+func WithMessageHandle(f func(ctx context.Context, message morpc.Message, cs morpc.ClientSession) error) Options {
+	return func(s *service) {
+		s.requestHandler = f
+	}
+}
diff --git a/pkg/cnservice/types.go b/pkg/cnservice/types.go
index fc624038e7674f01bb399851c3e9d91a0b6aee4b..b0f89369dbad1fcfcb38ca6445ee9dc53fa7b0fc 100644
--- a/pkg/cnservice/types.go
+++ b/pkg/cnservice/types.go
@@ -61,6 +61,11 @@ type Config struct {
 		S3 fileservice.S3Config `toml:"s3"`
 	}
 
+	// parameters for cn-server related buffer.
+	PayLoadCopyBufferSize int
+	ReadBufferSize        int
+	WriteBufferSize       int
+
 	// Pipeline configuration
 	Pipeline struct {
 		// HostSize is the memory limit
@@ -109,9 +114,10 @@ func (c *Config) Validate() error {
 
 type service struct {
 	cfg                    *Config
-	pool                   *sync.Pool
+	responsePool           *sync.Pool
 	logger                 *zap.Logger
 	server                 morpc.RPCServer
+	requestHandler         func(ctx context.Context, message morpc.Message, cs morpc.ClientSession) error
 	cancelMoServerFunc     context.CancelFunc
 	mo                     *frontend.MOServer
 	initHakeeperClientOnce sync.Once
diff --git a/pkg/pb/pipeline/pipeline.go b/pkg/pb/pipeline/pipeline.go
index d048419076eed759e511966c158d87542b01be03..b662a055d3a428192b9b50771d70069d76d87de2 100644
--- a/pkg/pb/pipeline/pipeline.go
+++ b/pkg/pb/pipeline/pipeline.go
@@ -14,7 +14,9 @@
 
 package pipeline
 
-import fmt "fmt"
+import "fmt"
+
+const MessageEnd = 1
 
 func (m *Message) Size() int {
 	return m.ProtoSize()
diff --git a/pkg/pb/pipeline/pipeline.pb.go b/pkg/pb/pipeline/pipeline.pb.go
index 125dad345c560a20493a7e284e2bc30eee508abd..05eddd2bb9a5a631d1e16cf57953f33110f23f7b 100644
--- a/pkg/pb/pipeline/pipeline.pb.go
+++ b/pkg/pb/pipeline/pipeline.pb.go
@@ -53,7 +53,7 @@ func (x Pipeline_PipelineType) String() string {
 }
 
 func (Pipeline_PipelineType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{13, 0}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{17, 0}
 }
 
 type Message struct {
@@ -61,6 +61,7 @@ type Message struct {
 	Cmd                  uint64   `protobuf:"varint,2,opt,name=cmd,proto3" json:"cmd,omitempty"`
 	Code                 []byte   `protobuf:"bytes,3,opt,name=code,proto3" json:"code,omitempty"`
 	Data                 []byte   `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
+	Analyse              []byte   `protobuf:"bytes,5,opt,name=analyse,proto3" json:"analyse,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
@@ -127,19 +128,81 @@ func (m *Message) GetData() []byte {
 	return nil
 }
 
-type Dispatch struct {
-	All                  bool     `protobuf:"varint,1,opt,name=all,proto3" json:"all,omitempty"`
-	Children             []int32  `protobuf:"varint,2,rep,packed,name=children,proto3" json:"children,omitempty"`
+func (m *Message) GetAnalyse() []byte {
+	if m != nil {
+		return m.Analyse
+	}
+	return nil
+}
+
+type Connector struct {
+	PipelineId           int32    `protobuf:"varint,1,opt,name=pipeline_id,json=pipelineId,proto3" json:"pipeline_id,omitempty"`
+	ConnectorIndex       int32    `protobuf:"varint,2,opt,name=connector_index,json=connectorIndex,proto3" json:"connector_index,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
 }
 
+func (m *Connector) Reset()         { *m = Connector{} }
+func (m *Connector) String() string { return proto.CompactTextString(m) }
+func (*Connector) ProtoMessage()    {}
+func (*Connector) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ac67a7adf3df9c7, []int{1}
+}
+func (m *Connector) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Connector) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Connector.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Connector) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Connector.Merge(m, src)
+}
+func (m *Connector) XXX_Size() int {
+	return m.ProtoSize()
+}
+func (m *Connector) XXX_DiscardUnknown() {
+	xxx_messageInfo_Connector.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Connector proto.InternalMessageInfo
+
+func (m *Connector) GetPipelineId() int32 {
+	if m != nil {
+		return m.PipelineId
+	}
+	return 0
+}
+
+func (m *Connector) GetConnectorIndex() int32 {
+	if m != nil {
+		return m.ConnectorIndex
+	}
+	return 0
+}
+
+type Dispatch struct {
+	All                  bool         `protobuf:"varint,1,opt,name=all,proto3" json:"all,omitempty"`
+	Connector            []*Connector `protobuf:"bytes,2,rep,name=connector,proto3" json:"connector,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
 func (m *Dispatch) Reset()         { *m = Dispatch{} }
 func (m *Dispatch) String() string { return proto.CompactTextString(m) }
 func (*Dispatch) ProtoMessage()    {}
 func (*Dispatch) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{1}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{2}
 }
 func (m *Dispatch) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -175,9 +238,9 @@ func (m *Dispatch) GetAll() bool {
 	return false
 }
 
-func (m *Dispatch) GetChildren() []int32 {
+func (m *Dispatch) GetConnector() []*Connector {
 	if m != nil {
-		return m.Children
+		return m.Connector
 	}
 	return nil
 }
@@ -195,7 +258,7 @@ func (m *Aggregate) Reset()         { *m = Aggregate{} }
 func (m *Aggregate) String() string { return proto.CompactTextString(m) }
 func (*Aggregate) ProtoMessage()    {}
 func (*Aggregate) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{2}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{3}
 }
 func (m *Aggregate) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -261,7 +324,7 @@ func (m *Group) Reset()         { *m = Group{} }
 func (m *Group) String() string { return proto.CompactTextString(m) }
 func (*Group) ProtoMessage()    {}
 func (*Group) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{3}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{4}
 }
 func (m *Group) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -332,12 +395,117 @@ func (m *Group) GetAggs() []*Aggregate {
 	return nil
 }
 
+type Join struct {
+	Ibucket              uint64       `protobuf:"varint,1,opt,name=ibucket,proto3" json:"ibucket,omitempty"`
+	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
+	RelList              []int32      `protobuf:"varint,3,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
+	ColList              []int32      `protobuf:"varint,4,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,5,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,6,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,7,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,8,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *Join) Reset()         { *m = Join{} }
+func (m *Join) String() string { return proto.CompactTextString(m) }
+func (*Join) ProtoMessage()    {}
+func (*Join) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ac67a7adf3df9c7, []int{5}
+}
+func (m *Join) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Join) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Join.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Join) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Join.Merge(m, src)
+}
+func (m *Join) XXX_Size() int {
+	return m.ProtoSize()
+}
+func (m *Join) XXX_DiscardUnknown() {
+	xxx_messageInfo_Join.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Join proto.InternalMessageInfo
+
+func (m *Join) GetIbucket() uint64 {
+	if m != nil {
+		return m.Ibucket
+	}
+	return 0
+}
+
+func (m *Join) GetNbucket() uint64 {
+	if m != nil {
+		return m.Nbucket
+	}
+	return 0
+}
+
+func (m *Join) GetRelList() []int32 {
+	if m != nil {
+		return m.RelList
+	}
+	return nil
+}
+
+func (m *Join) GetColList() []int32 {
+	if m != nil {
+		return m.ColList
+	}
+	return nil
+}
+
+func (m *Join) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
+func (m *Join) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
+func (m *Join) GetLeftCond() []*plan.Expr {
+	if m != nil {
+		return m.LeftCond
+	}
+	return nil
+}
+
+func (m *Join) GetRightCond() []*plan.Expr {
+	if m != nil {
+		return m.RightCond
+	}
+	return nil
+}
+
 type AntiJoin struct {
 	Ibucket              uint64       `protobuf:"varint,1,opt,name=ibucket,proto3" json:"ibucket,omitempty"`
 	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
 	Result               []int32      `protobuf:"varint,3,rep,packed,name=result,proto3" json:"result,omitempty"`
-	LeftCond             []*plan.Expr `protobuf:"bytes,4,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
-	RightCond            []*plan.Expr `protobuf:"bytes,5,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,4,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,5,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,6,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,7,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
 	XXX_unrecognized     []byte       `json:"-"`
 	XXX_sizecache        int32        `json:"-"`
@@ -347,7 +515,7 @@ func (m *AntiJoin) Reset()         { *m = AntiJoin{} }
 func (m *AntiJoin) String() string { return proto.CompactTextString(m) }
 func (*AntiJoin) ProtoMessage()    {}
 func (*AntiJoin) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{4}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{6}
 }
 func (m *AntiJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -397,6 +565,20 @@ func (m *AntiJoin) GetResult() []int32 {
 	return nil
 }
 
+func (m *AntiJoin) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
+func (m *AntiJoin) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
 func (m *AntiJoin) GetLeftCond() []*plan.Expr {
 	if m != nil {
 		return m.LeftCond
@@ -416,8 +598,10 @@ type InnerJoin struct {
 	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
 	RelList              []int32      `protobuf:"varint,3,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
 	ColList              []int32      `protobuf:"varint,4,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
-	LeftCond             []*plan.Expr `protobuf:"bytes,5,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
-	RightCond            []*plan.Expr `protobuf:"bytes,6,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,5,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,6,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,7,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,8,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
 	XXX_unrecognized     []byte       `json:"-"`
 	XXX_sizecache        int32        `json:"-"`
@@ -427,7 +611,7 @@ func (m *InnerJoin) Reset()         { *m = InnerJoin{} }
 func (m *InnerJoin) String() string { return proto.CompactTextString(m) }
 func (*InnerJoin) ProtoMessage()    {}
 func (*InnerJoin) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{5}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{7}
 }
 func (m *InnerJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -484,6 +668,20 @@ func (m *InnerJoin) GetColList() []int32 {
 	return nil
 }
 
+func (m *InnerJoin) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
+func (m *InnerJoin) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
 func (m *InnerJoin) GetLeftCond() []*plan.Expr {
 	if m != nil {
 		return m.LeftCond
@@ -503,9 +701,10 @@ type LeftJoin struct {
 	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
 	RelList              []int32      `protobuf:"varint,3,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
 	ColList              []int32      `protobuf:"varint,4,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
-	Types                []*plan.Type `protobuf:"bytes,5,rep,name=types,proto3" json:"types,omitempty"`
-	LeftCond             []*plan.Expr `protobuf:"bytes,6,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
-	RightCond            []*plan.Expr `protobuf:"bytes,7,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,5,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,6,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,7,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,8,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
 	XXX_unrecognized     []byte       `json:"-"`
 	XXX_sizecache        int32        `json:"-"`
@@ -515,7 +714,7 @@ func (m *LeftJoin) Reset()         { *m = LeftJoin{} }
 func (m *LeftJoin) String() string { return proto.CompactTextString(m) }
 func (*LeftJoin) ProtoMessage()    {}
 func (*LeftJoin) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{6}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{8}
 }
 func (m *LeftJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -572,6 +771,13 @@ func (m *LeftJoin) GetColList() []int32 {
 	return nil
 }
 
+func (m *LeftJoin) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
 func (m *LeftJoin) GetTypes() []*plan.Type {
 	if m != nil {
 		return m.Types
@@ -597,8 +803,10 @@ type SemiJoin struct {
 	Ibucket              uint64       `protobuf:"varint,1,opt,name=ibucket,proto3" json:"ibucket,omitempty"`
 	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
 	Result               []int32      `protobuf:"varint,3,rep,packed,name=result,proto3" json:"result,omitempty"`
-	LeftCond             []*plan.Expr `protobuf:"bytes,4,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
-	RightCond            []*plan.Expr `protobuf:"bytes,5,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,4,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,5,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,6,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,7,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
 	XXX_unrecognized     []byte       `json:"-"`
 	XXX_sizecache        int32        `json:"-"`
@@ -608,7 +816,7 @@ func (m *SemiJoin) Reset()         { *m = SemiJoin{} }
 func (m *SemiJoin) String() string { return proto.CompactTextString(m) }
 func (*SemiJoin) ProtoMessage()    {}
 func (*SemiJoin) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{7}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{9}
 }
 func (m *SemiJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -658,6 +866,20 @@ func (m *SemiJoin) GetResult() []int32 {
 	return nil
 }
 
+func (m *SemiJoin) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
+func (m *SemiJoin) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
 func (m *SemiJoin) GetLeftCond() []*plan.Expr {
 	if m != nil {
 		return m.LeftCond
@@ -677,9 +899,10 @@ type SingleJoin struct {
 	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
 	RelList              []int32      `protobuf:"varint,3,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
 	ColList              []int32      `protobuf:"varint,4,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
-	Types                []*plan.Type `protobuf:"bytes,5,rep,name=types,proto3" json:"types,omitempty"`
-	LeftCond             []*plan.Expr `protobuf:"bytes,6,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
-	RightCond            []*plan.Expr `protobuf:"bytes,7,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Expr                 *plan.Expr   `protobuf:"bytes,5,opt,name=expr,proto3" json:"expr,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,6,rep,name=types,proto3" json:"types,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,7,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,8,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
 	XXX_unrecognized     []byte       `json:"-"`
 	XXX_sizecache        int32        `json:"-"`
@@ -689,7 +912,7 @@ func (m *SingleJoin) Reset()         { *m = SingleJoin{} }
 func (m *SingleJoin) String() string { return proto.CompactTextString(m) }
 func (*SingleJoin) ProtoMessage()    {}
 func (*SingleJoin) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{8}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{10}
 }
 func (m *SingleJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -746,6 +969,13 @@ func (m *SingleJoin) GetColList() []int32 {
 	return nil
 }
 
+func (m *SingleJoin) GetExpr() *plan.Expr {
+	if m != nil {
+		return m.Expr
+	}
+	return nil
+}
+
 func (m *SingleJoin) GetTypes() []*plan.Type {
 	if m != nil {
 		return m.Types
@@ -767,26 +997,36 @@ func (m *SingleJoin) GetRightCond() []*plan.Expr {
 	return nil
 }
 
-type Product struct {
-	RelList              []int32  `protobuf:"varint,1,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
-	ColList              []int32  `protobuf:"varint,2,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+type MarkJoin struct {
+	Ibucket              uint64       `protobuf:"varint,1,opt,name=ibucket,proto3" json:"ibucket,omitempty"`
+	Nbucket              uint64       `protobuf:"varint,2,opt,name=nbucket,proto3" json:"nbucket,omitempty"`
+	Result               []int32      `protobuf:"varint,3,rep,packed,name=result,proto3" json:"result,omitempty"`
+	LeftCond             []*plan.Expr `protobuf:"bytes,4,rep,name=left_cond,json=leftCond,proto3" json:"left_cond,omitempty"`
+	RightCond            []*plan.Expr `protobuf:"bytes,5,rep,name=right_cond,json=rightCond,proto3" json:"right_cond,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,6,rep,name=types,proto3" json:"types,omitempty"`
+	Cond                 *plan.Expr   `protobuf:"bytes,7,opt,name=cond,proto3" json:"cond,omitempty"`
+	OnList               []*plan.Expr `protobuf:"bytes,8,rep,name=on_list,json=onList,proto3" json:"on_list,omitempty"`
+	OutputNull           bool         `protobuf:"varint,9,opt,name=output_null,json=outputNull,proto3" json:"output_null,omitempty"`
+	OutputMark           bool         `protobuf:"varint,10,opt,name=output_mark,json=outputMark,proto3" json:"output_mark,omitempty"`
+	OutputAnyway         bool         `protobuf:"varint,11,opt,name=output_anyway,json=outputAnyway,proto3" json:"output_anyway,omitempty"`
+	MarkMeaning          bool         `protobuf:"varint,12,opt,name=mark_meaning,json=markMeaning,proto3" json:"mark_meaning,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
 }
 
-func (m *Product) Reset()         { *m = Product{} }
-func (m *Product) String() string { return proto.CompactTextString(m) }
-func (*Product) ProtoMessage()    {}
-func (*Product) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{9}
+func (m *MarkJoin) Reset()         { *m = MarkJoin{} }
+func (m *MarkJoin) String() string { return proto.CompactTextString(m) }
+func (*MarkJoin) ProtoMessage()    {}
+func (*MarkJoin) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ac67a7adf3df9c7, []int{11}
 }
-func (m *Product) XXX_Unmarshal(b []byte) error {
+func (m *MarkJoin) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
 }
-func (m *Product) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+func (m *MarkJoin) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	if deterministic {
-		return xxx_messageInfo_Product.Marshal(b, m, deterministic)
+		return xxx_messageInfo_MarkJoin.Marshal(b, m, deterministic)
 	} else {
 		b = b[:cap(b)]
 		n, err := m.MarshalToSizedBuffer(b)
@@ -796,60 +1036,196 @@ func (m *Product) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 		return b[:n], nil
 	}
 }
-func (m *Product) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Product.Merge(m, src)
+func (m *MarkJoin) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MarkJoin.Merge(m, src)
 }
-func (m *Product) XXX_Size() int {
+func (m *MarkJoin) XXX_Size() int {
 	return m.ProtoSize()
 }
-func (m *Product) XXX_DiscardUnknown() {
-	xxx_messageInfo_Product.DiscardUnknown(m)
+func (m *MarkJoin) XXX_DiscardUnknown() {
+	xxx_messageInfo_MarkJoin.DiscardUnknown(m)
 }
 
-var xxx_messageInfo_Product proto.InternalMessageInfo
+var xxx_messageInfo_MarkJoin proto.InternalMessageInfo
 
-func (m *Product) GetRelList() []int32 {
+func (m *MarkJoin) GetIbucket() uint64 {
 	if m != nil {
-		return m.RelList
+		return m.Ibucket
 	}
-	return nil
+	return 0
 }
 
-func (m *Product) GetColList() []int32 {
+func (m *MarkJoin) GetNbucket() uint64 {
 	if m != nil {
-		return m.ColList
+		return m.Nbucket
+	}
+	return 0
+}
+
+func (m *MarkJoin) GetResult() []int32 {
+	if m != nil {
+		return m.Result
 	}
 	return nil
 }
 
-type Instruction struct {
-	// Op specified the operator code of an instruction.
-	Op int32 `protobuf:"varint,1,opt,name=op,proto3" json:"op,omitempty"`
-	// Idx specified the anaylze information index.
-	Idx                  int32               `protobuf:"varint,2,opt,name=idx,proto3" json:"idx,omitempty"`
-	Anti                 *AntiJoin           `protobuf:"bytes,3,opt,name=anti,proto3" json:"anti,omitempty"`
-	Dispatch             *Dispatch           `protobuf:"bytes,4,opt,name=dispatch,proto3" json:"dispatch,omitempty"`
-	Agg                  *Group              `protobuf:"bytes,5,opt,name=agg,proto3" json:"agg,omitempty"`
-	InnerJoin            *InnerJoin          `protobuf:"bytes,6,opt,name=inner_join,json=innerJoin,proto3" json:"inner_join,omitempty"`
-	LeftJoin             *LeftJoin           `protobuf:"bytes,7,opt,name=left_join,json=leftJoin,proto3" json:"left_join,omitempty"`
-	SemiJoin             *SemiJoin           `protobuf:"bytes,8,opt,name=semi_join,json=semiJoin,proto3" json:"semi_join,omitempty"`
-	SingleJoin           *SingleJoin         `protobuf:"bytes,9,opt,name=single_join,json=singleJoin,proto3" json:"single_join,omitempty"`
-	Product              *Product            `protobuf:"bytes,10,opt,name=product,proto3" json:"product,omitempty"`
-	OrderBy              []*plan.OrderBySpec `protobuf:"bytes,11,rep,name=order_by,json=orderBy,proto3" json:"order_by,omitempty"`
-	ProjectList          []*plan.Expr        `protobuf:"bytes,12,rep,name=project_list,json=projectList,proto3" json:"project_list,omitempty"`
-	Filter               *plan.Expr          `protobuf:"bytes,13,opt,name=filter,proto3" json:"filter,omitempty"`
-	Limit                uint64              `protobuf:"varint,14,opt,name=limit,proto3" json:"limit,omitempty"`
-	Offset               uint64              `protobuf:"varint,15,opt,name=offset,proto3" json:"offset,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
+func (m *MarkJoin) GetLeftCond() []*plan.Expr {
+	if m != nil {
+		return m.LeftCond
+	}
+	return nil
 }
 
-func (m *Instruction) Reset()         { *m = Instruction{} }
+func (m *MarkJoin) GetRightCond() []*plan.Expr {
+	if m != nil {
+		return m.RightCond
+	}
+	return nil
+}
+
+func (m *MarkJoin) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
+func (m *MarkJoin) GetCond() *plan.Expr {
+	if m != nil {
+		return m.Cond
+	}
+	return nil
+}
+
+func (m *MarkJoin) GetOnList() []*plan.Expr {
+	if m != nil {
+		return m.OnList
+	}
+	return nil
+}
+
+func (m *MarkJoin) GetOutputNull() bool {
+	if m != nil {
+		return m.OutputNull
+	}
+	return false
+}
+
+func (m *MarkJoin) GetOutputMark() bool {
+	if m != nil {
+		return m.OutputMark
+	}
+	return false
+}
+
+func (m *MarkJoin) GetOutputAnyway() bool {
+	if m != nil {
+		return m.OutputAnyway
+	}
+	return false
+}
+
+func (m *MarkJoin) GetMarkMeaning() bool {
+	if m != nil {
+		return m.MarkMeaning
+	}
+	return false
+}
+
+type Product struct {
+	RelList              []int32      `protobuf:"varint,1,rep,packed,name=rel_list,json=relList,proto3" json:"rel_list,omitempty"`
+	ColList              []int32      `protobuf:"varint,2,rep,packed,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
+	Types                []*plan.Type `protobuf:"bytes,3,rep,name=types,proto3" json:"types,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}     `json:"-"`
+	XXX_unrecognized     []byte       `json:"-"`
+	XXX_sizecache        int32        `json:"-"`
+}
+
+func (m *Product) Reset()         { *m = Product{} }
+func (m *Product) String() string { return proto.CompactTextString(m) }
+func (*Product) ProtoMessage()    {}
+func (*Product) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ac67a7adf3df9c7, []int{12}
+}
+func (m *Product) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *Product) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_Product.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *Product) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Product.Merge(m, src)
+}
+func (m *Product) XXX_Size() int {
+	return m.ProtoSize()
+}
+func (m *Product) XXX_DiscardUnknown() {
+	xxx_messageInfo_Product.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_Product proto.InternalMessageInfo
+
+func (m *Product) GetRelList() []int32 {
+	if m != nil {
+		return m.RelList
+	}
+	return nil
+}
+
+func (m *Product) GetColList() []int32 {
+	if m != nil {
+		return m.ColList
+	}
+	return nil
+}
+
+func (m *Product) GetTypes() []*plan.Type {
+	if m != nil {
+		return m.Types
+	}
+	return nil
+}
+
+type Instruction struct {
+	// Op specified the operator code of an instruction.
+	Op int32 `protobuf:"varint,1,opt,name=op,proto3" json:"op,omitempty"`
+	// Idx specified the anaylze information index.
+	Idx                  int32               `protobuf:"varint,2,opt,name=idx,proto3" json:"idx,omitempty"`
+	Anti                 *AntiJoin           `protobuf:"bytes,3,opt,name=anti,proto3" json:"anti,omitempty"`
+	Connect              *Connector          `protobuf:"bytes,4,opt,name=connect,proto3" json:"connect,omitempty"`
+	Dispatch             *Dispatch           `protobuf:"bytes,5,opt,name=dispatch,proto3" json:"dispatch,omitempty"`
+	Agg                  *Group              `protobuf:"bytes,6,opt,name=agg,proto3" json:"agg,omitempty"`
+	InnerJoin            *InnerJoin          `protobuf:"bytes,7,opt,name=inner_join,json=innerJoin,proto3" json:"inner_join,omitempty"`
+	LeftJoin             *LeftJoin           `protobuf:"bytes,8,opt,name=left_join,json=leftJoin,proto3" json:"left_join,omitempty"`
+	SemiJoin             *SemiJoin           `protobuf:"bytes,9,opt,name=semi_join,json=semiJoin,proto3" json:"semi_join,omitempty"`
+	SingleJoin           *SingleJoin         `protobuf:"bytes,10,opt,name=single_join,json=singleJoin,proto3" json:"single_join,omitempty"`
+	MarkJoin             *MarkJoin           `protobuf:"bytes,11,opt,name=mark_join,json=markJoin,proto3" json:"mark_join,omitempty"`
+	Join                 *Join               `protobuf:"bytes,12,opt,name=join,proto3" json:"join,omitempty"`
+	Product              *Product            `protobuf:"bytes,13,opt,name=product,proto3" json:"product,omitempty"`
+	OrderBy              []*plan.OrderBySpec `protobuf:"bytes,14,rep,name=order_by,json=orderBy,proto3" json:"order_by,omitempty"`
+	ProjectList          []*plan.Expr        `protobuf:"bytes,15,rep,name=project_list,json=projectList,proto3" json:"project_list,omitempty"`
+	Filter               *plan.Expr          `protobuf:"bytes,16,opt,name=filter,proto3" json:"filter,omitempty"`
+	Limit                uint64              `protobuf:"varint,17,opt,name=limit,proto3" json:"limit,omitempty"`
+	Offset               uint64              `protobuf:"varint,18,opt,name=offset,proto3" json:"offset,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *Instruction) Reset()         { *m = Instruction{} }
 func (m *Instruction) String() string { return proto.CompactTextString(m) }
 func (*Instruction) ProtoMessage()    {}
 func (*Instruction) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{10}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{13}
 }
 func (m *Instruction) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -899,6 +1275,13 @@ func (m *Instruction) GetAnti() *AntiJoin {
 	return nil
 }
 
+func (m *Instruction) GetConnect() *Connector {
+	if m != nil {
+		return m.Connect
+	}
+	return nil
+}
+
 func (m *Instruction) GetDispatch() *Dispatch {
 	if m != nil {
 		return m.Dispatch
@@ -941,6 +1324,20 @@ func (m *Instruction) GetSingleJoin() *SingleJoin {
 	return nil
 }
 
+func (m *Instruction) GetMarkJoin() *MarkJoin {
+	if m != nil {
+		return m.MarkJoin
+	}
+	return nil
+}
+
+func (m *Instruction) GetJoin() *Join {
+	if m != nil {
+		return m.Join
+	}
+	return nil
+}
+
 func (m *Instruction) GetProduct() *Product {
 	if m != nil {
 		return m.Product
@@ -983,11 +1380,60 @@ func (m *Instruction) GetOffset() uint64 {
 	return 0
 }
 
+type AnalysisList struct {
+	List                 []*plan.AnalyzeInfo `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
+	XXX_unrecognized     []byte              `json:"-"`
+	XXX_sizecache        int32               `json:"-"`
+}
+
+func (m *AnalysisList) Reset()         { *m = AnalysisList{} }
+func (m *AnalysisList) String() string { return proto.CompactTextString(m) }
+func (*AnalysisList) ProtoMessage()    {}
+func (*AnalysisList) Descriptor() ([]byte, []int) {
+	return fileDescriptor_7ac67a7adf3df9c7, []int{14}
+}
+func (m *AnalysisList) XXX_Unmarshal(b []byte) error {
+	return m.Unmarshal(b)
+}
+func (m *AnalysisList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	if deterministic {
+		return xxx_messageInfo_AnalysisList.Marshal(b, m, deterministic)
+	} else {
+		b = b[:cap(b)]
+		n, err := m.MarshalToSizedBuffer(b)
+		if err != nil {
+			return nil, err
+		}
+		return b[:n], nil
+	}
+}
+func (m *AnalysisList) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AnalysisList.Merge(m, src)
+}
+func (m *AnalysisList) XXX_Size() int {
+	return m.ProtoSize()
+}
+func (m *AnalysisList) XXX_DiscardUnknown() {
+	xxx_messageInfo_AnalysisList.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_AnalysisList proto.InternalMessageInfo
+
+func (m *AnalysisList) GetList() []*plan.AnalyzeInfo {
+	if m != nil {
+		return m.List
+	}
+	return nil
+}
+
 type Source struct {
 	SchemaName           string   `protobuf:"bytes,1,opt,name=schema_name,json=schemaName,proto3" json:"schema_name,omitempty"`
 	TableName            string   `protobuf:"bytes,2,opt,name=table_name,json=tableName,proto3" json:"table_name,omitempty"`
 	ColList              []string `protobuf:"bytes,3,rep,name=col_list,json=colList,proto3" json:"col_list,omitempty"`
 	Block                string   `protobuf:"bytes,4,opt,name=block,proto3" json:"block,omitempty"`
+	PushdownId           uint64   `protobuf:"varint,5,opt,name=pushdown_id,json=pushdownId,proto3" json:"pushdown_id,omitempty"`
+	PushdownAddr         string   `protobuf:"bytes,6,opt,name=pushdown_addr,json=pushdownAddr,proto3" json:"pushdown_addr,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
@@ -997,7 +1443,7 @@ func (m *Source) Reset()         { *m = Source{} }
 func (m *Source) String() string { return proto.CompactTextString(m) }
 func (*Source) ProtoMessage()    {}
 func (*Source) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{11}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{15}
 }
 func (m *Source) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -1054,6 +1500,20 @@ func (m *Source) GetBlock() string {
 	return ""
 }
 
+func (m *Source) GetPushdownId() uint64 {
+	if m != nil {
+		return m.PushdownId
+	}
+	return 0
+}
+
+func (m *Source) GetPushdownAddr() string {
+	if m != nil {
+		return m.PushdownAddr
+	}
+	return ""
+}
+
 type NodeInfo struct {
 	Mcpu                 int32    `protobuf:"varint,1,opt,name=mcpu,proto3" json:"mcpu,omitempty"`
 	Id                   string   `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
@@ -1068,7 +1528,7 @@ func (m *NodeInfo) Reset()         { *m = NodeInfo{} }
 func (m *NodeInfo) String() string { return proto.CompactTextString(m) }
 func (*NodeInfo) ProtoMessage()    {}
 func (*NodeInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{12}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{16}
 }
 func (m *NodeInfo) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -1133,8 +1593,10 @@ type Pipeline struct {
 	Children             []*Pipeline           `protobuf:"bytes,5,rep,name=children,proto3" json:"children,omitempty"`
 	InstructionList      []*Instruction        `protobuf:"bytes,6,rep,name=instruction_list,json=instructionList,proto3" json:"instruction_list,omitempty"`
 	IsEnd                bool                  `protobuf:"varint,7,opt,name=is_end,json=isEnd,proto3" json:"is_end,omitempty"`
-	Node                 *NodeInfo             `protobuf:"bytes,8,opt,name=node,proto3" json:"node,omitempty"`
-	PushDownInfo         int32                 `protobuf:"varint,9,opt,name=push_down_info,json=pushDownInfo,proto3" json:"push_down_info,omitempty"`
+	IsJoin               bool                  `protobuf:"varint,8,opt,name=is_join,json=isJoin,proto3" json:"is_join,omitempty"`
+	Node                 *NodeInfo             `protobuf:"bytes,9,opt,name=node,proto3" json:"node,omitempty"`
+	PushDownInfo         int32                 `protobuf:"varint,10,opt,name=push_down_info,json=pushDownInfo,proto3" json:"push_down_info,omitempty"`
+	ChildrenCount        int32                 `protobuf:"varint,11,opt,name=children_count,json=childrenCount,proto3" json:"children_count,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
 	XXX_unrecognized     []byte                `json:"-"`
 	XXX_sizecache        int32                 `json:"-"`
@@ -1144,7 +1606,7 @@ func (m *Pipeline) Reset()         { *m = Pipeline{} }
 func (m *Pipeline) String() string { return proto.CompactTextString(m) }
 func (*Pipeline) ProtoMessage()    {}
 func (*Pipeline) Descriptor() ([]byte, []int) {
-	return fileDescriptor_7ac67a7adf3df9c7, []int{13}
+	return fileDescriptor_7ac67a7adf3df9c7, []int{17}
 }
 func (m *Pipeline) XXX_Unmarshal(b []byte) error {
 	return m.Unmarshal(b)
@@ -1222,6 +1684,13 @@ func (m *Pipeline) GetIsEnd() bool {
 	return false
 }
 
+func (m *Pipeline) GetIsJoin() bool {
+	if m != nil {
+		return m.IsJoin
+	}
+	return false
+}
+
 func (m *Pipeline) GetNode() *NodeInfo {
 	if m != nil {
 		return m.Node
@@ -1236,19 +1705,30 @@ func (m *Pipeline) GetPushDownInfo() int32 {
 	return 0
 }
 
+func (m *Pipeline) GetChildrenCount() int32 {
+	if m != nil {
+		return m.ChildrenCount
+	}
+	return 0
+}
+
 func init() {
 	proto.RegisterEnum("pipeline.Pipeline_PipelineType", Pipeline_PipelineType_name, Pipeline_PipelineType_value)
 	proto.RegisterType((*Message)(nil), "pipeline.Message")
+	proto.RegisterType((*Connector)(nil), "pipeline.Connector")
 	proto.RegisterType((*Dispatch)(nil), "pipeline.Dispatch")
 	proto.RegisterType((*Aggregate)(nil), "pipeline.Aggregate")
 	proto.RegisterType((*Group)(nil), "pipeline.Group")
+	proto.RegisterType((*Join)(nil), "pipeline.Join")
 	proto.RegisterType((*AntiJoin)(nil), "pipeline.AntiJoin")
 	proto.RegisterType((*InnerJoin)(nil), "pipeline.InnerJoin")
 	proto.RegisterType((*LeftJoin)(nil), "pipeline.LeftJoin")
 	proto.RegisterType((*SemiJoin)(nil), "pipeline.SemiJoin")
 	proto.RegisterType((*SingleJoin)(nil), "pipeline.SingleJoin")
+	proto.RegisterType((*MarkJoin)(nil), "pipeline.MarkJoin")
 	proto.RegisterType((*Product)(nil), "pipeline.Product")
 	proto.RegisterType((*Instruction)(nil), "pipeline.Instruction")
+	proto.RegisterType((*AnalysisList)(nil), "pipeline.AnalysisList")
 	proto.RegisterType((*Source)(nil), "pipeline.Source")
 	proto.RegisterType((*NodeInfo)(nil), "pipeline.NodeInfo")
 	proto.RegisterType((*Pipeline)(nil), "pipeline.Pipeline")
@@ -1257,78 +1737,98 @@ func init() {
 func init() { proto.RegisterFile("pipeline.proto", fileDescriptor_7ac67a7adf3df9c7) }
 
 var fileDescriptor_7ac67a7adf3df9c7 = []byte{
-	// 1125 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x56, 0x4f, 0x8f, 0xdb, 0x44,
-	0x14, 0xaf, 0x93, 0x38, 0xb1, 0x5f, 0xd2, 0x6d, 0x3a, 0xb4, 0xc8, 0x2d, 0xb0, 0x5d, 0x2c, 0x44,
-	0x8b, 0xa0, 0x89, 0x5a, 0x84, 0xc4, 0xad, 0xf4, 0x9f, 0xd0, 0xa2, 0x76, 0xbb, 0x9a, 0x85, 0x0b,
-	0x42, 0x8a, 0x26, 0xf6, 0xc4, 0x99, 0xae, 0x3d, 0x63, 0xc6, 0x4e, 0xbb, 0xf9, 0x36, 0x9c, 0xf9,
-	0x1e, 0x48, 0x70, 0x43, 0xe2, 0xc6, 0x09, 0x2d, 0x37, 0xc4, 0x87, 0x40, 0xf3, 0xc6, 0x4e, 0x9c,
-	0x0d, 0xad, 0x56, 0x1c, 0x90, 0x10, 0xb7, 0xf7, 0xd7, 0xf3, 0xfe, 0xfe, 0xfc, 0x60, 0x27, 0x17,
-	0x39, 0x4f, 0x85, 0xe4, 0xa3, 0x5c, 0xab, 0x52, 0x11, 0xaf, 0xe6, 0xaf, 0xdf, 0x4e, 0x44, 0x39,
-	0x5f, 0x4c, 0x47, 0x91, 0xca, 0xc6, 0x89, 0x4a, 0xd4, 0x18, 0x0d, 0xa6, 0x8b, 0x19, 0x72, 0xc8,
-	0x20, 0x65, 0x1d, 0xaf, 0x43, 0x9e, 0x32, 0x69, 0xe9, 0xf0, 0x2b, 0xe8, 0x3d, 0xe5, 0x45, 0xc1,
-	0x12, 0x4e, 0x86, 0xd0, 0x2e, 0x44, 0x1c, 0x38, 0x7b, 0xce, 0xad, 0x0e, 0x35, 0xa4, 0x91, 0x44,
-	0x59, 0x1c, 0xb4, 0xac, 0x24, 0xca, 0x62, 0x42, 0xa0, 0x13, 0xa9, 0x98, 0x07, 0xed, 0x3d, 0xe7,
-	0xd6, 0x80, 0x22, 0x6d, 0x64, 0x31, 0x2b, 0x59, 0xd0, 0xb1, 0x32, 0x43, 0x87, 0x9f, 0x82, 0xf7,
-	0x48, 0x14, 0x39, 0x2b, 0xa3, 0xb9, 0xf9, 0x0a, 0x4b, 0x53, 0xfc, 0xae, 0x47, 0x0d, 0x49, 0xae,
-	0x83, 0x17, 0xcd, 0x45, 0x1a, 0x6b, 0x2e, 0x83, 0xd6, 0x5e, 0xfb, 0x96, 0x4b, 0x57, 0x7c, 0xf8,
-	0x0c, 0xfc, 0xfb, 0x49, 0xa2, 0x79, 0xc2, 0x4a, 0x4e, 0x76, 0xa0, 0xa5, 0x72, 0xf4, 0x74, 0x69,
-	0x4b, 0xe5, 0xf8, 0x94, 0x28, 0x4a, 0x8c, 0xc8, 0xa3, 0x48, 0x93, 0x5d, 0xe8, 0xf0, 0x93, 0x5c,
-	0x63, 0x48, 0xfd, 0xbb, 0x30, 0xc2, 0xe4, 0x1e, 0x9f, 0xe4, 0x9a, 0xa2, 0x3c, 0xfc, 0xc1, 0x01,
-	0xf7, 0x73, 0xad, 0x16, 0x39, 0x79, 0x0b, 0x7c, 0xc9, 0x79, 0x3c, 0xe1, 0x2f, 0x58, 0x1d, 0x8e,
-	0x67, 0x04, 0x8f, 0x5f, 0xb0, 0x94, 0x04, 0xd0, 0x13, 0xd3, 0x45, 0x74, 0xcc, 0xcb, 0x2a, 0xdf,
-	0x9a, 0x35, 0x1a, 0x59, 0x69, 0xda, 0x56, 0x53, 0xb1, 0x64, 0x0f, 0x5c, 0xf3, 0x44, 0x11, 0x74,
-	0xf6, 0xda, 0x67, 0xde, 0xb6, 0x0a, 0x63, 0x51, 0x2e, 0x73, 0x5e, 0x04, 0x6e, 0xd3, 0xe2, 0xcb,
-	0x65, 0xce, 0xa9, 0x55, 0x90, 0x9b, 0xd0, 0x61, 0x49, 0x52, 0x04, 0x5d, 0x34, 0x78, 0x63, 0xb4,
-	0x6a, 0xf2, 0xaa, 0x0a, 0x14, 0x0d, 0xc2, 0xef, 0x1d, 0xf0, 0xee, 0xcb, 0x52, 0x7c, 0xa1, 0x84,
-	0x6c, 0x46, 0xeb, 0xbc, 0x32, 0xda, 0xd6, 0x66, 0xb4, 0x6f, 0x42, 0x57, 0xf3, 0x62, 0x91, 0x9a,
-	0x34, 0x4c, 0xcd, 0x2b, 0x8e, 0xdc, 0x04, 0x3f, 0xe5, 0xb3, 0x72, 0x12, 0x29, 0x19, 0xff, 0x4d,
-	0x26, 0x9e, 0x51, 0x3e, 0x54, 0x32, 0x26, 0x1f, 0x00, 0x68, 0x91, 0xcc, 0x2b, 0x4b, 0x77, 0xcb,
-	0xd2, 0x47, 0xad, 0x31, 0x0d, 0x7f, 0x72, 0xc0, 0xdf, 0x97, 0x92, 0xeb, 0x7f, 0x1c, 0xed, 0x35,
-	0xf0, 0x34, 0x4f, 0x27, 0xa9, 0x69, 0xb7, 0x8d, 0xb7, 0xa7, 0x79, 0xfa, 0xc4, 0x74, 0xfc, 0x1a,
-	0x78, 0x91, 0xaa, 0x54, 0x1d, 0xab, 0x8a, 0x94, 0x55, 0x6d, 0xe4, 0xe2, 0x9e, 0x3b, 0x97, 0xee,
-	0xeb, 0x72, 0xf9, 0xc3, 0x01, 0xef, 0x09, 0x9f, 0x95, 0xff, 0x72, 0x2a, 0xe7, 0x19, 0x9d, 0x46,
-	0xb2, 0xdd, 0x73, 0x27, 0xdb, 0x7b, 0x5d, 0xb2, 0x66, 0xca, 0x8e, 0x78, 0xf6, 0xdf, 0x98, 0xb2,
-	0x3f, 0x1d, 0x80, 0x23, 0x21, 0x93, 0x94, 0xff, 0x2f, 0x7a, 0x73, 0x0f, 0x7a, 0x87, 0x5a, 0xc5,
-	0x8b, 0x68, 0x33, 0x6c, 0xe7, 0xd5, 0x61, 0xb7, 0x36, 0xc2, 0x0e, 0x7f, 0xe9, 0x40, 0x7f, 0x5f,
-	0x16, 0xa5, 0x5e, 0x44, 0xa5, 0x50, 0x72, 0x0b, 0x5e, 0x87, 0xd0, 0x16, 0xf1, 0x09, 0x96, 0xc8,
-	0xa5, 0x86, 0x24, 0xef, 0x43, 0x87, 0xc9, 0x52, 0x54, 0xe0, 0x4a, 0x1a, 0xe8, 0x54, 0x21, 0x11,
-	0x45, 0x3d, 0x19, 0x81, 0x17, 0x57, 0x78, 0x8f, 0xff, 0x81, 0x0d, 0xdb, 0xfa, 0x4f, 0x40, 0x57,
-	0x36, 0xe4, 0x5d, 0x68, 0xb3, 0x24, 0x09, 0x5c, 0x34, 0xbd, 0xb4, 0x36, 0x45, 0xa0, 0xa6, 0x46,
-	0x47, 0xee, 0x02, 0x08, 0x83, 0x20, 0x93, 0xe7, 0x4a, 0xc8, 0xa0, 0x8b, 0x96, 0x0d, 0x78, 0x5c,
-	0xa1, 0x0b, 0xf5, 0xc5, 0x0a, 0x68, 0xc6, 0x55, 0xd5, 0xd1, 0xa5, 0x77, 0x36, 0x8e, 0x7a, 0x89,
-	0x6d, 0xf5, 0x6b, 0x87, 0x82, 0x67, 0xc2, 0x3a, 0x78, 0x67, 0x1d, 0xea, 0x45, 0xa0, 0x5e, 0x51,
-	0xaf, 0xc4, 0x27, 0xd0, 0x2f, 0x70, 0xe2, 0xac, 0x8b, 0x8f, 0x2e, 0x57, 0x1a, 0x2e, 0xab, 0x71,
-	0xa4, 0x50, 0xac, 0x47, 0xf3, 0x43, 0xe8, 0xe5, 0xb6, 0x75, 0x01, 0xa0, 0xcb, 0xe5, 0xb5, 0x4b,
-	0xd5, 0x53, 0x5a, 0x5b, 0x90, 0x8f, 0xc0, 0x53, 0x3a, 0xe6, 0x7a, 0x32, 0x5d, 0x06, 0x7d, 0x1c,
-	0x88, 0xcb, 0x76, 0x20, 0x9e, 0x19, 0xe9, 0x83, 0xe5, 0x51, 0xce, 0x23, 0xda, 0x53, 0x96, 0x21,
-	0xb7, 0x61, 0x90, 0x6b, 0xf5, 0x9c, 0x47, 0xa5, 0xed, 0xf9, 0x60, 0x6b, 0x84, 0xfa, 0x95, 0x1e,
-	0xc7, 0x23, 0x84, 0xee, 0x4c, 0xa4, 0x25, 0xd7, 0xc1, 0xc5, 0xad, 0x1f, 0x66, 0xa5, 0x21, 0x57,
-	0xc0, 0x4d, 0x45, 0x26, 0xca, 0x60, 0x07, 0x97, 0xc5, 0x32, 0x66, 0xb3, 0xd5, 0x6c, 0x56, 0xf0,
-	0x32, 0xb8, 0x84, 0xe2, 0x8a, 0x0b, 0x97, 0xd0, 0x3d, 0x52, 0x0b, 0x1d, 0x71, 0x72, 0x03, 0xfa,
-	0x45, 0x34, 0xe7, 0x19, 0x9b, 0x48, 0x96, 0x71, 0x1c, 0x2c, 0x9f, 0x82, 0x15, 0x1d, 0xb0, 0x8c,
-	0x93, 0x77, 0x00, 0x4a, 0x36, 0x4d, 0xb9, 0xd5, 0xb7, 0x50, 0xef, 0xa3, 0x04, 0xd5, 0xcd, 0xd1,
-	0x35, 0xcb, 0xe8, 0xaf, 0x37, 0xee, 0x0a, 0xb8, 0xd3, 0x54, 0x45, 0xc7, 0x38, 0x5d, 0x3e, 0xb5,
-	0x4c, 0xf8, 0x0d, 0x78, 0x07, 0x2a, 0xe6, 0xfb, 0x72, 0xa6, 0xcc, 0x6d, 0x90, 0x45, 0xf9, 0xa2,
-	0x1a, 0x67, 0xa4, 0xcd, 0x80, 0x8b, 0xb8, 0x7a, 0xa7, 0x25, 0xf0, 0x7c, 0x61, 0x71, 0x6c, 0x6f,
-	0x05, 0x9f, 0x22, 0x6d, 0xb0, 0x21, 0x67, 0xcb, 0x54, 0x31, 0x0b, 0x4b, 0x3e, 0xad, 0xd9, 0xf0,
-	0xd7, 0x36, 0x78, 0x87, 0x55, 0x97, 0xc8, 0x23, 0xb8, 0x58, 0x77, 0x6c, 0x62, 0x56, 0x1c, 0xdf,
-	0xd9, 0xb9, 0x7b, 0xa3, 0xd1, 0xc7, 0xb3, 0x04, 0xe2, 0xc1, 0x20, 0x6f, 0x70, 0xa6, 0x42, 0xab,
-	0xaf, 0x54, 0x91, 0xb9, 0x14, 0x6a, 0xd1, 0x7e, 0x4c, 0xde, 0x86, 0xf6, 0xb7, 0x7a, 0xb9, 0x79,
-	0xcc, 0x1c, 0xa6, 0x4c, 0x52, 0x23, 0x26, 0x77, 0xa0, 0x6f, 0xce, 0xab, 0x49, 0x81, 0xf5, 0xae,
-	0x36, 0x6d, 0xd8, 0x98, 0x3e, 0x94, 0x53, 0x30, 0x46, 0x55, 0x4f, 0x46, 0x8d, 0x5b, 0xcb, 0xa2,
-	0x15, 0xd9, 0x0e, 0x79, 0x7d, 0x7f, 0x91, 0xcf, 0x60, 0x28, 0xd6, 0x10, 0x61, 0x7b, 0x61, 0xf1,
-	0xeb, 0x6a, 0x73, 0xf9, 0x56, 0x16, 0xf4, 0x52, 0xc3, 0x1c, 0x5b, 0x75, 0x15, 0xba, 0xa2, 0x98,
-	0x70, 0x44, 0x33, 0x73, 0x63, 0xb9, 0xa2, 0x78, 0x2c, 0x63, 0x03, 0x25, 0xd2, 0x9c, 0x8e, 0x5b,
-	0x5b, 0x56, 0x77, 0x90, 0xa2, 0x9e, 0xbc, 0x07, 0x3b, 0xf9, 0xa2, 0x98, 0x4f, 0x62, 0xf5, 0x52,
-	0x4e, 0x84, 0x9c, 0x29, 0x5c, 0x32, 0x97, 0x0e, 0x8c, 0xf4, 0x91, 0x7a, 0x29, 0x8d, 0x6d, 0x78,
-	0x0f, 0x06, 0xcd, 0x32, 0x13, 0x1f, 0xdc, 0xa7, 0x5c, 0x27, 0x7c, 0x78, 0x81, 0x00, 0x74, 0x0f,
-	0x94, 0xce, 0x58, 0x3a, 0x74, 0x0c, 0x4d, 0x79, 0xa6, 0x4a, 0x3e, 0x6c, 0x91, 0x01, 0x78, 0x87,
-	0x4c, 0xb3, 0x34, 0xe5, 0xe9, 0xb0, 0xfd, 0xe0, 0xe1, 0x8f, 0xa7, 0xbb, 0xce, 0xcf, 0xa7, 0xbb,
-	0xce, 0x6f, 0xa7, 0xbb, 0x17, 0xbe, 0xfb, 0x7d, 0xd7, 0xf9, 0xfa, 0x4e, 0xe3, 0x8a, 0xce, 0x58,
-	0xa9, 0xc5, 0x89, 0xd2, 0x22, 0x11, 0xb2, 0x66, 0x24, 0x1f, 0xe7, 0xc7, 0xc9, 0x38, 0x9f, 0x8e,
-	0xeb, 0xd0, 0xa7, 0x5d, 0x3c, 0xa2, 0x3f, 0xfe, 0x2b, 0x00, 0x00, 0xff, 0xff, 0x34, 0x25, 0x4d,
-	0xc1, 0x9b, 0x0b, 0x00, 0x00,
+	// 1444 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x58, 0x4f, 0x6f, 0xdb, 0x46,
+	0x16, 0x8f, 0xfe, 0x50, 0x22, 0x9f, 0x64, 0x59, 0x99, 0x4d, 0x76, 0x99, 0xec, 0xae, 0xe3, 0x30,
+	0x9b, 0xc4, 0x8b, 0xdd, 0xd8, 0x88, 0x17, 0x39, 0xef, 0x3a, 0x4e, 0xb0, 0x70, 0x11, 0x3b, 0xc6,
+	0xb8, 0xbd, 0x14, 0x05, 0x84, 0x11, 0x39, 0xa2, 0x27, 0x26, 0x67, 0x58, 0xfe, 0x49, 0xac, 0x7e,
+	0x80, 0x1e, 0xda, 0x7e, 0x82, 0xf6, 0xd2, 0x6f, 0xd2, 0x53, 0x81, 0x1e, 0x7b, 0xeb, 0xb5, 0x48,
+	0xaf, 0xfd, 0x10, 0xc5, 0xbc, 0x21, 0x29, 0xda, 0x8a, 0x53, 0xa3, 0xe8, 0xad, 0xb9, 0xcd, 0xfb,
+	0xbd, 0xdf, 0x88, 0xef, 0xff, 0xcc, 0x08, 0x46, 0x89, 0x48, 0x78, 0x24, 0x24, 0xdf, 0x4c, 0x52,
+	0x95, 0x2b, 0x62, 0x57, 0xf2, 0xcd, 0x07, 0xa1, 0xc8, 0x8f, 0x8b, 0xe9, 0xa6, 0xaf, 0xe2, 0xad,
+	0x50, 0x85, 0x6a, 0x0b, 0x09, 0xd3, 0x62, 0x86, 0x12, 0x0a, 0xb8, 0x32, 0x1b, 0x6f, 0x42, 0x12,
+	0x31, 0x69, 0xd6, 0x9e, 0x82, 0xfe, 0x3e, 0xcf, 0x32, 0x16, 0x72, 0x32, 0x86, 0x4e, 0x26, 0x02,
+	0xb7, 0xb5, 0xde, 0xda, 0xe8, 0x52, 0xbd, 0xd4, 0x88, 0x1f, 0x07, 0x6e, 0xdb, 0x20, 0x7e, 0x1c,
+	0x10, 0x02, 0x5d, 0x5f, 0x05, 0xdc, 0xed, 0xac, 0xb7, 0x36, 0x86, 0x14, 0xd7, 0x1a, 0x0b, 0x58,
+	0xce, 0xdc, 0xae, 0xc1, 0xf4, 0x9a, 0xb8, 0xd0, 0x67, 0x92, 0x45, 0xf3, 0x8c, 0xbb, 0x16, 0xc2,
+	0x95, 0xe8, 0x7d, 0x00, 0xce, 0xae, 0x92, 0x92, 0xfb, 0xb9, 0x4a, 0xc9, 0x2d, 0x18, 0x54, 0x4e,
+	0x4c, 0xca, 0x4f, 0x5b, 0x14, 0x2a, 0x68, 0x2f, 0x20, 0xf7, 0x61, 0xd5, 0xaf, 0xd8, 0x13, 0x21,
+	0x03, 0x7e, 0x8a, 0xd6, 0x58, 0x74, 0x54, 0xc3, 0x7b, 0x1a, 0xf5, 0x9e, 0x83, 0xfd, 0x44, 0x64,
+	0x09, 0xcb, 0xfd, 0x63, 0x6d, 0x36, 0x8b, 0x22, 0xfc, 0x35, 0x9b, 0xea, 0x25, 0x79, 0x08, 0x4e,
+	0xcd, 0x77, 0xdb, 0xeb, 0x9d, 0x8d, 0xc1, 0xf6, 0x9f, 0x36, 0xeb, 0x70, 0xd6, 0xf6, 0xd0, 0x05,
+	0xcb, 0x7b, 0x0e, 0xce, 0x4e, 0x18, 0xa6, 0x3c, 0x64, 0x39, 0x27, 0x23, 0x68, 0xab, 0xa4, 0x34,
+	0xaf, 0xad, 0x12, 0x74, 0x59, 0x64, 0x39, 0xda, 0x62, 0x53, 0x5c, 0x93, 0x35, 0xe8, 0xf2, 0xd3,
+	0x24, 0xc5, 0xd0, 0x0c, 0xb6, 0x61, 0x13, 0x83, 0xfc, 0xf4, 0x34, 0x49, 0x29, 0xe2, 0xde, 0xb7,
+	0x2d, 0xb0, 0xfe, 0x9f, 0xaa, 0x22, 0x21, 0x7f, 0x05, 0x47, 0x72, 0x1e, 0x4c, 0xf8, 0x4b, 0x56,
+	0x59, 0x69, 0x6b, 0xe0, 0xe9, 0x4b, 0x16, 0xe9, 0xc8, 0x89, 0x69, 0xe1, 0x9f, 0xf0, 0xbc, 0x8c,
+	0x7b, 0x25, 0x6a, 0x8d, 0x2c, 0x35, 0x1d, 0xa3, 0x29, 0x45, 0xb2, 0x0e, 0x96, 0xfe, 0x44, 0xe6,
+	0x76, 0xd1, 0xb5, 0xe6, 0xb7, 0x8d, 0x42, 0x33, 0xf2, 0x79, 0xc2, 0x33, 0xd7, 0x6a, 0x32, 0xde,
+	0x9f, 0x27, 0x9c, 0x1a, 0x05, 0xb9, 0x0f, 0x5d, 0x16, 0x86, 0x99, 0xdb, 0x3b, 0x1f, 0x9d, 0x3a,
+	0x0a, 0x14, 0x09, 0xde, 0xa7, 0x6d, 0xe8, 0xbe, 0xa7, 0x84, 0x6c, 0x5a, 0xda, 0xba, 0xd0, 0xd2,
+	0xf6, 0x59, 0x4b, 0x6f, 0x80, 0x9d, 0xf2, 0x68, 0x12, 0xe9, 0xe0, 0x75, 0xd6, 0x3b, 0x1b, 0x16,
+	0xed, 0xa7, 0x3c, 0x7a, 0xa6, 0xe3, 0x77, 0x03, 0x6c, 0x5f, 0x95, 0xaa, 0xae, 0x51, 0xf9, 0xca,
+	0xa8, 0xaa, 0xd0, 0x5a, 0x6f, 0x0e, 0xed, 0xc2, 0xbb, 0xde, 0xc5, 0xde, 0x39, 0x11, 0x9f, 0xe5,
+	0x13, 0x5f, 0xc9, 0xc0, 0xed, 0x2f, 0x45, 0xc9, 0xd6, 0xca, 0x5d, 0x25, 0x03, 0xf2, 0x4f, 0x80,
+	0x54, 0x84, 0xc7, 0x25, 0xd3, 0x5e, 0x62, 0x3a, 0xa8, 0xd5, 0x54, 0xef, 0xe7, 0x16, 0xd8, 0x3b,
+	0x32, 0x17, 0xbf, 0x39, 0x18, 0x7f, 0x86, 0x5e, 0xca, 0xb3, 0x22, 0xaa, 0x42, 0x51, 0x4a, 0xb5,
+	0xbb, 0xdd, 0x5f, 0x73, 0xd7, 0xba, 0x94, 0xbb, 0xbd, 0x4b, 0xbb, 0xdb, 0x7f, 0x9b, 0xbb, 0x9f,
+	0xb7, 0xc1, 0xd9, 0x93, 0x92, 0xa7, 0xef, 0x92, 0x2f, 0x03, 0xef, 0xb3, 0x36, 0xd8, 0xcf, 0xf8,
+	0x2c, 0x7f, 0x17, 0x8c, 0xb2, 0x13, 0x8e, 0x78, 0xfc, 0x47, 0xe9, 0x84, 0x2f, 0xda, 0x00, 0x47,
+	0x42, 0x86, 0x11, 0x7f, 0x97, 0x7d, 0x19, 0x78, 0x5f, 0x75, 0xc0, 0xde, 0x67, 0xe9, 0xc9, 0xef,
+	0x9e, 0xfd, 0x33, 0xc6, 0x76, 0x2f, 0x6d, 0xac, 0xf5, 0x16, 0x63, 0x2f, 0x11, 0xa2, 0x35, 0x7d,
+	0xc5, 0xc1, 0xe8, 0x2c, 0x05, 0x59, 0xe3, 0xe4, 0x0e, 0xf4, 0x95, 0x34, 0xe9, 0x59, 0x0e, 0x4b,
+	0x4f, 0x49, 0xcc, 0xd4, 0x2d, 0x18, 0xa8, 0x22, 0x4f, 0x8a, 0x7c, 0x22, 0x8b, 0x28, 0x72, 0x1d,
+	0x3c, 0xe4, 0xc1, 0x40, 0x07, 0x45, 0x14, 0x35, 0x08, 0x31, 0x4b, 0x4f, 0x5c, 0x68, 0x12, 0x74,
+	0x30, 0xc9, 0x1d, 0x58, 0x29, 0x09, 0x4c, 0xce, 0x5f, 0xb1, 0xb9, 0x3b, 0x40, 0xca, 0xd0, 0x80,
+	0x3b, 0x88, 0x91, 0xdb, 0x30, 0xd4, 0xdb, 0x27, 0x31, 0x67, 0x52, 0xc8, 0xd0, 0x1d, 0x22, 0x67,
+	0xa0, 0xb1, 0x7d, 0x03, 0x79, 0x0c, 0xfa, 0x87, 0xa9, 0x0a, 0x0a, 0xff, 0x6c, 0xd1, 0xb5, 0x2e,
+	0x2e, 0xba, 0xf6, 0xd9, 0xa2, 0xab, 0x23, 0xd6, 0xb9, 0x20, 0x62, 0xde, 0x0f, 0x16, 0x0c, 0xf6,
+	0x64, 0x96, 0xa7, 0x85, 0x9f, 0x0b, 0x25, 0x97, 0x6e, 0x4b, 0x63, 0xe8, 0x88, 0xa0, 0xba, 0xb8,
+	0xe9, 0x25, 0xb9, 0x07, 0x5d, 0x26, 0x73, 0x51, 0xde, 0x95, 0x48, 0xe3, 0xb2, 0x51, 0x9e, 0xa7,
+	0x14, 0xf5, 0xe4, 0x01, 0xf4, 0xcb, 0x1b, 0x59, 0x39, 0x02, 0xde, 0x78, 0x6b, 0xab, 0x38, 0x64,
+	0x13, 0xec, 0xa0, 0xbc, 0x04, 0x96, 0x3d, 0xd2, 0xf8, 0xe9, 0xea, 0x7a, 0x48, 0x6b, 0x0e, 0xb9,
+	0x0d, 0x1d, 0x16, 0x86, 0x6e, 0x0f, 0xa9, 0xab, 0x0b, 0x2a, 0x5e, 0xd3, 0xa8, 0xd6, 0x91, 0x6d,
+	0x00, 0xa1, 0x0f, 0xbd, 0xc9, 0x0b, 0x25, 0x64, 0x59, 0x13, 0x0d, 0x23, 0xea, 0x03, 0x91, 0x3a,
+	0xa2, 0x3e, 0x1b, 0xb7, 0xca, 0xba, 0xc5, 0x2d, 0xf6, 0x79, 0x3b, 0xaa, 0x53, 0xc3, 0xd4, 0x6f,
+	0xb5, 0x21, 0xe3, 0xb1, 0x30, 0x1b, 0x9c, 0xf3, 0x1b, 0xaa, 0xc9, 0x4a, 0xed, 0xac, 0x9a, 0xb1,
+	0x8f, 0x60, 0x90, 0xe1, 0x00, 0x32, 0x5b, 0x00, 0xb7, 0x5c, 0x6b, 0x6c, 0xa9, 0xa7, 0x13, 0x85,
+	0x6c, 0x31, 0xa9, 0xb6, 0xc0, 0xc1, 0x72, 0xc1, 0x4d, 0x83, 0xf3, 0xdf, 0xa9, 0x7a, 0x98, 0xda,
+	0x71, 0xd5, 0xcd, 0x1e, 0x74, 0x91, 0x3b, 0x44, 0xee, 0x68, 0xc1, 0x35, 0x39, 0xd2, 0x3a, 0xf2,
+	0x2f, 0xe8, 0x27, 0xa6, 0xc0, 0xdc, 0x15, 0xa4, 0x5d, 0x5d, 0xd0, 0xca, 0xca, 0xa3, 0x15, 0x83,
+	0xfc, 0x1b, 0x6c, 0x95, 0x06, 0x3c, 0x9d, 0x4c, 0xe7, 0xee, 0x08, 0xeb, 0xe9, 0xaa, 0xa9, 0xa7,
+	0xe7, 0x1a, 0x7d, 0x3c, 0x3f, 0x4a, 0xb8, 0x4f, 0xfb, 0xca, 0x08, 0xe4, 0x01, 0x0c, 0x93, 0x54,
+	0xbd, 0xe0, 0x7e, 0x6e, 0x2a, 0x73, 0x75, 0xa9, 0xdf, 0x06, 0xa5, 0x1e, 0x2b, 0xd5, 0x83, 0xde,
+	0x4c, 0x44, 0x39, 0x4f, 0xdd, 0xf1, 0x52, 0xef, 0x96, 0x1a, 0x72, 0x0d, 0xac, 0x48, 0xc4, 0x22,
+	0x77, 0xaf, 0xe2, 0x0c, 0x32, 0x82, 0x9e, 0x40, 0x6a, 0x36, 0xcb, 0x78, 0xee, 0x12, 0x84, 0x4b,
+	0xc9, 0x7b, 0x04, 0xc3, 0x1d, 0x7c, 0xb7, 0x88, 0x0c, 0xbf, 0x70, 0x17, 0xba, 0x75, 0xf7, 0xd4,
+	0xa6, 0x23, 0xe3, 0x13, 0xbe, 0x27, 0x67, 0x8a, 0xa2, 0xda, 0xfb, 0xa6, 0x05, 0xbd, 0x23, 0x55,
+	0xa4, 0x3e, 0xd7, 0x7d, 0x9e, 0xf9, 0xc7, 0x3c, 0x66, 0x13, 0xc9, 0x62, 0x8e, 0x4d, 0xe1, 0x50,
+	0x30, 0xd0, 0x01, 0x8b, 0x39, 0xf9, 0x3b, 0x40, 0xce, 0xa6, 0x11, 0x37, 0xfa, 0x36, 0xea, 0x1d,
+	0x44, 0x50, 0xdd, 0x6c, 0x4c, 0xdd, 0x80, 0xce, 0xa2, 0x31, 0xaf, 0x81, 0x35, 0x8d, 0x94, 0x7f,
+	0x82, 0xad, 0xe1, 0x50, 0x23, 0xe0, 0x93, 0xaa, 0xc8, 0x8e, 0x03, 0xf5, 0x4a, 0xea, 0x27, 0x95,
+	0x85, 0xfe, 0x40, 0x05, 0xed, 0xe9, 0xf9, 0xb5, 0x52, 0x13, 0x58, 0x10, 0xa4, 0x58, 0xfe, 0x0e,
+	0x1d, 0x56, 0xe0, 0x4e, 0x10, 0xa4, 0xde, 0x47, 0x60, 0x1f, 0xa8, 0x00, 0x7d, 0xd2, 0x8f, 0x9d,
+	0xd8, 0x4f, 0x8a, 0xb2, 0xa1, 0x71, 0xad, 0x5b, 0x5c, 0x04, 0xa5, 0xb5, 0x6d, 0x81, 0xef, 0x42,
+	0xfc, 0xad, 0x0e, 0x22, 0xb8, 0xd6, 0x03, 0x3f, 0x61, 0xf3, 0x48, 0x31, 0x33, 0xbc, 0x1d, 0x5a,
+	0x89, 0xde, 0x97, 0x5d, 0xb0, 0x0f, 0xcb, 0x1a, 0x21, 0x4f, 0x60, 0xa5, 0x7e, 0x03, 0xea, 0x79,
+	0x82, 0xdf, 0x19, 0x6d, 0xdf, 0x6a, 0x54, 0xd1, 0xf9, 0x05, 0x0e, 0x9f, 0x61, 0xd2, 0x90, 0xce,
+	0xbf, 0x24, 0xdb, 0x4b, 0x2f, 0xc9, 0xbf, 0x41, 0xe7, 0xe3, 0x74, 0x7e, 0xf6, 0x75, 0x76, 0x18,
+	0x31, 0x49, 0x35, 0x4c, 0x1e, 0xc2, 0x40, 0xbf, 0x5b, 0x27, 0x19, 0x66, 0xad, 0x1c, 0x36, 0xe3,
+	0x46, 0x43, 0x21, 0x4e, 0x41, 0x93, 0xca, 0xcc, 0x6e, 0x82, 0xed, 0x1f, 0x8b, 0x28, 0x48, 0xb9,
+	0x2c, 0x8f, 0x1c, 0xb2, 0x6c, 0x32, 0xad, 0x39, 0xe4, 0x7f, 0x30, 0x16, 0x8b, 0x21, 0x69, 0x32,
+	0x6a, 0x0e, 0xa1, 0xeb, 0xcd, 0x79, 0x52, 0x33, 0xe8, 0x6a, 0x83, 0x8e, 0x09, 0xbf, 0x0e, 0x3d,
+	0x91, 0x4d, 0x78, 0x79, 0x36, 0xd9, 0xd4, 0x12, 0xd9, 0x53, 0x19, 0x90, 0xbf, 0x40, 0x5f, 0x64,
+	0x8b, 0x61, 0x63, 0xd3, 0x9e, 0xc8, 0xb0, 0x7b, 0xef, 0x41, 0x57, 0xea, 0xc7, 0xfa, 0xd2, 0x44,
+	0xa9, 0x52, 0x4b, 0x51, 0x4f, 0xfe, 0x01, 0x23, 0x9d, 0xfc, 0x89, 0xa9, 0x19, 0x39, 0x53, 0x38,
+	0x50, 0x2c, 0x53, 0x12, 0x4f, 0x74, 0xd5, 0xe8, 0x32, 0xb8, 0x0b, 0xa3, 0xca, 0x97, 0x89, 0xaf,
+	0x0a, 0x99, 0xe3, 0x04, 0xb1, 0xe8, 0x4a, 0x85, 0xee, 0x6a, 0xd0, 0xfb, 0x2f, 0x0c, 0x9b, 0x69,
+	0x22, 0x0e, 0x58, 0xfb, 0x3c, 0x0d, 0xf9, 0xf8, 0x0a, 0x01, 0xe8, 0x1d, 0xa8, 0x34, 0x66, 0xd1,
+	0xb8, 0xa5, 0xd7, 0x94, 0xc7, 0x2a, 0xe7, 0xe3, 0x36, 0x19, 0x82, 0x7d, 0xc8, 0x52, 0x16, 0x45,
+	0x3c, 0x1a, 0x77, 0x1e, 0xef, 0x7e, 0xf7, 0x7a, 0xad, 0xf5, 0xfd, 0xeb, 0xb5, 0xd6, 0x8f, 0xaf,
+	0xd7, 0xae, 0x7c, 0xfd, 0xd3, 0x5a, 0xeb, 0xc3, 0x87, 0x8d, 0xbf, 0x37, 0x62, 0x96, 0xa7, 0xe2,
+	0x54, 0xa5, 0x22, 0x14, 0xb2, 0x12, 0x24, 0xdf, 0x4a, 0x4e, 0xc2, 0xad, 0x64, 0xba, 0x55, 0x79,
+	0x38, 0xed, 0xe1, 0xbf, 0x1b, 0xff, 0xf9, 0x25, 0x00, 0x00, 0xff, 0xff, 0x49, 0x80, 0xd5, 0x98,
+	0x34, 0x11, 0x00, 0x00,
 }
 
 func (m *Message) Marshal() (dAtA []byte, err error) {
@@ -1355,6 +1855,13 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
+	if len(m.Analyse) > 0 {
+		i -= len(m.Analyse)
+		copy(dAtA[i:], m.Analyse)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Analyse)))
+		i--
+		dAtA[i] = 0x2a
+	}
 	if len(m.Data) > 0 {
 		i -= len(m.Data)
 		copy(dAtA[i:], m.Data)
@@ -1382,6 +1889,43 @@ func (m *Message) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
+func (m *Connector) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *Connector) MarshalTo(dAtA []byte) (int, error) {
+	size := m.ProtoSize()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Connector) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
+	var l int
+	_ = l
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
+	}
+	if m.ConnectorIndex != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.ConnectorIndex))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.PipelineId != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.PipelineId))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
 func (m *Dispatch) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
@@ -1406,24 +1950,19 @@ func (m *Dispatch) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.Children) > 0 {
-		dAtA2 := make([]byte, len(m.Children)*10)
-		var j1 int
-		for _, num1 := range m.Children {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA2[j1] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j1++
+	if len(m.Connector) > 0 {
+		for iNdEx := len(m.Connector) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Connector[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			dAtA2[j1] = uint8(num)
-			j1++
+			i--
+			dAtA[i] = 0x12
 		}
-		i -= j1
-		copy(dAtA[i:], dAtA2[:j1])
-		i = encodeVarintPipeline(dAtA, i, uint64(j1))
-		i--
-		dAtA[i] = 0x12
 	}
 	if m.All {
 		i--
@@ -1581,7 +2120,7 @@ func (m *Group) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *AntiJoin) Marshal() (dAtA []byte, err error) {
+func (m *Join) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -1591,12 +2130,12 @@ func (m *AntiJoin) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *AntiJoin) MarshalTo(dAtA []byte) (int, error) {
+func (m *Join) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *AntiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *Join) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -1616,7 +2155,7 @@ func (m *AntiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x2a
+			dAtA[i] = 0x42
 		}
 	}
 	if len(m.LeftCond) > 0 {
@@ -1630,30 +2169,75 @@ func (m *AntiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x22
+			dAtA[i] = 0x3a
 		}
 	}
-	if len(m.Result) > 0 {
-		dAtA5 := make([]byte, len(m.Result)*10)
-		var j4 int
-		for _, num1 := range m.Result {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA5[j4] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j4++
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			dAtA5[j4] = uint8(num)
-			j4++
+			i--
+			dAtA[i] = 0x32
 		}
-		i -= j4
-		copy(dAtA[i:], dAtA5[:j4])
-		i = encodeVarintPipeline(dAtA, i, uint64(j4))
-		i--
-		dAtA[i] = 0x1a
 	}
-	if m.Nbucket != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Nbucket))
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x2a
+	}
+	if len(m.ColList) > 0 {
+		dAtA4 := make([]byte, len(m.ColList)*10)
+		var j3 int
+		for _, num1 := range m.ColList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA4[j3] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j3++
+			}
+			dAtA4[j3] = uint8(num)
+			j3++
+		}
+		i -= j3
+		copy(dAtA[i:], dAtA4[:j3])
+		i = encodeVarintPipeline(dAtA, i, uint64(j3))
+		i--
+		dAtA[i] = 0x22
+	}
+	if len(m.RelList) > 0 {
+		dAtA6 := make([]byte, len(m.RelList)*10)
+		var j5 int
+		for _, num1 := range m.RelList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA6[j5] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j5++
+			}
+			dAtA6[j5] = uint8(num)
+			j5++
+		}
+		i -= j5
+		copy(dAtA[i:], dAtA6[:j5])
+		i = encodeVarintPipeline(dAtA, i, uint64(j5))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.Nbucket != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Nbucket))
 		i--
 		dAtA[i] = 0x10
 	}
@@ -1665,7 +2249,7 @@ func (m *AntiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *InnerJoin) Marshal() (dAtA []byte, err error) {
+func (m *AntiJoin) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -1675,12 +2259,12 @@ func (m *InnerJoin) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *InnerJoin) MarshalTo(dAtA []byte) (int, error) {
+func (m *AntiJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *InnerJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *AntiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -1700,7 +2284,7 @@ func (m *InnerJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x32
+			dAtA[i] = 0x3a
 		}
 	}
 	if len(m.LeftCond) > 0 {
@@ -1714,32 +2298,39 @@ func (m *InnerJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
+			dAtA[i] = 0x32
+		}
+	}
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
 			dAtA[i] = 0x2a
 		}
 	}
-	if len(m.ColList) > 0 {
-		dAtA7 := make([]byte, len(m.ColList)*10)
-		var j6 int
-		for _, num1 := range m.ColList {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA7[j6] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j6++
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
 			}
-			dAtA7[j6] = uint8(num)
-			j6++
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
 		}
-		i -= j6
-		copy(dAtA[i:], dAtA7[:j6])
-		i = encodeVarintPipeline(dAtA, i, uint64(j6))
 		i--
 		dAtA[i] = 0x22
 	}
-	if len(m.RelList) > 0 {
-		dAtA9 := make([]byte, len(m.RelList)*10)
+	if len(m.Result) > 0 {
+		dAtA9 := make([]byte, len(m.Result)*10)
 		var j8 int
-		for _, num1 := range m.RelList {
+		for _, num1 := range m.Result {
 			num := uint64(num1)
 			for num >= 1<<7 {
 				dAtA9[j8] = uint8(uint64(num)&0x7f | 0x80)
@@ -1768,7 +2359,7 @@ func (m *InnerJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *LeftJoin) Marshal() (dAtA []byte, err error) {
+func (m *InnerJoin) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -1778,12 +2369,12 @@ func (m *LeftJoin) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *LeftJoin) MarshalTo(dAtA []byte) (int, error) {
+func (m *InnerJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *InnerJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -1803,7 +2394,7 @@ func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x3a
+			dAtA[i] = 0x42
 		}
 	}
 	if len(m.LeftCond) > 0 {
@@ -1817,7 +2408,7 @@ func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x32
+			dAtA[i] = 0x3a
 		}
 	}
 	if len(m.Types) > 0 {
@@ -1831,44 +2422,56 @@ func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x2a
+			dAtA[i] = 0x32
+		}
+	}
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
 		}
+		i--
+		dAtA[i] = 0x2a
 	}
 	if len(m.ColList) > 0 {
-		dAtA11 := make([]byte, len(m.ColList)*10)
-		var j10 int
+		dAtA12 := make([]byte, len(m.ColList)*10)
+		var j11 int
 		for _, num1 := range m.ColList {
 			num := uint64(num1)
 			for num >= 1<<7 {
-				dAtA11[j10] = uint8(uint64(num)&0x7f | 0x80)
+				dAtA12[j11] = uint8(uint64(num)&0x7f | 0x80)
 				num >>= 7
-				j10++
+				j11++
 			}
-			dAtA11[j10] = uint8(num)
-			j10++
+			dAtA12[j11] = uint8(num)
+			j11++
 		}
-		i -= j10
-		copy(dAtA[i:], dAtA11[:j10])
-		i = encodeVarintPipeline(dAtA, i, uint64(j10))
+		i -= j11
+		copy(dAtA[i:], dAtA12[:j11])
+		i = encodeVarintPipeline(dAtA, i, uint64(j11))
 		i--
 		dAtA[i] = 0x22
 	}
 	if len(m.RelList) > 0 {
-		dAtA13 := make([]byte, len(m.RelList)*10)
-		var j12 int
+		dAtA14 := make([]byte, len(m.RelList)*10)
+		var j13 int
 		for _, num1 := range m.RelList {
 			num := uint64(num1)
 			for num >= 1<<7 {
-				dAtA13[j12] = uint8(uint64(num)&0x7f | 0x80)
+				dAtA14[j13] = uint8(uint64(num)&0x7f | 0x80)
 				num >>= 7
-				j12++
+				j13++
 			}
-			dAtA13[j12] = uint8(num)
-			j12++
+			dAtA14[j13] = uint8(num)
+			j13++
 		}
-		i -= j12
-		copy(dAtA[i:], dAtA13[:j12])
-		i = encodeVarintPipeline(dAtA, i, uint64(j12))
+		i -= j13
+		copy(dAtA[i:], dAtA14[:j13])
+		i = encodeVarintPipeline(dAtA, i, uint64(j13))
 		i--
 		dAtA[i] = 0x1a
 	}
@@ -1885,7 +2488,7 @@ func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *SemiJoin) Marshal() (dAtA []byte, err error) {
+func (m *LeftJoin) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -1895,12 +2498,12 @@ func (m *SemiJoin) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *SemiJoin) MarshalTo(dAtA []byte) (int, error) {
+func (m *LeftJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *SemiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *LeftJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -1920,7 +2523,7 @@ func (m *SemiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x2a
+			dAtA[i] = 0x42
 		}
 	}
 	if len(m.LeftCond) > 0 {
@@ -1934,25 +2537,70 @@ func (m *SemiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x22
+			dAtA[i] = 0x3a
 		}
 	}
-	if len(m.Result) > 0 {
-		dAtA15 := make([]byte, len(m.Result)*10)
-		var j14 int
-		for _, num1 := range m.Result {
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x32
+		}
+	}
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x2a
+	}
+	if len(m.ColList) > 0 {
+		dAtA17 := make([]byte, len(m.ColList)*10)
+		var j16 int
+		for _, num1 := range m.ColList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j16++
+			}
+			dAtA17[j16] = uint8(num)
+			j16++
+		}
+		i -= j16
+		copy(dAtA[i:], dAtA17[:j16])
+		i = encodeVarintPipeline(dAtA, i, uint64(j16))
+		i--
+		dAtA[i] = 0x22
+	}
+	if len(m.RelList) > 0 {
+		dAtA19 := make([]byte, len(m.RelList)*10)
+		var j18 int
+		for _, num1 := range m.RelList {
 			num := uint64(num1)
 			for num >= 1<<7 {
-				dAtA15[j14] = uint8(uint64(num)&0x7f | 0x80)
+				dAtA19[j18] = uint8(uint64(num)&0x7f | 0x80)
 				num >>= 7
-				j14++
+				j18++
 			}
-			dAtA15[j14] = uint8(num)
-			j14++
+			dAtA19[j18] = uint8(num)
+			j18++
 		}
-		i -= j14
-		copy(dAtA[i:], dAtA15[:j14])
-		i = encodeVarintPipeline(dAtA, i, uint64(j14))
+		i -= j18
+		copy(dAtA[i:], dAtA19[:j18])
+		i = encodeVarintPipeline(dAtA, i, uint64(j18))
 		i--
 		dAtA[i] = 0x1a
 	}
@@ -1969,7 +2617,7 @@ func (m *SemiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *SingleJoin) Marshal() (dAtA []byte, err error) {
+func (m *SemiJoin) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -1979,12 +2627,12 @@ func (m *SingleJoin) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *SingleJoin) MarshalTo(dAtA []byte) (int, error) {
+func (m *SemiJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *SingleJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *SemiJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -2035,41 +2683,34 @@ func (m *SingleJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 			dAtA[i] = 0x2a
 		}
 	}
-	if len(m.ColList) > 0 {
-		dAtA17 := make([]byte, len(m.ColList)*10)
-		var j16 int
-		for _, num1 := range m.ColList {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA17[j16] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j16++
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
 			}
-			dAtA17[j16] = uint8(num)
-			j16++
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
 		}
-		i -= j16
-		copy(dAtA[i:], dAtA17[:j16])
-		i = encodeVarintPipeline(dAtA, i, uint64(j16))
 		i--
 		dAtA[i] = 0x22
 	}
-	if len(m.RelList) > 0 {
-		dAtA19 := make([]byte, len(m.RelList)*10)
-		var j18 int
-		for _, num1 := range m.RelList {
+	if len(m.Result) > 0 {
+		dAtA22 := make([]byte, len(m.Result)*10)
+		var j21 int
+		for _, num1 := range m.Result {
 			num := uint64(num1)
 			for num >= 1<<7 {
-				dAtA19[j18] = uint8(uint64(num)&0x7f | 0x80)
+				dAtA22[j21] = uint8(uint64(num)&0x7f | 0x80)
 				num >>= 7
-				j18++
+				j21++
 			}
-			dAtA19[j18] = uint8(num)
-			j18++
+			dAtA22[j21] = uint8(num)
+			j21++
 		}
-		i -= j18
-		copy(dAtA[i:], dAtA19[:j18])
-		i = encodeVarintPipeline(dAtA, i, uint64(j18))
+		i -= j21
+		copy(dAtA[i:], dAtA22[:j21])
+		i = encodeVarintPipeline(dAtA, i, uint64(j21))
 		i--
 		dAtA[i] = 0x1a
 	}
@@ -2086,7 +2727,7 @@ func (m *SingleJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	return len(dAtA) - i, nil
 }
 
-func (m *Product) Marshal() (dAtA []byte, err error) {
+func (m *SingleJoin) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -2096,12 +2737,12 @@ func (m *Product) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *Product) MarshalTo(dAtA []byte) (int, error) {
+func (m *SingleJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *SingleJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -2110,63 +2751,127 @@ func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.ColList) > 0 {
-		dAtA21 := make([]byte, len(m.ColList)*10)
-		var j20 int
-		for _, num1 := range m.ColList {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA21[j20] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j20++
+	if len(m.RightCond) > 0 {
+		for iNdEx := len(m.RightCond) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.RightCond[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			dAtA21[j20] = uint8(num)
-			j20++
+			i--
+			dAtA[i] = 0x42
 		}
-		i -= j20
-		copy(dAtA[i:], dAtA21[:j20])
-		i = encodeVarintPipeline(dAtA, i, uint64(j20))
-		i--
-		dAtA[i] = 0x12
 	}
-	if len(m.RelList) > 0 {
-		dAtA23 := make([]byte, len(m.RelList)*10)
-		var j22 int
-		for _, num1 := range m.RelList {
-			num := uint64(num1)
-			for num >= 1<<7 {
-				dAtA23[j22] = uint8(uint64(num)&0x7f | 0x80)
-				num >>= 7
-				j22++
+	if len(m.LeftCond) > 0 {
+		for iNdEx := len(m.LeftCond) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.LeftCond[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			dAtA23[j22] = uint8(num)
-			j22++
+			i--
+			dAtA[i] = 0x3a
 		}
-		i -= j22
-		copy(dAtA[i:], dAtA23[:j22])
-		i = encodeVarintPipeline(dAtA, i, uint64(j22))
-		i--
-		dAtA[i] = 0xa
-	}
-	return len(dAtA) - i, nil
-}
-
-func (m *Instruction) Marshal() (dAtA []byte, err error) {
-	size := m.ProtoSize()
-	dAtA = make([]byte, size)
-	n, err := m.MarshalToSizedBuffer(dAtA[:size])
-	if err != nil {
-		return nil, err
 	}
-	return dAtA[:n], nil
-}
-
-func (m *Instruction) MarshalTo(dAtA []byte) (int, error) {
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x32
+		}
+	}
+	if m.Expr != nil {
+		{
+			size, err := m.Expr.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x2a
+	}
+	if len(m.ColList) > 0 {
+		dAtA25 := make([]byte, len(m.ColList)*10)
+		var j24 int
+		for _, num1 := range m.ColList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA25[j24] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j24++
+			}
+			dAtA25[j24] = uint8(num)
+			j24++
+		}
+		i -= j24
+		copy(dAtA[i:], dAtA25[:j24])
+		i = encodeVarintPipeline(dAtA, i, uint64(j24))
+		i--
+		dAtA[i] = 0x22
+	}
+	if len(m.RelList) > 0 {
+		dAtA27 := make([]byte, len(m.RelList)*10)
+		var j26 int
+		for _, num1 := range m.RelList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA27[j26] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j26++
+			}
+			dAtA27[j26] = uint8(num)
+			j26++
+		}
+		i -= j26
+		copy(dAtA[i:], dAtA27[:j26])
+		i = encodeVarintPipeline(dAtA, i, uint64(j26))
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.Nbucket != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Nbucket))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.Ibucket != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Ibucket))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func (m *MarkJoin) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
+	}
+	return dAtA[:n], nil
+}
+
+func (m *MarkJoin) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *MarkJoin) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -2175,46 +2880,50 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if m.Offset != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Offset))
+	if m.MarkMeaning {
 		i--
-		dAtA[i] = 0x78
+		if m.MarkMeaning {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x60
 	}
-	if m.Limit != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Limit))
+	if m.OutputAnyway {
+		i--
+		if m.OutputAnyway {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
 		i--
-		dAtA[i] = 0x70
+		dAtA[i] = 0x58
 	}
-	if m.Filter != nil {
-		{
-			size, err := m.Filter.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
-			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+	if m.OutputMark {
+		i--
+		if m.OutputMark {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
 		}
 		i--
-		dAtA[i] = 0x6a
+		dAtA[i] = 0x50
 	}
-	if len(m.ProjectList) > 0 {
-		for iNdEx := len(m.ProjectList) - 1; iNdEx >= 0; iNdEx-- {
-			{
-				size, err := m.ProjectList[iNdEx].MarshalToSizedBuffer(dAtA[:i])
-				if err != nil {
-					return 0, err
-				}
-				i -= size
-				i = encodeVarintPipeline(dAtA, i, uint64(size))
-			}
-			i--
-			dAtA[i] = 0x62
+	if m.OutputNull {
+		i--
+		if m.OutputNull {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
 		}
+		i--
+		dAtA[i] = 0x48
 	}
-	if len(m.OrderBy) > 0 {
-		for iNdEx := len(m.OrderBy) - 1; iNdEx >= 0; iNdEx-- {
+	if len(m.OnList) > 0 {
+		for iNdEx := len(m.OnList) - 1; iNdEx >= 0; iNdEx-- {
 			{
-				size, err := m.OrderBy[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				size, err := m.OnList[iNdEx].MarshalToSizedBuffer(dAtA[:i])
 				if err != nil {
 					return 0, err
 				}
@@ -2222,48 +2931,12 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x5a
-		}
-	}
-	if m.Product != nil {
-		{
-			size, err := m.Product.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
-			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
-		}
-		i--
-		dAtA[i] = 0x52
-	}
-	if m.SingleJoin != nil {
-		{
-			size, err := m.SingleJoin.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
-			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
-		}
-		i--
-		dAtA[i] = 0x4a
-	}
-	if m.SemiJoin != nil {
-		{
-			size, err := m.SemiJoin.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
-			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+			dAtA[i] = 0x42
 		}
-		i--
-		dAtA[i] = 0x42
 	}
-	if m.LeftJoin != nil {
+	if m.Cond != nil {
 		{
-			size, err := m.LeftJoin.MarshalToSizedBuffer(dAtA[:i])
+			size, err := m.Cond.MarshalToSizedBuffer(dAtA[:i])
 			if err != nil {
 				return 0, err
 			}
@@ -2273,68 +2946,81 @@ func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i--
 		dAtA[i] = 0x3a
 	}
-	if m.InnerJoin != nil {
-		{
-			size, err := m.InnerJoin.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+			i--
+			dAtA[i] = 0x32
 		}
-		i--
-		dAtA[i] = 0x32
 	}
-	if m.Agg != nil {
-		{
-			size, err := m.Agg.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
+	if len(m.RightCond) > 0 {
+		for iNdEx := len(m.RightCond) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.RightCond[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+			i--
+			dAtA[i] = 0x2a
 		}
-		i--
-		dAtA[i] = 0x2a
 	}
-	if m.Dispatch != nil {
-		{
-			size, err := m.Dispatch.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
+	if len(m.LeftCond) > 0 {
+		for iNdEx := len(m.LeftCond) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.LeftCond[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+			i--
+			dAtA[i] = 0x22
 		}
-		i--
-		dAtA[i] = 0x22
 	}
-	if m.Anti != nil {
-		{
-			size, err := m.Anti.MarshalToSizedBuffer(dAtA[:i])
-			if err != nil {
-				return 0, err
+	if len(m.Result) > 0 {
+		dAtA30 := make([]byte, len(m.Result)*10)
+		var j29 int
+		for _, num1 := range m.Result {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA30[j29] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j29++
 			}
-			i -= size
-			i = encodeVarintPipeline(dAtA, i, uint64(size))
+			dAtA30[j29] = uint8(num)
+			j29++
 		}
+		i -= j29
+		copy(dAtA[i:], dAtA30[:j29])
+		i = encodeVarintPipeline(dAtA, i, uint64(j29))
 		i--
 		dAtA[i] = 0x1a
 	}
-	if m.Idx != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Idx))
+	if m.Nbucket != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Nbucket))
 		i--
 		dAtA[i] = 0x10
 	}
-	if m.Op != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Op))
+	if m.Ibucket != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Ibucket))
 		i--
 		dAtA[i] = 0x8
 	}
 	return len(dAtA) - i, nil
 }
 
-func (m *Source) Marshal() (dAtA []byte, err error) {
+func (m *Product) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -2344,12 +3030,12 @@ func (m *Source) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *Source) MarshalTo(dAtA []byte) (int, error) {
+func (m *Product) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *Source) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *Product) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -2358,40 +3044,62 @@ func (m *Source) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.Block) > 0 {
-		i -= len(m.Block)
-		copy(dAtA[i:], m.Block)
-		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Block)))
-		i--
-		dAtA[i] = 0x22
-	}
-	if len(m.ColList) > 0 {
-		for iNdEx := len(m.ColList) - 1; iNdEx >= 0; iNdEx-- {
-			i -= len(m.ColList[iNdEx])
-			copy(dAtA[i:], m.ColList[iNdEx])
-			i = encodeVarintPipeline(dAtA, i, uint64(len(m.ColList[iNdEx])))
-			i--
-			dAtA[i] = 0x1a
-		}
+	if len(m.Types) > 0 {
+		for iNdEx := len(m.Types) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Types[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x1a
+		}
 	}
-	if len(m.TableName) > 0 {
-		i -= len(m.TableName)
-		copy(dAtA[i:], m.TableName)
-		i = encodeVarintPipeline(dAtA, i, uint64(len(m.TableName)))
+	if len(m.ColList) > 0 {
+		dAtA32 := make([]byte, len(m.ColList)*10)
+		var j31 int
+		for _, num1 := range m.ColList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA32[j31] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j31++
+			}
+			dAtA32[j31] = uint8(num)
+			j31++
+		}
+		i -= j31
+		copy(dAtA[i:], dAtA32[:j31])
+		i = encodeVarintPipeline(dAtA, i, uint64(j31))
 		i--
 		dAtA[i] = 0x12
 	}
-	if len(m.SchemaName) > 0 {
-		i -= len(m.SchemaName)
-		copy(dAtA[i:], m.SchemaName)
-		i = encodeVarintPipeline(dAtA, i, uint64(len(m.SchemaName)))
+	if len(m.RelList) > 0 {
+		dAtA34 := make([]byte, len(m.RelList)*10)
+		var j33 int
+		for _, num1 := range m.RelList {
+			num := uint64(num1)
+			for num >= 1<<7 {
+				dAtA34[j33] = uint8(uint64(num)&0x7f | 0x80)
+				num >>= 7
+				j33++
+			}
+			dAtA34[j33] = uint8(num)
+			j33++
+		}
+		i -= j33
+		copy(dAtA[i:], dAtA34[:j33])
+		i = encodeVarintPipeline(dAtA, i, uint64(j33))
 		i--
 		dAtA[i] = 0xa
 	}
 	return len(dAtA) - i, nil
 }
 
-func (m *NodeInfo) Marshal() (dAtA []byte, err error) {
+func (m *Instruction) Marshal() (dAtA []byte, err error) {
 	size := m.ProtoSize()
 	dAtA = make([]byte, size)
 	n, err := m.MarshalToSizedBuffer(dAtA[:size])
@@ -2401,12 +3109,12 @@ func (m *NodeInfo) Marshal() (dAtA []byte, err error) {
 	return dAtA[:n], nil
 }
 
-func (m *NodeInfo) MarshalTo(dAtA []byte) (int, error) {
+func (m *Instruction) MarshalTo(dAtA []byte) (int, error) {
 	size := m.ProtoSize()
 	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *NodeInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+func (m *Instruction) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 	i := len(dAtA)
 	_ = i
 	var l int
@@ -2415,69 +3123,23 @@ func (m *NodeInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i -= len(m.XXX_unrecognized)
 		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.Payload) > 0 {
-		for iNdEx := len(m.Payload) - 1; iNdEx >= 0; iNdEx-- {
-			i -= len(m.Payload[iNdEx])
-			copy(dAtA[i:], m.Payload[iNdEx])
-			i = encodeVarintPipeline(dAtA, i, uint64(len(m.Payload[iNdEx])))
-			i--
-			dAtA[i] = 0x22
-		}
-	}
-	if len(m.Addr) > 0 {
-		i -= len(m.Addr)
-		copy(dAtA[i:], m.Addr)
-		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Addr)))
+	if m.Offset != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Offset))
 		i--
-		dAtA[i] = 0x1a
-	}
-	if len(m.Id) > 0 {
-		i -= len(m.Id)
-		copy(dAtA[i:], m.Id)
-		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Id)))
+		dAtA[i] = 0x1
 		i--
-		dAtA[i] = 0x12
+		dAtA[i] = 0x90
 	}
-	if m.Mcpu != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.Mcpu))
+	if m.Limit != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Limit))
 		i--
-		dAtA[i] = 0x8
-	}
-	return len(dAtA) - i, nil
-}
-
-func (m *Pipeline) Marshal() (dAtA []byte, err error) {
-	size := m.ProtoSize()
-	dAtA = make([]byte, size)
-	n, err := m.MarshalToSizedBuffer(dAtA[:size])
-	if err != nil {
-		return nil, err
-	}
-	return dAtA[:n], nil
-}
-
-func (m *Pipeline) MarshalTo(dAtA []byte) (int, error) {
-	size := m.ProtoSize()
-	return m.MarshalToSizedBuffer(dAtA[:size])
-}
-
-func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
-	i := len(dAtA)
-	_ = i
-	var l int
-	_ = l
-	if m.XXX_unrecognized != nil {
-		i -= len(m.XXX_unrecognized)
-		copy(dAtA[i:], m.XXX_unrecognized)
-	}
-	if m.PushDownInfo != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.PushDownInfo))
+		dAtA[i] = 0x1
 		i--
-		dAtA[i] = 0x48
+		dAtA[i] = 0x88
 	}
-	if m.Node != nil {
+	if m.Filter != nil {
 		{
-			size, err := m.Node.MarshalToSizedBuffer(dAtA[:i])
+			size, err := m.Filter.MarshalToSizedBuffer(dAtA[:i])
 			if err != nil {
 				return 0, err
 			}
@@ -2485,22 +3147,14 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 			i = encodeVarintPipeline(dAtA, i, uint64(size))
 		}
 		i--
-		dAtA[i] = 0x42
-	}
-	if m.IsEnd {
-		i--
-		if m.IsEnd {
-			dAtA[i] = 1
-		} else {
-			dAtA[i] = 0
-		}
+		dAtA[i] = 0x1
 		i--
-		dAtA[i] = 0x38
+		dAtA[i] = 0x82
 	}
-	if len(m.InstructionList) > 0 {
-		for iNdEx := len(m.InstructionList) - 1; iNdEx >= 0; iNdEx-- {
+	if len(m.ProjectList) > 0 {
+		for iNdEx := len(m.ProjectList) - 1; iNdEx >= 0; iNdEx-- {
 			{
-				size, err := m.InstructionList[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				size, err := m.ProjectList[iNdEx].MarshalToSizedBuffer(dAtA[:i])
 				if err != nil {
 					return 0, err
 				}
@@ -2508,13 +3162,13 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x32
+			dAtA[i] = 0x7a
 		}
 	}
-	if len(m.Children) > 0 {
-		for iNdEx := len(m.Children) - 1; iNdEx >= 0; iNdEx-- {
+	if len(m.OrderBy) > 0 {
+		for iNdEx := len(m.OrderBy) - 1; iNdEx >= 0; iNdEx-- {
 			{
-				size, err := m.Children[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				size, err := m.OrderBy[iNdEx].MarshalToSizedBuffer(dAtA[:i])
 				if err != nil {
 					return 0, err
 				}
@@ -2522,12 +3176,120 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 				i = encodeVarintPipeline(dAtA, i, uint64(size))
 			}
 			i--
-			dAtA[i] = 0x2a
+			dAtA[i] = 0x72
 		}
 	}
-	if m.DataSource != nil {
+	if m.Product != nil {
 		{
-			size, err := m.DataSource.MarshalToSizedBuffer(dAtA[:i])
+			size, err := m.Product.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x6a
+	}
+	if m.Join != nil {
+		{
+			size, err := m.Join.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x62
+	}
+	if m.MarkJoin != nil {
+		{
+			size, err := m.MarkJoin.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x5a
+	}
+	if m.SingleJoin != nil {
+		{
+			size, err := m.SingleJoin.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x52
+	}
+	if m.SemiJoin != nil {
+		{
+			size, err := m.SemiJoin.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x4a
+	}
+	if m.LeftJoin != nil {
+		{
+			size, err := m.LeftJoin.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x42
+	}
+	if m.InnerJoin != nil {
+		{
+			size, err := m.InnerJoin.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x3a
+	}
+	if m.Agg != nil {
+		{
+			size, err := m.Agg.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x32
+	}
+	if m.Dispatch != nil {
+		{
+			size, err := m.Dispatch.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x2a
+	}
+	if m.Connect != nil {
+		{
+			size, err := m.Connect.MarshalToSizedBuffer(dAtA[:i])
 			if err != nil {
 				return 0, err
 			}
@@ -2537,9 +3299,9 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i--
 		dAtA[i] = 0x22
 	}
-	if m.Qry != nil {
+	if m.Anti != nil {
 		{
-			size, err := m.Qry.MarshalToSizedBuffer(dAtA[:i])
+			size, err := m.Anti.MarshalToSizedBuffer(dAtA[:i])
 			if err != nil {
 				return 0, err
 			}
@@ -2549,210 +3311,446 @@ func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
 		i--
 		dAtA[i] = 0x1a
 	}
-	if m.PipelineId != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.PipelineId))
+	if m.Idx != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Idx))
 		i--
 		dAtA[i] = 0x10
 	}
-	if m.PipelineType != 0 {
-		i = encodeVarintPipeline(dAtA, i, uint64(m.PipelineType))
+	if m.Op != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Op))
 		i--
 		dAtA[i] = 0x8
 	}
 	return len(dAtA) - i, nil
 }
 
-func encodeVarintPipeline(dAtA []byte, offset int, v uint64) int {
-	offset -= sovPipeline(v)
-	base := offset
-	for v >= 1<<7 {
-		dAtA[offset] = uint8(v&0x7f | 0x80)
-		v >>= 7
-		offset++
+func (m *AnalysisList) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
 	}
-	dAtA[offset] = uint8(v)
-	return base
+	return dAtA[:n], nil
 }
-func (m *Message) ProtoSize() (n int) {
-	if m == nil {
-		return 0
-	}
-	var l int
-	_ = l
-	if m.Sid != 0 {
-		n += 1 + sovPipeline(uint64(m.Sid))
-	}
-	if m.Cmd != 0 {
-		n += 1 + sovPipeline(uint64(m.Cmd))
-	}
-	l = len(m.Code)
-	if l > 0 {
-		n += 1 + l + sovPipeline(uint64(l))
-	}
-	l = len(m.Data)
-	if l > 0 {
-		n += 1 + l + sovPipeline(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
+
+func (m *AnalysisList) MarshalTo(dAtA []byte) (int, error) {
+	size := m.ProtoSize()
+	return m.MarshalToSizedBuffer(dAtA[:size])
 }
 
-func (m *Dispatch) ProtoSize() (n int) {
-	if m == nil {
-		return 0
-	}
+func (m *AnalysisList) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
 	var l int
 	_ = l
-	if m.All {
-		n += 2
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.Children) > 0 {
-		l = 0
-		for _, e := range m.Children {
-			l += sovPipeline(uint64(e))
+	if len(m.List) > 0 {
+		for iNdEx := len(m.List) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.List[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0xa
 		}
-		n += 1 + sovPipeline(uint64(l)) + l
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
 	}
-	return n
+	return len(dAtA) - i, nil
 }
 
-func (m *Aggregate) ProtoSize() (n int) {
-	if m == nil {
-		return 0
-	}
-	var l int
-	_ = l
-	if m.Op != 0 {
-		n += 1 + sovPipeline(uint64(m.Op))
-	}
-	if m.Dist {
-		n += 2
-	}
-	if m.Expr != nil {
-		l = m.Expr.ProtoSize()
-		n += 1 + l + sovPipeline(uint64(l))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
+func (m *Source) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
 	}
-	return n
+	return dAtA[:n], nil
 }
 
-func (m *Group) ProtoSize() (n int) {
-	if m == nil {
-		return 0
-	}
+func (m *Source) MarshalTo(dAtA []byte) (int, error) {
+	size := m.ProtoSize()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Source) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
 	var l int
 	_ = l
-	if m.NeedEval {
-		n += 2
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if m.Ibucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Ibucket))
+	if len(m.PushdownAddr) > 0 {
+		i -= len(m.PushdownAddr)
+		copy(dAtA[i:], m.PushdownAddr)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.PushdownAddr)))
+		i--
+		dAtA[i] = 0x32
 	}
-	if m.Nbucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Nbucket))
+	if m.PushdownId != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.PushdownId))
+		i--
+		dAtA[i] = 0x28
 	}
-	if len(m.Exprs) > 0 {
-		for _, e := range m.Exprs {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
-		}
+	if len(m.Block) > 0 {
+		i -= len(m.Block)
+		copy(dAtA[i:], m.Block)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Block)))
+		i--
+		dAtA[i] = 0x22
 	}
-	if len(m.Types) > 0 {
-		for _, e := range m.Types {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
+	if len(m.ColList) > 0 {
+		for iNdEx := len(m.ColList) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.ColList[iNdEx])
+			copy(dAtA[i:], m.ColList[iNdEx])
+			i = encodeVarintPipeline(dAtA, i, uint64(len(m.ColList[iNdEx])))
+			i--
+			dAtA[i] = 0x1a
 		}
 	}
-	if len(m.Aggs) > 0 {
-		for _, e := range m.Aggs {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
-		}
+	if len(m.TableName) > 0 {
+		i -= len(m.TableName)
+		copy(dAtA[i:], m.TableName)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.TableName)))
+		i--
+		dAtA[i] = 0x12
 	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
+	if len(m.SchemaName) > 0 {
+		i -= len(m.SchemaName)
+		copy(dAtA[i:], m.SchemaName)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.SchemaName)))
+		i--
+		dAtA[i] = 0xa
 	}
-	return n
+	return len(dAtA) - i, nil
 }
 
-func (m *AntiJoin) ProtoSize() (n int) {
-	if m == nil {
-		return 0
+func (m *NodeInfo) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
 	}
+	return dAtA[:n], nil
+}
+
+func (m *NodeInfo) MarshalTo(dAtA []byte) (int, error) {
+	size := m.ProtoSize()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *NodeInfo) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
 	var l int
 	_ = l
-	if m.Ibucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Ibucket))
-	}
-	if m.Nbucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Nbucket))
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.Result) > 0 {
-		l = 0
-		for _, e := range m.Result {
-			l += sovPipeline(uint64(e))
+	if len(m.Payload) > 0 {
+		for iNdEx := len(m.Payload) - 1; iNdEx >= 0; iNdEx-- {
+			i -= len(m.Payload[iNdEx])
+			copy(dAtA[i:], m.Payload[iNdEx])
+			i = encodeVarintPipeline(dAtA, i, uint64(len(m.Payload[iNdEx])))
+			i--
+			dAtA[i] = 0x22
 		}
-		n += 1 + sovPipeline(uint64(l)) + l
 	}
-	if len(m.LeftCond) > 0 {
-		for _, e := range m.LeftCond {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
-		}
+	if len(m.Addr) > 0 {
+		i -= len(m.Addr)
+		copy(dAtA[i:], m.Addr)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Addr)))
+		i--
+		dAtA[i] = 0x1a
 	}
-	if len(m.RightCond) > 0 {
-		for _, e := range m.RightCond {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
-		}
+	if len(m.Id) > 0 {
+		i -= len(m.Id)
+		copy(dAtA[i:], m.Id)
+		i = encodeVarintPipeline(dAtA, i, uint64(len(m.Id)))
+		i--
+		dAtA[i] = 0x12
 	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
+	if m.Mcpu != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.Mcpu))
+		i--
+		dAtA[i] = 0x8
 	}
-	return n
+	return len(dAtA) - i, nil
 }
 
-func (m *InnerJoin) ProtoSize() (n int) {
-	if m == nil {
-		return 0
+func (m *Pipeline) Marshal() (dAtA []byte, err error) {
+	size := m.ProtoSize()
+	dAtA = make([]byte, size)
+	n, err := m.MarshalToSizedBuffer(dAtA[:size])
+	if err != nil {
+		return nil, err
 	}
+	return dAtA[:n], nil
+}
+
+func (m *Pipeline) MarshalTo(dAtA []byte) (int, error) {
+	size := m.ProtoSize()
+	return m.MarshalToSizedBuffer(dAtA[:size])
+}
+
+func (m *Pipeline) MarshalToSizedBuffer(dAtA []byte) (int, error) {
+	i := len(dAtA)
+	_ = i
 	var l int
 	_ = l
-	if m.Ibucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Ibucket))
-	}
-	if m.Nbucket != 0 {
-		n += 1 + sovPipeline(uint64(m.Nbucket))
+	if m.XXX_unrecognized != nil {
+		i -= len(m.XXX_unrecognized)
+		copy(dAtA[i:], m.XXX_unrecognized)
 	}
-	if len(m.RelList) > 0 {
-		l = 0
-		for _, e := range m.RelList {
-			l += sovPipeline(uint64(e))
-		}
-		n += 1 + sovPipeline(uint64(l)) + l
+	if m.ChildrenCount != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.ChildrenCount))
+		i--
+		dAtA[i] = 0x58
 	}
-	if len(m.ColList) > 0 {
-		l = 0
-		for _, e := range m.ColList {
-			l += sovPipeline(uint64(e))
+	if m.PushDownInfo != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.PushDownInfo))
+		i--
+		dAtA[i] = 0x50
+	}
+	if m.Node != nil {
+		{
+			size, err := m.Node.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
 		}
-		n += 1 + sovPipeline(uint64(l)) + l
+		i--
+		dAtA[i] = 0x4a
 	}
-	if len(m.LeftCond) > 0 {
-		for _, e := range m.LeftCond {
+	if m.IsJoin {
+		i--
+		if m.IsJoin {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x40
+	}
+	if m.IsEnd {
+		i--
+		if m.IsEnd {
+			dAtA[i] = 1
+		} else {
+			dAtA[i] = 0
+		}
+		i--
+		dAtA[i] = 0x38
+	}
+	if len(m.InstructionList) > 0 {
+		for iNdEx := len(m.InstructionList) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.InstructionList[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x32
+		}
+	}
+	if len(m.Children) > 0 {
+		for iNdEx := len(m.Children) - 1; iNdEx >= 0; iNdEx-- {
+			{
+				size, err := m.Children[iNdEx].MarshalToSizedBuffer(dAtA[:i])
+				if err != nil {
+					return 0, err
+				}
+				i -= size
+				i = encodeVarintPipeline(dAtA, i, uint64(size))
+			}
+			i--
+			dAtA[i] = 0x2a
+		}
+	}
+	if m.DataSource != nil {
+		{
+			size, err := m.DataSource.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x22
+	}
+	if m.Qry != nil {
+		{
+			size, err := m.Qry.MarshalToSizedBuffer(dAtA[:i])
+			if err != nil {
+				return 0, err
+			}
+			i -= size
+			i = encodeVarintPipeline(dAtA, i, uint64(size))
+		}
+		i--
+		dAtA[i] = 0x1a
+	}
+	if m.PipelineId != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.PipelineId))
+		i--
+		dAtA[i] = 0x10
+	}
+	if m.PipelineType != 0 {
+		i = encodeVarintPipeline(dAtA, i, uint64(m.PipelineType))
+		i--
+		dAtA[i] = 0x8
+	}
+	return len(dAtA) - i, nil
+}
+
+func encodeVarintPipeline(dAtA []byte, offset int, v uint64) int {
+	offset -= sovPipeline(v)
+	base := offset
+	for v >= 1<<7 {
+		dAtA[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	dAtA[offset] = uint8(v)
+	return base
+}
+func (m *Message) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Sid != 0 {
+		n += 1 + sovPipeline(uint64(m.Sid))
+	}
+	if m.Cmd != 0 {
+		n += 1 + sovPipeline(uint64(m.Cmd))
+	}
+	l = len(m.Code)
+	if l > 0 {
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	l = len(m.Data)
+	if l > 0 {
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	l = len(m.Analyse)
+	if l > 0 {
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Connector) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.PipelineId != 0 {
+		n += 1 + sovPipeline(uint64(m.PipelineId))
+	}
+	if m.ConnectorIndex != 0 {
+		n += 1 + sovPipeline(uint64(m.ConnectorIndex))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Dispatch) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.All {
+		n += 2
+	}
+	if len(m.Connector) > 0 {
+		for _, e := range m.Connector {
 			l = e.ProtoSize()
 			n += 1 + l + sovPipeline(uint64(l))
 		}
 	}
-	if len(m.RightCond) > 0 {
-		for _, e := range m.RightCond {
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Aggregate) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Op != 0 {
+		n += 1 + sovPipeline(uint64(m.Op))
+	}
+	if m.Dist {
+		n += 2
+	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Group) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.NeedEval {
+		n += 2
+	}
+	if m.Ibucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Ibucket))
+	}
+	if m.Nbucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Nbucket))
+	}
+	if len(m.Exprs) > 0 {
+		for _, e := range m.Exprs {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.Aggs) > 0 {
+		for _, e := range m.Aggs {
 			l = e.ProtoSize()
 			n += 1 + l + sovPipeline(uint64(l))
 		}
@@ -2763,7 +3761,7 @@ func (m *InnerJoin) ProtoSize() (n int) {
 	return n
 }
 
-func (m *LeftJoin) ProtoSize() (n int) {
+func (m *Join) ProtoSize() (n int) {
 	if m == nil {
 		return 0
 	}
@@ -2789,6 +3787,10 @@ func (m *LeftJoin) ProtoSize() (n int) {
 		}
 		n += 1 + sovPipeline(uint64(l)) + l
 	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
 	if len(m.Types) > 0 {
 		for _, e := range m.Types {
 			l = e.ProtoSize()
@@ -2813,7 +3815,7 @@ func (m *LeftJoin) ProtoSize() (n int) {
 	return n
 }
 
-func (m *SemiJoin) ProtoSize() (n int) {
+func (m *AntiJoin) ProtoSize() (n int) {
 	if m == nil {
 		return 0
 	}
@@ -2832,6 +3834,16 @@ func (m *SemiJoin) ProtoSize() (n int) {
 		}
 		n += 1 + sovPipeline(uint64(l)) + l
 	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
 	if len(m.LeftCond) > 0 {
 		for _, e := range m.LeftCond {
 			l = e.ProtoSize()
@@ -2850,7 +3862,7 @@ func (m *SemiJoin) ProtoSize() (n int) {
 	return n
 }
 
-func (m *SingleJoin) ProtoSize() (n int) {
+func (m *InnerJoin) ProtoSize() (n int) {
 	if m == nil {
 		return 0
 	}
@@ -2876,6 +3888,10 @@ func (m *SingleJoin) ProtoSize() (n int) {
 		}
 		n += 1 + sovPipeline(uint64(l)) + l
 	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
 	if len(m.Types) > 0 {
 		for _, e := range m.Types {
 			l = e.ProtoSize()
@@ -2900,17 +3916,237 @@ func (m *SingleJoin) ProtoSize() (n int) {
 	return n
 }
 
-func (m *Product) ProtoSize() (n int) {
+func (m *LeftJoin) ProtoSize() (n int) {
 	if m == nil {
 		return 0
 	}
 	var l int
 	_ = l
-	if len(m.RelList) > 0 {
-		l = 0
-		for _, e := range m.RelList {
-			l += sovPipeline(uint64(e))
-		}
+	if m.Ibucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Ibucket))
+	}
+	if m.Nbucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Nbucket))
+	}
+	if len(m.RelList) > 0 {
+		l = 0
+		for _, e := range m.RelList {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if len(m.ColList) > 0 {
+		l = 0
+		for _, e := range m.ColList {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.LeftCond) > 0 {
+		for _, e := range m.LeftCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.RightCond) > 0 {
+		for _, e := range m.RightCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *SemiJoin) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Ibucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Ibucket))
+	}
+	if m.Nbucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Nbucket))
+	}
+	if len(m.Result) > 0 {
+		l = 0
+		for _, e := range m.Result {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.LeftCond) > 0 {
+		for _, e := range m.LeftCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.RightCond) > 0 {
+		for _, e := range m.RightCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *SingleJoin) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Ibucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Ibucket))
+	}
+	if m.Nbucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Nbucket))
+	}
+	if len(m.RelList) > 0 {
+		l = 0
+		for _, e := range m.RelList {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if len(m.ColList) > 0 {
+		l = 0
+		for _, e := range m.ColList {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if m.Expr != nil {
+		l = m.Expr.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.LeftCond) > 0 {
+		for _, e := range m.LeftCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.RightCond) > 0 {
+		for _, e := range m.RightCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *MarkJoin) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.Ibucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Ibucket))
+	}
+	if m.Nbucket != 0 {
+		n += 1 + sovPipeline(uint64(m.Nbucket))
+	}
+	if len(m.Result) > 0 {
+		l = 0
+		for _, e := range m.Result {
+			l += sovPipeline(uint64(e))
+		}
+		n += 1 + sovPipeline(uint64(l)) + l
+	}
+	if len(m.LeftCond) > 0 {
+		for _, e := range m.LeftCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.RightCond) > 0 {
+		for _, e := range m.RightCond {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.Cond != nil {
+		l = m.Cond.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.OnList) > 0 {
+		for _, e := range m.OnList {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.OutputNull {
+		n += 2
+	}
+	if m.OutputMark {
+		n += 2
+	}
+	if m.OutputAnyway {
+		n += 2
+	}
+	if m.MarkMeaning {
+		n += 2
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Product) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if len(m.RelList) > 0 {
+		l = 0
+		for _, e := range m.RelList {
+			l += sovPipeline(uint64(e))
+		}
 		n += 1 + sovPipeline(uint64(l)) + l
 	}
 	if len(m.ColList) > 0 {
@@ -2920,6 +4156,12 @@ func (m *Product) ProtoSize() (n int) {
 		}
 		n += 1 + sovPipeline(uint64(l)) + l
 	}
+	if len(m.Types) > 0 {
+		for _, e := range m.Types {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
 	if m.XXX_unrecognized != nil {
 		n += len(m.XXX_unrecognized)
 	}
@@ -2942,6 +4184,10 @@ func (m *Instruction) ProtoSize() (n int) {
 		l = m.Anti.ProtoSize()
 		n += 1 + l + sovPipeline(uint64(l))
 	}
+	if m.Connect != nil {
+		l = m.Connect.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
 	if m.Dispatch != nil {
 		l = m.Dispatch.ProtoSize()
 		n += 1 + l + sovPipeline(uint64(l))
@@ -2966,6 +4212,14 @@ func (m *Instruction) ProtoSize() (n int) {
 		l = m.SingleJoin.ProtoSize()
 		n += 1 + l + sovPipeline(uint64(l))
 	}
+	if m.MarkJoin != nil {
+		l = m.MarkJoin.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if m.Join != nil {
+		l = m.Join.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
 	if m.Product != nil {
 		l = m.Product.ProtoSize()
 		n += 1 + l + sovPipeline(uint64(l))
@@ -2984,13 +4238,31 @@ func (m *Instruction) ProtoSize() (n int) {
 	}
 	if m.Filter != nil {
 		l = m.Filter.ProtoSize()
-		n += 1 + l + sovPipeline(uint64(l))
+		n += 2 + l + sovPipeline(uint64(l))
 	}
 	if m.Limit != 0 {
-		n += 1 + sovPipeline(uint64(m.Limit))
+		n += 2 + sovPipeline(uint64(m.Limit))
 	}
 	if m.Offset != 0 {
-		n += 1 + sovPipeline(uint64(m.Offset))
+		n += 2 + sovPipeline(uint64(m.Offset))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *AnalysisList) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if len(m.List) > 0 {
+		for _, e := range m.List {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
 	}
 	if m.XXX_unrecognized != nil {
 		n += len(m.XXX_unrecognized)
@@ -3022,6 +4294,13 @@ func (m *Source) ProtoSize() (n int) {
 	if l > 0 {
 		n += 1 + l + sovPipeline(uint64(l))
 	}
+	if m.PushdownId != 0 {
+		n += 1 + sovPipeline(uint64(m.PushdownId))
+	}
+	l = len(m.PushdownAddr)
+	if l > 0 {
+		n += 1 + l + sovPipeline(uint64(l))
+	}
 	if m.XXX_unrecognized != nil {
 		n += len(m.XXX_unrecognized)
 	}
@@ -3051,67 +4330,584 @@ func (m *NodeInfo) ProtoSize() (n int) {
 			n += 1 + l + sovPipeline(uint64(l))
 		}
 	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func (m *Pipeline) ProtoSize() (n int) {
+	if m == nil {
+		return 0
+	}
+	var l int
+	_ = l
+	if m.PipelineType != 0 {
+		n += 1 + sovPipeline(uint64(m.PipelineType))
+	}
+	if m.PipelineId != 0 {
+		n += 1 + sovPipeline(uint64(m.PipelineId))
+	}
+	if m.Qry != nil {
+		l = m.Qry.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if m.DataSource != nil {
+		l = m.DataSource.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if len(m.Children) > 0 {
+		for _, e := range m.Children {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if len(m.InstructionList) > 0 {
+		for _, e := range m.InstructionList {
+			l = e.ProtoSize()
+			n += 1 + l + sovPipeline(uint64(l))
+		}
+	}
+	if m.IsEnd {
+		n += 2
+	}
+	if m.IsJoin {
+		n += 2
+	}
+	if m.Node != nil {
+		l = m.Node.ProtoSize()
+		n += 1 + l + sovPipeline(uint64(l))
+	}
+	if m.PushDownInfo != 0 {
+		n += 1 + sovPipeline(uint64(m.PushDownInfo))
+	}
+	if m.ChildrenCount != 0 {
+		n += 1 + sovPipeline(uint64(m.ChildrenCount))
+	}
+	if m.XXX_unrecognized != nil {
+		n += len(m.XXX_unrecognized)
+	}
+	return n
+}
+
+func sovPipeline(x uint64) (n int) {
+	return (math_bits.Len64(x|1) + 6) / 7
+}
+func sozPipeline(x uint64) (n int) {
+	return sovPipeline(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (m *Message) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowPipeline
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Message: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Message: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Sid", wireType)
+			}
+			m.Sid = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Sid |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Cmd", wireType)
+			}
+			m.Cmd = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Cmd |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Code", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Code = append(m.Code[:0], dAtA[iNdEx:postIndex]...)
+			if m.Code == nil {
+				m.Code = []byte{}
+			}
+			iNdEx = postIndex
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...)
+			if m.Data == nil {
+				m.Data = []byte{}
+			}
+			iNdEx = postIndex
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Analyse", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				byteLen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Analyse = append(m.Analyse[:0], dAtA[iNdEx:postIndex]...)
+			if m.Analyse == nil {
+				m.Analyse = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipPipeline(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if (skippy < 0) || (iNdEx+skippy) < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *Connector) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowPipeline
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Connector: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Connector: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PipelineId", wireType)
+			}
+			m.PipelineId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.PipelineId |= int32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ConnectorIndex", wireType)
+			}
+			m.ConnectorIndex = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ConnectorIndex |= int32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		default:
+			iNdEx = preIndex
+			skippy, err := skipPipeline(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if (skippy < 0) || (iNdEx+skippy) < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *Dispatch) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowPipeline
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Dispatch: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Dispatch: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field All", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.All = bool(v != 0)
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Connector", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Connector = append(m.Connector, &Connector{})
+			if err := m.Connector[len(m.Connector)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipPipeline(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if (skippy < 0) || (iNdEx+skippy) < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
 
-func (m *Pipeline) ProtoSize() (n int) {
-	if m == nil {
-		return 0
-	}
-	var l int
-	_ = l
-	if m.PipelineType != 0 {
-		n += 1 + sovPipeline(uint64(m.PipelineType))
-	}
-	if m.PipelineId != 0 {
-		n += 1 + sovPipeline(uint64(m.PipelineId))
-	}
-	if m.Qry != nil {
-		l = m.Qry.ProtoSize()
-		n += 1 + l + sovPipeline(uint64(l))
-	}
-	if m.DataSource != nil {
-		l = m.DataSource.ProtoSize()
-		n += 1 + l + sovPipeline(uint64(l))
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
 	}
-	if len(m.Children) > 0 {
-		for _, e := range m.Children {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
+	return nil
+}
+func (m *Aggregate) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowPipeline
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
 		}
-	}
-	if len(m.InstructionList) > 0 {
-		for _, e := range m.InstructionList {
-			l = e.ProtoSize()
-			n += 1 + l + sovPipeline(uint64(l))
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: Aggregate: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: Aggregate: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType)
+			}
+			m.Op = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Op |= int32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Dist", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Dist = bool(v != 0)
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipPipeline(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if (skippy < 0) || (iNdEx+skippy) < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
 		}
 	}
-	if m.IsEnd {
-		n += 2
-	}
-	if m.Node != nil {
-		l = m.Node.ProtoSize()
-		n += 1 + l + sovPipeline(uint64(l))
-	}
-	if m.PushDownInfo != 0 {
-		n += 1 + sovPipeline(uint64(m.PushDownInfo))
-	}
-	if m.XXX_unrecognized != nil {
-		n += len(m.XXX_unrecognized)
-	}
-	return n
-}
 
-func sovPipeline(x uint64) (n int) {
-	return (math_bits.Len64(x|1) + 6) / 7
-}
-func sozPipeline(x uint64) (n int) {
-	return sovPipeline(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
 }
-func (m *Message) Unmarshal(dAtA []byte) error {
+func (m *Group) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -3134,17 +4930,17 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: Message: wiretype end group for non-group")
+			return fmt.Errorf("proto: Group: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Message: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: Group: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
 			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Sid", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field NeedEval", wireType)
 			}
-			m.Sid = 0
+			var v int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3154,16 +4950,17 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				m.Sid |= uint64(b&0x7F) << shift
+				v |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
+			m.NeedEval = bool(v != 0)
 		case 2:
 			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Cmd", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Ibucket", wireType)
 			}
-			m.Cmd = 0
+			m.Ibucket = 0
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3173,16 +4970,69 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				m.Cmd |= uint64(b&0x7F) << shift
+				m.Ibucket |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Nbucket", wireType)
+			}
+			m.Nbucket = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Nbucket |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Exprs", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-		case 3:
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Exprs = append(m.Exprs, &plan.Expr{})
+			if err := m.Exprs[len(m.Exprs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 5:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Code", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
 			}
-			var byteLen int
+			var msglen int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3192,31 +5042,31 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				byteLen |= int(b&0x7F) << shift
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-			if byteLen < 0 {
+			if msglen < 0 {
 				return ErrInvalidLengthPipeline
 			}
-			postIndex := iNdEx + byteLen
+			postIndex := iNdEx + msglen
 			if postIndex < 0 {
 				return ErrInvalidLengthPipeline
 			}
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.Code = append(m.Code[:0], dAtA[iNdEx:postIndex]...)
-			if m.Code == nil {
-				m.Code = []byte{}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
 			}
 			iNdEx = postIndex
-		case 4:
+		case 6:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Aggs", wireType)
 			}
-			var byteLen int
+			var msglen int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3226,24 +5076,24 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				byteLen |= int(b&0x7F) << shift
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-			if byteLen < 0 {
+			if msglen < 0 {
 				return ErrInvalidLengthPipeline
 			}
-			postIndex := iNdEx + byteLen
+			postIndex := iNdEx + msglen
 			if postIndex < 0 {
 				return ErrInvalidLengthPipeline
 			}
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.Data = append(m.Data[:0], dAtA[iNdEx:postIndex]...)
-			if m.Data == nil {
-				m.Data = []byte{}
+			m.Aggs = append(m.Aggs, &Aggregate{})
+			if err := m.Aggs[len(m.Aggs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
 			}
 			iNdEx = postIndex
 		default:
@@ -3268,7 +5118,7 @@ func (m *Message) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *Dispatch) Unmarshal(dAtA []byte) error {
+func (m *Join) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -3291,17 +5141,17 @@ func (m *Dispatch) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: Dispatch: wiretype end group for non-group")
+			return fmt.Errorf("proto: Join: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Dispatch: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: Join: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
 			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field All", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Ibucket", wireType)
 			}
-			var v int
+			m.Ibucket = 0
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3311,13 +5161,31 @@ func (m *Dispatch) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				v |= int(b&0x7F) << shift
+				m.Ibucket |= uint64(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-			m.All = bool(v != 0)
 		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Nbucket", wireType)
+			}
+			m.Nbucket = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Nbucket |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 3:
 			if wireType == 0 {
 				var v int32
 				for shift := uint(0); ; shift += 7 {
@@ -3334,7 +5202,7 @@ func (m *Dispatch) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.Children = append(m.Children, v)
+				m.RelList = append(m.RelList, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -3369,8 +5237,8 @@ func (m *Dispatch) Unmarshal(dAtA []byte) error {
 					}
 				}
 				elementCount = count
-				if elementCount != 0 && len(m.Children) == 0 {
-					m.Children = make([]int32, 0, elementCount)
+				if elementCount != 0 && len(m.RelList) == 0 {
+					m.RelList = make([]int32, 0, elementCount)
 				}
 				for iNdEx < postIndex {
 					var v int32
@@ -3388,67 +5256,128 @@ func (m *Dispatch) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.Children = append(m.Children, v)
+					m.RelList = append(m.RelList, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Children", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType)
 			}
-		default:
-			iNdEx = preIndex
-			skippy, err := skipPipeline(dAtA[iNdEx:])
-			if err != nil {
-				return err
+		case 4:
+			if wireType == 0 {
+				var v int32
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= int32(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.ColList = append(m.ColList, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.ColList) == 0 {
+					m.ColList = make([]int32, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v int32
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowPipeline
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= int32(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.ColList = append(m.ColList, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
 			}
-			if (skippy < 0) || (iNdEx+skippy) < 0 {
-				return ErrInvalidLengthPipeline
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
 			}
-			if (iNdEx + skippy) > l {
-				return io.ErrUnexpectedEOF
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
 			}
-			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
-			iNdEx += skippy
-		}
-	}
-
-	if iNdEx > l {
-		return io.ErrUnexpectedEOF
-	}
-	return nil
-}
-func (m *Aggregate) Unmarshal(dAtA []byte) error {
-	l := len(dAtA)
-	iNdEx := 0
-	for iNdEx < l {
-		preIndex := iNdEx
-		var wire uint64
-		for shift := uint(0); ; shift += 7 {
-			if shift >= 64 {
-				return ErrIntOverflowPipeline
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
 			}
-			if iNdEx >= l {
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			b := dAtA[iNdEx]
-			iNdEx++
-			wire |= uint64(b&0x7F) << shift
-			if b < 0x80 {
-				break
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
 			}
-		}
-		fieldNum := int32(wire >> 3)
-		wireType := int(wire & 0x7)
-		if wireType == 4 {
-			return fmt.Errorf("proto: Aggregate: wiretype end group for non-group")
-		}
-		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Aggregate: illegal tag %d (wire type %d)", fieldNum, wire)
-		}
-		switch fieldNum {
-		case 1:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Op", wireType)
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
 			}
-			m.Op = 0
+			iNdEx = postIndex
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+			}
+			var msglen int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3458,16 +5387,31 @@ func (m *Aggregate) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				m.Op |= int32(b&0x7F) << shift
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-		case 2:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Dist", wireType)
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 7:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
 			}
-			var v int
+			var msglen int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3477,15 +5421,29 @@ func (m *Aggregate) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				v |= int(b&0x7F) << shift
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-			m.Dist = bool(v != 0)
-		case 3:
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.LeftCond = append(m.LeftCond, &plan.Expr{})
+			if err := m.LeftCond[len(m.LeftCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 8:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -3512,10 +5470,8 @@ func (m *Aggregate) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			if m.Expr == nil {
-				m.Expr = &plan.Expr{}
-			}
-			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			m.RightCond = append(m.RightCond, &plan.Expr{})
+			if err := m.RightCond[len(m.RightCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
@@ -3541,7 +5497,7 @@ func (m *Aggregate) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *Group) Unmarshal(dAtA []byte) error {
+func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -3564,17 +5520,17 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: Group: wiretype end group for non-group")
+			return fmt.Errorf("proto: AntiJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: Group: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: AntiJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
 			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field NeedEval", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Ibucket", wireType)
 			}
-			var v int
+			m.Ibucket = 0
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3584,17 +5540,16 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				v |= int(b&0x7F) << shift
+				m.Ibucket |= uint64(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-			m.NeedEval = bool(v != 0)
 		case 2:
 			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Ibucket", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Nbucket", wireType)
 			}
-			m.Ibucket = 0
+			m.Nbucket = 0
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3604,16 +5559,92 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				m.Ibucket |= uint64(b&0x7F) << shift
+				m.Nbucket |= uint64(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
 		case 3:
-			if wireType != 0 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Nbucket", wireType)
+			if wireType == 0 {
+				var v int32
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= int32(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.Result = append(m.Result, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.Result) == 0 {
+					m.Result = make([]int32, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v int32
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowPipeline
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= int32(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.Result = append(m.Result, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
 			}
-			m.Nbucket = 0
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
 			for shift := uint(0); ; shift += 7 {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
@@ -3623,14 +5654,31 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 				}
 				b := dAtA[iNdEx]
 				iNdEx++
-				m.Nbucket |= uint64(b&0x7F) << shift
+				msglen |= int(b&0x7F) << shift
 				if b < 0x80 {
 					break
 				}
 			}
-		case 4:
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 5:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Exprs", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -3657,14 +5705,14 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.Exprs = append(m.Exprs, &plan.Expr{})
-			if err := m.Exprs[len(m.Exprs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
-		case 5:
+		case 6:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -3691,14 +5739,14 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.Types = append(m.Types, &plan.Type{})
-			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			m.LeftCond = append(m.LeftCond, &plan.Expr{})
+			if err := m.LeftCond[len(m.LeftCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
-		case 6:
+		case 7:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Aggs", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -3725,8 +5773,8 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.Aggs = append(m.Aggs, &Aggregate{})
-			if err := m.Aggs[len(m.Aggs)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			m.RightCond = append(m.RightCond, &plan.Expr{})
+			if err := m.RightCond[len(m.RightCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
@@ -3752,7 +5800,7 @@ func (m *Group) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *AntiJoin) Unmarshal(dAtA []byte) error {
+func (m *InnerJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -3775,10 +5823,10 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: AntiJoin: wiretype end group for non-group")
+			return fmt.Errorf("proto: InnerJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: AntiJoin: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: InnerJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
@@ -3819,7 +5867,83 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 					break
 				}
 			}
-		case 3:
+		case 3:
+			if wireType == 0 {
+				var v int32
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= int32(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.RelList = append(m.RelList, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.RelList) == 0 {
+					m.RelList = make([]int32, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v int32
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowPipeline
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= int32(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.RelList = append(m.RelList, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType)
+			}
+		case 4:
 			if wireType == 0 {
 				var v int32
 				for shift := uint(0); ; shift += 7 {
@@ -3836,7 +5960,7 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.Result = append(m.Result, v)
+				m.ColList = append(m.ColList, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -3871,8 +5995,8 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 					}
 				}
 				elementCount = count
-				if elementCount != 0 && len(m.Result) == 0 {
-					m.Result = make([]int32, 0, elementCount)
+				if elementCount != 0 && len(m.ColList) == 0 {
+					m.ColList = make([]int32, 0, elementCount)
 				}
 				for iNdEx < postIndex {
 					var v int32
@@ -3890,12 +6014,82 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.Result = append(m.Result, v)
+					m.ColList = append(m.ColList, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
 			}
-		case 4:
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 7:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
 			}
@@ -3929,7 +6123,7 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 5:
+		case 8:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
 			}
@@ -3985,7 +6179,7 @@ func (m *AntiJoin) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *InnerJoin) Unmarshal(dAtA []byte) error {
+func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -4008,10 +6202,10 @@ func (m *InnerJoin) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: InnerJoin: wiretype end group for non-group")
+			return fmt.Errorf("proto: LeftJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: InnerJoin: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: LeftJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
@@ -4205,6 +6399,76 @@ func (m *InnerJoin) Unmarshal(dAtA []byte) error {
 				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
 			}
 		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 7:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
 			}
@@ -4238,7 +6502,7 @@ func (m *InnerJoin) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 6:
+		case 8:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
 			}
@@ -4294,7 +6558,7 @@ func (m *InnerJoin) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *LeftJoin) Unmarshal(dAtA []byte) error {
+func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -4317,10 +6581,10 @@ func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: LeftJoin: wiretype end group for non-group")
+			return fmt.Errorf("proto: SemiJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: LeftJoin: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: SemiJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
@@ -4378,7 +6642,7 @@ func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.RelList = append(m.RelList, v)
+				m.Result = append(m.Result, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -4413,8 +6677,8 @@ func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 					}
 				}
 				elementCount = count
-				if elementCount != 0 && len(m.RelList) == 0 {
-					m.RelList = make([]int32, 0, elementCount)
+				if elementCount != 0 && len(m.Result) == 0 {
+					m.Result = make([]int32, 0, elementCount)
 				}
 				for iNdEx < postIndex {
 					var v int32
@@ -4432,87 +6696,47 @@ func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.RelList = append(m.RelList, v)
+					m.Result = append(m.Result, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
 			}
 		case 4:
-			if wireType == 0 {
-				var v int32
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowPipeline
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= int32(b&0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.ColList = append(m.ColList, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowPipeline
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= int(b&0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthPipeline
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex < 0 {
-					return ErrInvalidLengthPipeline
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
 				}
-				if postIndex > l {
+				if iNdEx >= l {
 					return io.ErrUnexpectedEOF
 				}
-				var elementCount int
-				var count int
-				for _, integer := range dAtA[iNdEx:postIndex] {
-					if integer < 128 {
-						count++
-					}
-				}
-				elementCount = count
-				if elementCount != 0 && len(m.ColList) == 0 {
-					m.ColList = make([]int32, 0, elementCount)
-				}
-				for iNdEx < postIndex {
-					var v int32
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowPipeline
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= int32(b&0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.ColList = append(m.ColList, v)
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
 				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
 			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
 		case 5:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
@@ -4637,7 +6861,7 @@ func (m *LeftJoin) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *SemiJoin) Unmarshal(dAtA []byte) error {
+func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -4660,10 +6884,10 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: SemiJoin: wiretype end group for non-group")
+			return fmt.Errorf("proto: SingleJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: SemiJoin: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: SingleJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
@@ -4721,7 +6945,7 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.Result = append(m.Result, v)
+				m.RelList = append(m.RelList, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -4756,8 +6980,8 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 					}
 				}
 				elementCount = count
-				if elementCount != 0 && len(m.Result) == 0 {
-					m.Result = make([]int32, 0, elementCount)
+				if elementCount != 0 && len(m.RelList) == 0 {
+					m.RelList = make([]int32, 0, elementCount)
 				}
 				for iNdEx < postIndex {
 					var v int32
@@ -4775,12 +6999,158 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.Result = append(m.Result, v)
+					m.RelList = append(m.RelList, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType)
 			}
 		case 4:
+			if wireType == 0 {
+				var v int32
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					v |= int32(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				m.ColList = append(m.ColList, v)
+			} else if wireType == 2 {
+				var packedLen int
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return ErrIntOverflowPipeline
+					}
+					if iNdEx >= l {
+						return io.ErrUnexpectedEOF
+					}
+					b := dAtA[iNdEx]
+					iNdEx++
+					packedLen |= int(b&0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				if packedLen < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				postIndex := iNdEx + packedLen
+				if postIndex < 0 {
+					return ErrInvalidLengthPipeline
+				}
+				if postIndex > l {
+					return io.ErrUnexpectedEOF
+				}
+				var elementCount int
+				var count int
+				for _, integer := range dAtA[iNdEx:postIndex] {
+					if integer < 128 {
+						count++
+					}
+				}
+				elementCount = count
+				if elementCount != 0 && len(m.ColList) == 0 {
+					m.ColList = make([]int32, 0, elementCount)
+				}
+				for iNdEx < postIndex {
+					var v int32
+					for shift := uint(0); ; shift += 7 {
+						if shift >= 64 {
+							return ErrIntOverflowPipeline
+						}
+						if iNdEx >= l {
+							return io.ErrUnexpectedEOF
+						}
+						b := dAtA[iNdEx]
+						iNdEx++
+						v |= int32(b&0x7F) << shift
+						if b < 0x80 {
+							break
+						}
+					}
+					m.ColList = append(m.ColList, v)
+				}
+			} else {
+				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
+			}
+		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Expr", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Expr == nil {
+				m.Expr = &plan.Expr{}
+			}
+			if err := m.Expr.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 7:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
 			}
@@ -4814,7 +7184,7 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 5:
+		case 8:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
 			}
@@ -4870,7 +7240,7 @@ func (m *SemiJoin) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
-func (m *SingleJoin) Unmarshal(dAtA []byte) error {
+func (m *MarkJoin) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
 	for iNdEx < l {
@@ -4893,10 +7263,10 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 		fieldNum := int32(wire >> 3)
 		wireType := int(wire & 0x7)
 		if wireType == 4 {
-			return fmt.Errorf("proto: SingleJoin: wiretype end group for non-group")
+			return fmt.Errorf("proto: MarkJoin: wiretype end group for non-group")
 		}
 		if fieldNum <= 0 {
-			return fmt.Errorf("proto: SingleJoin: illegal tag %d (wire type %d)", fieldNum, wire)
+			return fmt.Errorf("proto: MarkJoin: illegal tag %d (wire type %d)", fieldNum, wire)
 		}
 		switch fieldNum {
 		case 1:
@@ -4927,93 +7297,17 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 				if shift >= 64 {
 					return ErrIntOverflowPipeline
 				}
-				if iNdEx >= l {
-					return io.ErrUnexpectedEOF
-				}
-				b := dAtA[iNdEx]
-				iNdEx++
-				m.Nbucket |= uint64(b&0x7F) << shift
-				if b < 0x80 {
-					break
-				}
-			}
-		case 3:
-			if wireType == 0 {
-				var v int32
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowPipeline
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					v |= int32(b&0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				m.RelList = append(m.RelList, v)
-			} else if wireType == 2 {
-				var packedLen int
-				for shift := uint(0); ; shift += 7 {
-					if shift >= 64 {
-						return ErrIntOverflowPipeline
-					}
-					if iNdEx >= l {
-						return io.ErrUnexpectedEOF
-					}
-					b := dAtA[iNdEx]
-					iNdEx++
-					packedLen |= int(b&0x7F) << shift
-					if b < 0x80 {
-						break
-					}
-				}
-				if packedLen < 0 {
-					return ErrInvalidLengthPipeline
-				}
-				postIndex := iNdEx + packedLen
-				if postIndex < 0 {
-					return ErrInvalidLengthPipeline
-				}
-				if postIndex > l {
-					return io.ErrUnexpectedEOF
-				}
-				var elementCount int
-				var count int
-				for _, integer := range dAtA[iNdEx:postIndex] {
-					if integer < 128 {
-						count++
-					}
-				}
-				elementCount = count
-				if elementCount != 0 && len(m.RelList) == 0 {
-					m.RelList = make([]int32, 0, elementCount)
-				}
-				for iNdEx < postIndex {
-					var v int32
-					for shift := uint(0); ; shift += 7 {
-						if shift >= 64 {
-							return ErrIntOverflowPipeline
-						}
-						if iNdEx >= l {
-							return io.ErrUnexpectedEOF
-						}
-						b := dAtA[iNdEx]
-						iNdEx++
-						v |= int32(b&0x7F) << shift
-						if b < 0x80 {
-							break
-						}
-					}
-					m.RelList = append(m.RelList, v)
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.Nbucket |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
 				}
-			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field RelList", wireType)
 			}
-		case 4:
+		case 3:
 			if wireType == 0 {
 				var v int32
 				for shift := uint(0); ; shift += 7 {
@@ -5030,7 +7324,7 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 						break
 					}
 				}
-				m.ColList = append(m.ColList, v)
+				m.Result = append(m.Result, v)
 			} else if wireType == 2 {
 				var packedLen int
 				for shift := uint(0); ; shift += 7 {
@@ -5065,8 +7359,8 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 					}
 				}
 				elementCount = count
-				if elementCount != 0 && len(m.ColList) == 0 {
-					m.ColList = make([]int32, 0, elementCount)
+				if elementCount != 0 && len(m.Result) == 0 {
+					m.Result = make([]int32, 0, elementCount)
 				}
 				for iNdEx < postIndex {
 					var v int32
@@ -5084,12 +7378,80 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 							break
 						}
 					}
-					m.ColList = append(m.ColList, v)
+					m.Result = append(m.Result, v)
 				}
 			} else {
-				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Result", wireType)
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.LeftCond = append(m.LeftCond, &plan.Expr{})
+			if err := m.LeftCond[len(m.LeftCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
 			}
+			iNdEx = postIndex
 		case 5:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.RightCond = append(m.RightCond, &plan.Expr{})
+			if err := m.RightCond[len(m.RightCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 6:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
 			}
@@ -5123,9 +7485,9 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 6:
+		case 7:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field LeftCond", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Cond", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -5152,14 +7514,16 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.LeftCond = append(m.LeftCond, &plan.Expr{})
-			if err := m.LeftCond[len(m.LeftCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			if m.Cond == nil {
+				m.Cond = &plan.Expr{}
+			}
+			if err := m.Cond.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
-		case 7:
+		case 8:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field RightCond", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field OnList", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -5186,11 +7550,91 @@ func (m *SingleJoin) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			m.RightCond = append(m.RightCond, &plan.Expr{})
-			if err := m.RightCond[len(m.RightCond)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			m.OnList = append(m.OnList, &plan.Expr{})
+			if err := m.OnList[len(m.OnList)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
+		case 9:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field OutputNull", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.OutputNull = bool(v != 0)
+		case 10:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field OutputMark", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.OutputMark = bool(v != 0)
+		case 11:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field OutputAnyway", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.OutputAnyway = bool(v != 0)
+		case 12:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MarkMeaning", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.MarkMeaning = bool(v != 0)
 		default:
 			iNdEx = preIndex
 			skippy, err := skipPipeline(dAtA[iNdEx:])
@@ -5394,6 +7838,40 @@ func (m *Product) Unmarshal(dAtA []byte) error {
 			} else {
 				return fmt.Errorf("proto: wrong wireType = %d for field ColList", wireType)
 			}
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Types", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Types = append(m.Types, &plan.Type{})
+			if err := m.Types[len(m.Types)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
 		default:
 			iNdEx = preIndex
 			skippy, err := skipPipeline(dAtA[iNdEx:])
@@ -5483,9 +7961,45 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 					break
 				}
 			}
-		case 3:
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Anti", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Anti == nil {
+				m.Anti = &AntiJoin{}
+			}
+			if err := m.Anti.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 4:
 			if wireType != 2 {
-				return fmt.Errorf("proto: wrong wireType = %d for field Anti", wireType)
+				return fmt.Errorf("proto: wrong wireType = %d for field Connect", wireType)
 			}
 			var msglen int
 			for shift := uint(0); ; shift += 7 {
@@ -5512,14 +8026,14 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 			if postIndex > l {
 				return io.ErrUnexpectedEOF
 			}
-			if m.Anti == nil {
-				m.Anti = &AntiJoin{}
+			if m.Connect == nil {
+				m.Connect = &Connector{}
 			}
-			if err := m.Anti.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+			if err := m.Connect.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
 				return err
 			}
 			iNdEx = postIndex
-		case 4:
+		case 5:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Dispatch", wireType)
 			}
@@ -5555,7 +8069,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 5:
+		case 6:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Agg", wireType)
 			}
@@ -5591,7 +8105,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 6:
+		case 7:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field InnerJoin", wireType)
 			}
@@ -5627,7 +8141,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 7:
+		case 8:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field LeftJoin", wireType)
 			}
@@ -5663,7 +8177,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 8:
+		case 9:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field SemiJoin", wireType)
 			}
@@ -5699,7 +8213,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 9:
+		case 10:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field SingleJoin", wireType)
 			}
@@ -5735,7 +8249,79 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 10:
+		case 11:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field MarkJoin", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.MarkJoin == nil {
+				m.MarkJoin = &MarkJoin{}
+			}
+			if err := m.MarkJoin.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 12:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Join", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Join == nil {
+				m.Join = &Join{}
+			}
+			if err := m.Join.Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 13:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Product", wireType)
 			}
@@ -5771,7 +8357,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 11:
+		case 14:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field OrderBy", wireType)
 			}
@@ -5805,7 +8391,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 12:
+		case 15:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field ProjectList", wireType)
 			}
@@ -5839,7 +8425,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 13:
+		case 16:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Filter", wireType)
 			}
@@ -5875,7 +8461,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 14:
+		case 17:
 			if wireType != 0 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Limit", wireType)
 			}
@@ -5894,7 +8480,7 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 					break
 				}
 			}
-		case 15:
+		case 18:
 			if wireType != 0 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType)
 			}
@@ -5935,6 +8521,91 @@ func (m *Instruction) Unmarshal(dAtA []byte) error {
 	}
 	return nil
 }
+func (m *AnalysisList) Unmarshal(dAtA []byte) error {
+	l := len(dAtA)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowPipeline
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := dAtA[iNdEx]
+			iNdEx++
+			wire |= uint64(b&0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: AnalysisList: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: AnalysisList: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field List", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				msglen |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + msglen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.List = append(m.List, &plan.AnalyzeInfo{})
+			if err := m.List[len(m.List)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipPipeline(dAtA[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if (skippy < 0) || (iNdEx+skippy) < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.XXX_unrecognized = append(m.XXX_unrecognized, dAtA[iNdEx:iNdEx+skippy]...)
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
 func (m *Source) Unmarshal(dAtA []byte) error {
 	l := len(dAtA)
 	iNdEx := 0
@@ -6092,6 +8763,57 @@ func (m *Source) Unmarshal(dAtA []byte) error {
 			}
 			m.Block = string(dAtA[iNdEx:postIndex])
 			iNdEx = postIndex
+		case 5:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PushdownId", wireType)
+			}
+			m.PushdownId = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.PushdownId |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 6:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field PushdownAddr", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				stringLen |= uint64(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex < 0 {
+				return ErrInvalidLengthPipeline
+			}
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.PushdownAddr = string(dAtA[iNdEx:postIndex])
+			iNdEx = postIndex
 		default:
 			iNdEx = preIndex
 			skippy, err := skipPipeline(dAtA[iNdEx:])
@@ -6508,6 +9230,26 @@ func (m *Pipeline) Unmarshal(dAtA []byte) error {
 			}
 			m.IsEnd = bool(v != 0)
 		case 8:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field IsJoin", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				v |= int(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.IsJoin = bool(v != 0)
+		case 9:
 			if wireType != 2 {
 				return fmt.Errorf("proto: wrong wireType = %d for field Node", wireType)
 			}
@@ -6543,7 +9285,7 @@ func (m *Pipeline) Unmarshal(dAtA []byte) error {
 				return err
 			}
 			iNdEx = postIndex
-		case 9:
+		case 10:
 			if wireType != 0 {
 				return fmt.Errorf("proto: wrong wireType = %d for field PushDownInfo", wireType)
 			}
@@ -6562,6 +9304,25 @@ func (m *Pipeline) Unmarshal(dAtA []byte) error {
 					break
 				}
 			}
+		case 11:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ChildrenCount", wireType)
+			}
+			m.ChildrenCount = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowPipeline
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := dAtA[iNdEx]
+				iNdEx++
+				m.ChildrenCount |= int32(b&0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
 		default:
 			iNdEx = preIndex
 			skippy, err := skipPipeline(dAtA[iNdEx:])
diff --git a/pkg/sql/colexec/join/join_test.go b/pkg/sql/colexec/join/join_test.go
index 3e70c42742c862c608de2123e1be6ed4757aa0da..bf769b924a49516478bfd65c3b5a4921b125700d 100644
--- a/pkg/sql/colexec/join/join_test.go
+++ b/pkg/sql/colexec/join/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -51,7 +52,7 @@ var (
 
 func init() {
 	tcs = []joinTestCase{
-		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}},
+		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0)},
 			[][]*plan.Expr{
 				{
 					newExpr(0, types.Type{Oid: types.T_int8}),
@@ -60,7 +61,7 @@ func init() {
 					newExpr(0, types.Type{Oid: types.T_int8}),
 				},
 			}),
-		newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+		newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 			[][]*plan.Expr{
 				{
 					newExpr(0, types.Type{Oid: types.T_int8}),
@@ -104,7 +105,7 @@ func TestJoin(t *testing.T) {
 func BenchmarkJoin(b *testing.B) {
 	for i := 0; i < b.N; i++ {
 		tcs = []joinTestCase{
-			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -113,7 +114,7 @@ func BenchmarkJoin(b *testing.B) {
 						newExpr(0, types.Type{Oid: types.T_int8}),
 					},
 				}),
-			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -161,7 +162,7 @@ func newExpr(pos int32, typ types.Type) *plan.Expr {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos, cs [][]*plan.Expr) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos, cs [][]*plan.Expr) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/join/types.go b/pkg/sql/colexec/join/types.go
index 1207de1f2aea38d57fb3a9f1907c95d8e119bfac..1bc79b938343847532fb6c5c47c8a417666ba21c 100644
--- a/pkg/sql/colexec/join/types.go
+++ b/pkg/sql/colexec/join/types.go
@@ -19,6 +19,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan"
 )
 
@@ -46,16 +47,11 @@ type container struct {
 	mp *hashmap.JoinMap
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr        *container
 	Ibucket    uint64 // index in buckets
 	Nbucket    uint64 // buckets count
-	Result     []ResultPos
+	Result     []colexec.ResultPos
 	Typs       []types.Type
 	Cond       *plan.Expr
 	Conditions [][]*plan.Expr
diff --git a/pkg/sql/colexec/left/join_test.go b/pkg/sql/colexec/left/join_test.go
index c97d3e207a980337f2b23de6ec82e077ab784ee3..9918ad6e2bd8704c1141da37bce11e8ad76f6e4e 100644
--- a/pkg/sql/colexec/left/join_test.go
+++ b/pkg/sql/colexec/left/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -51,7 +52,7 @@ var (
 
 func init() {
 	tcs = []joinTestCase{
-		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}},
+		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0)},
 			[][]*plan.Expr{
 				{
 					newExpr(0, types.Type{Oid: types.T_int8}),
@@ -60,7 +61,7 @@ func init() {
 					newExpr(0, types.Type{Oid: types.T_int8}),
 				},
 			}),
-		newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+		newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 			[][]*plan.Expr{
 				{
 					newExpr(0, types.Type{Oid: types.T_int8}),
@@ -123,7 +124,7 @@ func TestJoin(t *testing.T) {
 func BenchmarkJoin(b *testing.B) {
 	for i := 0; i < b.N; i++ {
 		tcs = []joinTestCase{
-			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -132,7 +133,7 @@ func BenchmarkJoin(b *testing.B) {
 						newExpr(0, types.Type{Oid: types.T_int8}),
 					},
 				}),
-			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -180,7 +181,7 @@ func newExpr(pos int32, typ types.Type) *plan.Expr {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos, cs [][]*plan.Expr) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos, cs [][]*plan.Expr) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/left/types.go b/pkg/sql/colexec/left/types.go
index 5a400978322918853f4da54d124d1feabd47b628..7a52519c62f78b21606e33557a690b6fcc04e383 100644
--- a/pkg/sql/colexec/left/types.go
+++ b/pkg/sql/colexec/left/types.go
@@ -19,6 +19,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan"
 )
 
@@ -46,16 +47,11 @@ type container struct {
 	mp *hashmap.JoinMap
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr        *container
 	Ibucket    uint64
 	Nbucket    uint64
-	Result     []ResultPos
+	Result     []colexec.ResultPos
 	Typs       []types.Type
 	Cond       *plan.Expr
 	Conditions [][]*plan.Expr
diff --git a/pkg/sql/colexec/loopjoin/join_test.go b/pkg/sql/colexec/loopjoin/join_test.go
index 652d7a77bb98ec4374a66bc8f1b6322f98ab833c..63b99054d78cfe5a96c512b9851d705dc671fec6 100644
--- a/pkg/sql/colexec/loopjoin/join_test.go
+++ b/pkg/sql/colexec/loopjoin/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -55,7 +56,7 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []joinTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 	}
 }
 
@@ -100,8 +101,8 @@ func BenchmarkJoin(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []joinTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -125,7 +126,7 @@ func BenchmarkJoin(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/loopjoin/types.go b/pkg/sql/colexec/loopjoin/types.go
index 4c9c756528fc538566663e1cb3b5a564c6915d81..2f1756896f4b704006092b91c5242b0968cf0360 100644
--- a/pkg/sql/colexec/loopjoin/types.go
+++ b/pkg/sql/colexec/loopjoin/types.go
@@ -18,6 +18,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -31,14 +32,9 @@ type container struct {
 	bat   *batch.Batch
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr    *container
 	Cond   *plan.Expr
-	Result []ResultPos
+	Result []colexec.ResultPos
 	Typs   []types.Type
 }
diff --git a/pkg/sql/colexec/loopleft/join_test.go b/pkg/sql/colexec/loopleft/join_test.go
index 314e9f110466b233ab25738038ac4143ad781829..23962949c7d7100054e1119c0790be528de82cf3 100644
--- a/pkg/sql/colexec/loopleft/join_test.go
+++ b/pkg/sql/colexec/loopleft/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -55,8 +56,8 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []joinTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 	}
 }
 
@@ -120,8 +121,8 @@ func BenchmarkJoin(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []joinTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -145,7 +146,7 @@ func BenchmarkJoin(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/loopleft/types.go b/pkg/sql/colexec/loopleft/types.go
index 1ff1811e4628c5b6eb58e46a15e9e8558ce6dd8c..85aab10d673f040b887c73eb48788becc8bab725 100644
--- a/pkg/sql/colexec/loopleft/types.go
+++ b/pkg/sql/colexec/loopleft/types.go
@@ -18,6 +18,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -31,14 +32,9 @@ type container struct {
 	bat   *batch.Batch
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr    *container
 	Typs   []types.Type
 	Cond   *plan.Expr
-	Result []ResultPos
+	Result []colexec.ResultPos
 }
diff --git a/pkg/sql/colexec/loopsingle/join_test.go b/pkg/sql/colexec/loopsingle/join_test.go
index 40b719995a28be37575ed2c5fcf922a4616bdb24..139ac86bebce4a862b0a87324a0f3f0cb73b1c23 100644
--- a/pkg/sql/colexec/loopsingle/join_test.go
+++ b/pkg/sql/colexec/loopsingle/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -55,8 +56,8 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []joinTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 	}
 }
 
@@ -120,8 +121,8 @@ func BenchmarkJoin(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []joinTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -145,7 +146,7 @@ func BenchmarkJoin(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/loopsingle/types.go b/pkg/sql/colexec/loopsingle/types.go
index 1efcbdb4a99acf94dfa6881682fb693964730265..a20b151068632cba5c16e11e8ea32935bf74a3c6 100644
--- a/pkg/sql/colexec/loopsingle/types.go
+++ b/pkg/sql/colexec/loopsingle/types.go
@@ -18,6 +18,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -31,14 +32,9 @@ type container struct {
 	bat   *batch.Batch
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr    *container
 	Cond   *plan.Expr
-	Result []ResultPos
 	Typs   []types.Type
+	Result []colexec.ResultPos
 }
diff --git a/pkg/sql/colexec/mergeorder/order.go b/pkg/sql/colexec/mergeorder/order.go
index 85e35138a3ea9dc91f6af386477e4ffcfe870e51..ee2866a3d77b22c033c7894693306f9698e3757f 100644
--- a/pkg/sql/colexec/mergeorder/order.go
+++ b/pkg/sql/colexec/mergeorder/order.go
@@ -21,7 +21,6 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/order"
 	"github.com/matrixorigin/matrixone/pkg/vm/process"
 )
 
@@ -128,7 +127,7 @@ func (ctr *container) build(ap *Argument, proc *process.Process, anal process.An
 				ctr.cmps = make([]compare.Compare, len(bat.Vecs))
 				for i := range ctr.cmps {
 					if pos, ok := mp[i]; ok {
-						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == order.Descending)
+						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == colexec.Descending)
 					} else {
 						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, true)
 					}
diff --git a/pkg/sql/colexec/mergeorder/order_test.go b/pkg/sql/colexec/mergeorder/order_test.go
index bad24eb18e544f6ee08f71734dadd64e92044797..4e96e528efd98c3526abf9735546b9ad4ed36932 100644
--- a/pkg/sql/colexec/mergeorder/order_test.go
+++ b/pkg/sql/colexec/mergeorder/order_test.go
@@ -22,7 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/order"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
 	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
 	"github.com/matrixorigin/matrixone/pkg/vm/mmu/guest"
@@ -53,11 +53,11 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []orderTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []order.Field{{E: newExpression(0), Type: 0}}),
-		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []order.Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []bool{false, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []order.Field{{E: newExpression(1), Type: 0}}),
-		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []order.Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []order.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []bool{false, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []colexec.Field{{E: newExpression(1), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []colexec.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
 	}
 }
 
@@ -110,8 +110,8 @@ func BenchmarkOrder(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []orderTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []order.Field{{E: newExpression(0), Type: 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []order.Field{{E: newExpression(0), Type: 2}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 0}}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 2}}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -143,7 +143,7 @@ func BenchmarkOrder(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, ds []bool, ts []types.Type, fs []order.Field) orderTestCase {
+func newTestCase(m *mheap.Mheap, ds []bool, ts []types.Type, fs []colexec.Field) orderTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/mergeorder/types.go b/pkg/sql/colexec/mergeorder/types.go
index e7632b5f04dfb24af91c6191372b83048aa5d3ff..6dc4621b84cbe6c359877e5ae203a9be5ae10aad 100644
--- a/pkg/sql/colexec/mergeorder/types.go
+++ b/pkg/sql/colexec/mergeorder/types.go
@@ -17,7 +17,7 @@ package mergeorder
 import (
 	"github.com/matrixorigin/matrixone/pkg/compare"
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/order"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -36,6 +36,6 @@ type container struct {
 }
 
 type Argument struct {
-	Fs  []order.Field // Fields store the order information
-	ctr *container    // ctr stores the attributes needn't do Serialization work
+	ctr *container      // ctr stores the attributes needn't do Serialization work
+	Fs  []colexec.Field // Fields store the order information
 }
diff --git a/pkg/sql/colexec/mergetop/top.go b/pkg/sql/colexec/mergetop/top.go
index a20dd6a7578ee5a42e60a52fec109cf64cf18bdf..355c8bb13bd771ca7a42b913fb8b630b60493eb1 100644
--- a/pkg/sql/colexec/mergetop/top.go
+++ b/pkg/sql/colexec/mergetop/top.go
@@ -23,7 +23,6 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/top"
 	"github.com/matrixorigin/matrixone/pkg/vm/process"
 )
 
@@ -130,7 +129,7 @@ func (ctr *container) build(ap *Argument, proc *process.Process, anal process.An
 				ctr.cmps = make([]compare.Compare, len(bat.Vecs))
 				for i := range ctr.cmps {
 					if pos, ok := mp[i]; ok {
-						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == top.Descending)
+						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == colexec.Descending)
 					} else {
 						ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, true)
 					}
diff --git a/pkg/sql/colexec/mergetop/top_test.go b/pkg/sql/colexec/mergetop/top_test.go
index 809d3cc04063ba87711b54b38513ba77c64a7a1b..df0cd82cf7fc94c7295a34565bfe80272ea7a643 100644
--- a/pkg/sql/colexec/mergetop/top_test.go
+++ b/pkg/sql/colexec/mergetop/top_test.go
@@ -22,7 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/top"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
 	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
 	"github.com/matrixorigin/matrixone/pkg/vm/mmu/guest"
@@ -53,11 +53,11 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []topTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, 3, []top.Field{{E: newExpression(0), Type: 0}}),
-		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, 3, []top.Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []bool{false, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []top.Field{{E: newExpression(0), Type: 0}}),
-		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []top.Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []top.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []bool{false, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []colexec.Field{{E: newExpression(0), Type: 0}}),
+		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []bool{true, false}, []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
 	}
 
 }
@@ -111,8 +111,8 @@ func BenchmarkTop(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []topTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, 3, []top.Field{{E: newExpression(0), Type: 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, 3, []top.Field{{E: newExpression(0), Type: 2}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 0}}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -144,7 +144,7 @@ func BenchmarkTop(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, ds []bool, ts []types.Type, limit int64, fs []top.Field) topTestCase {
+func newTestCase(m *mheap.Mheap, ds []bool, ts []types.Type, limit int64, fs []colexec.Field) topTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/mergetop/types.go b/pkg/sql/colexec/mergetop/types.go
index b861ebd4c30e530d4829c8b9c36f9e6cfbeca9b0..6205c3e13389d2e2c2b048dd5041d72bf9264ed1 100644
--- a/pkg/sql/colexec/mergetop/types.go
+++ b/pkg/sql/colexec/mergetop/types.go
@@ -17,7 +17,7 @@ package mergetop
 import (
 	"github.com/matrixorigin/matrixone/pkg/compare"
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
-	"github.com/matrixorigin/matrixone/pkg/sql/colexec/top"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -37,9 +37,9 @@ type container struct {
 }
 
 type Argument struct {
-	Fs    []top.Field // Fs store the order information
-	Limit int64       // Limit store the number of mergeTop-operator
-	ctr   *container  // ctr stores the attributes needn't do Serialization work
+	Limit int64           // Limit store the number of mergeTop-operator
+	ctr   *container      // ctr stores the attributes needn't do Serialization work
+	Fs    []colexec.Field // Fs store the order information
 }
 
 func (ctr *container) compare(vi, vj int, i, j int64) int {
diff --git a/pkg/sql/colexec/order/order.go b/pkg/sql/colexec/order/order.go
index 68d1bdb575b7be8ff6b6e5d3e5296cec1576ed9a..08d16086e8489862c307f382644d588c573c1f79 100644
--- a/pkg/sql/colexec/order/order.go
+++ b/pkg/sql/colexec/order/order.go
@@ -44,7 +44,7 @@ func Prepare(_ *process.Process, arg any) error {
 		ap.ctr.ds = make([]bool, len(ap.Fs))
 		ap.ctr.vecs = make([]evalVector, len(ap.Fs))
 		for i, f := range ap.Fs {
-			ap.ctr.ds[i] = f.Type == Descending
+			ap.ctr.ds[i] = f.Type == colexec.Descending
 		}
 	}
 	return nil
diff --git a/pkg/sql/colexec/order/order_test.go b/pkg/sql/colexec/order/order_test.go
index 21181c96679156c98fe3548527bb061a4c32f7c9..a23f88417d7a67a3a3923da3799382c7314b23e3 100644
--- a/pkg/sql/colexec/order/order_test.go
+++ b/pkg/sql/colexec/order/order_test.go
@@ -21,6 +21,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
 	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
 	"github.com/matrixorigin/matrixone/pkg/vm/mmu/guest"
@@ -49,10 +50,10 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []orderTestCase{
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []Field{{E: newExpression(0), Type: 0}}),
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []Field{{E: newExpression(0), Type: 0}, {E: newExpression(1), Type: 0}}),
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 2}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 0}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []colexec.Field{{E: newExpression(0), Type: 0}, {E: newExpression(1), Type: 0}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, []colexec.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 2}}),
 	}
 }
 
@@ -97,8 +98,8 @@ func BenchmarkOrder(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []orderTestCase{
-			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []Field{{E: newExpression(0), Type: 0}}),
-			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []Field{{E: newExpression(0), Type: 2}}),
+			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 0}}),
+			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, []colexec.Field{{E: newExpression(0), Type: 2}}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -122,7 +123,7 @@ func BenchmarkOrder(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, ts []types.Type, fs []Field) orderTestCase {
+func newTestCase(m *mheap.Mheap, ts []types.Type, fs []colexec.Field) orderTestCase {
 	return orderTestCase{
 		types: ts,
 		proc:  process.New(m),
diff --git a/pkg/sql/colexec/order/types.go b/pkg/sql/colexec/order/types.go
index 49133d836ed389b16cd6ad8fb485154024eaef07..bc7f3f89a072bdd9955c6845170cdbcf1d8c8856 100644
--- a/pkg/sql/colexec/order/types.go
+++ b/pkg/sql/colexec/order/types.go
@@ -15,20 +15,8 @@
 package order
 
 import (
-	"fmt"
-
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
-	"github.com/matrixorigin/matrixone/pkg/pb/plan"
-)
-
-// Direction for ordering results.
-type Direction int8
-
-// Direction values.
-const (
-	DefaultDirection Direction = iota
-	Ascending
-	Descending
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 type evalVector struct {
@@ -41,33 +29,7 @@ type Container struct {
 	vecs []evalVector // sorted list of attributes
 }
 
-type Field struct {
-	E    *plan.Expr
-	Type Direction
-}
-
 type Argument struct {
-	Fs  []Field
 	ctr *Container
-}
-
-var directionName = [...]string{
-	DefaultDirection: "",
-	Ascending:        "ASC",
-	Descending:       "DESC",
-}
-
-func (n Field) String() string {
-	s := fmt.Sprintf("%v", n.E)
-	if n.Type != DefaultDirection {
-		s += " " + n.Type.String()
-	}
-	return s
-}
-
-func (i Direction) String() string {
-	if i < 0 || i > Direction(len(directionName)-1) {
-		return fmt.Sprintf("Direction(%d)", i)
-	}
-	return directionName[i]
+	Fs  []colexec.Field
 }
diff --git a/pkg/sql/colexec/product/product_test.go b/pkg/sql/colexec/product/product_test.go
index 79444b2da0956aec83ac2c25e544f20241258e85..00ecdd25977c5e4d9641d2d01e3b5c25cb3c876b 100644
--- a/pkg/sql/colexec/product/product_test.go
+++ b/pkg/sql/colexec/product/product_test.go
@@ -21,6 +21,7 @@ import (
 
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
 	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
@@ -53,8 +54,8 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []productTestCase{
-		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+		newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+		newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 	}
 }
 
@@ -99,8 +100,8 @@ func BenchmarkProduct(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []productTestCase{
-			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
-			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}}),
+			newTestCase(mheap.New(gm), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
+			newTestCase(mheap.New(gm), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -124,7 +125,7 @@ func BenchmarkProduct(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos) productTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos) productTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/product/types.go b/pkg/sql/colexec/product/types.go
index b0e3568bea756ebc9937bc4d7e6b4a9be60aab5d..7529a073100a1e03a1b3bda3b1e8b6102539cd3c 100644
--- a/pkg/sql/colexec/product/types.go
+++ b/pkg/sql/colexec/product/types.go
@@ -17,6 +17,7 @@ package product
 import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -30,13 +31,8 @@ type container struct {
 	bat   *batch.Batch
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr    *container
-	Result []ResultPos
 	Typs   []types.Type
+	Result []colexec.ResultPos
 }
diff --git a/pkg/sql/colexec/single/join_test.go b/pkg/sql/colexec/single/join_test.go
index 9e672a48026ad72c1d4d726caff1be02266277f4..58a41db3f141e3a7a6e543f04baebc6e6a4c225a 100644
--- a/pkg/sql/colexec/single/join_test.go
+++ b/pkg/sql/colexec/single/join_test.go
@@ -22,6 +22,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/colexec/hashbuild"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
@@ -51,7 +52,7 @@ var (
 
 func init() {
 	tcs = []joinTestCase{
-		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}},
+		newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0)},
 			[][]*plan.Expr{
 				{
 					newExpr(0, types.Type{Oid: types.T_int8}),
@@ -113,7 +114,7 @@ func TestJoin(t *testing.T) {
 func BenchmarkJoin(b *testing.B) {
 	for i := 0; i < b.N; i++ {
 		tcs = []joinTestCase{
-			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{false}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -122,7 +123,7 @@ func BenchmarkJoin(b *testing.B) {
 						newExpr(0, types.Type{Oid: types.T_int8}),
 					},
 				}),
-			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []ResultPos{{0, 0}, {1, 0}},
+			newTestCase(testutil.NewMheap(), []bool{true}, []types.Type{{Oid: types.T_int8}}, []colexec.ResultPos{colexec.NewResultPos(0, 0), colexec.NewResultPos(1, 0)},
 				[][]*plan.Expr{
 					{
 						newExpr(0, types.Type{Oid: types.T_int8}),
@@ -170,7 +171,7 @@ func newExpr(pos int32, typ types.Type) *plan.Expr {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []ResultPos, cs [][]*plan.Expr) joinTestCase {
+func newTestCase(m *mheap.Mheap, flgs []bool, ts []types.Type, rp []colexec.ResultPos, cs [][]*plan.Expr) joinTestCase {
 	proc := process.New(m)
 	proc.Reg.MergeReceivers = make([]*process.WaitRegister, 2)
 	ctx, cancel := context.WithCancel(context.Background())
diff --git a/pkg/sql/colexec/single/types.go b/pkg/sql/colexec/single/types.go
index cbcd13ae8c5e2e1ceaa90168cd7c90131d3b7737..15d834868026a4c317a9303c1b743231f48d086a 100644
--- a/pkg/sql/colexec/single/types.go
+++ b/pkg/sql/colexec/single/types.go
@@ -19,6 +19,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/container/vector"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/sql/plan"
 )
 
@@ -46,17 +47,12 @@ type container struct {
 	mp *hashmap.JoinMap
 }
 
-type ResultPos struct {
-	Rel int32
-	Pos int32
-}
-
 type Argument struct {
 	ctr        *container
 	Ibucket    uint64
 	Nbucket    uint64
-	Result     []ResultPos
 	Typs       []types.Type
 	Cond       *plan.Expr
 	Conditions [][]*plan.Expr
+	Result     []colexec.ResultPos
 }
diff --git a/pkg/sql/colexec/top/top.go b/pkg/sql/colexec/top/top.go
index cdfce0bd3bd7a21338cf577eee98d6e8bd0df875..7f27bed6ded4fb7af29c5670dd74b9ecc54c7755 100644
--- a/pkg/sql/colexec/top/top.go
+++ b/pkg/sql/colexec/top/top.go
@@ -116,7 +116,7 @@ func (ctr *Container) build(ap *Argument, bat *batch.Batch, proc *process.Proces
 		ctr.cmps = make([]compare.Compare, len(bat.Vecs))
 		for i := range ctr.cmps {
 			if pos, ok := mp[i]; ok {
-				ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == Descending)
+				ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, ap.Fs[pos].Type == colexec.Descending)
 			} else {
 				ctr.cmps[i] = compare.New(bat.Vecs[i].Typ, true)
 			}
diff --git a/pkg/sql/colexec/top/top_test.go b/pkg/sql/colexec/top/top_test.go
index 1bd316d23edf658d9632ec848f107b330f1b4bbb..46027fe81492a0b2f77ab485214aac20ab8e47a8 100644
--- a/pkg/sql/colexec/top/top_test.go
+++ b/pkg/sql/colexec/top/top_test.go
@@ -21,6 +21,7 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 	"github.com/matrixorigin/matrixone/pkg/testutil"
 	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
 	"github.com/matrixorigin/matrixone/pkg/vm/mmu/guest"
@@ -49,9 +50,9 @@ func init() {
 	hm := host.New(1 << 30)
 	gm := guest.New(1<<30, hm)
 	tcs = []topTestCase{
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []Field{{E: newExpression(0), Type: 0}}),
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []Field{{E: newExpression(0), Type: 2}}),
-		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 0}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}}),
+		newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}, {Oid: types.T_int64}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}, {E: newExpression(1), Type: 0}}),
 	}
 }
 
@@ -95,8 +96,8 @@ func BenchmarkTop(b *testing.B) {
 		hm := host.New(1 << 30)
 		gm := guest.New(1<<30, hm)
 		tcs = []topTestCase{
-			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []Field{{E: newExpression(0), Type: 0}}),
-			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []Field{{E: newExpression(0), Type: 2}}),
+			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 0}}),
+			newTestCase(mheap.New(gm), []types.Type{{Oid: types.T_int8}}, 3, []colexec.Field{{E: newExpression(0), Type: 2}}),
 		}
 		t := new(testing.T)
 		for _, tc := range tcs {
@@ -117,7 +118,7 @@ func BenchmarkTop(b *testing.B) {
 	}
 }
 
-func newTestCase(m *mheap.Mheap, ts []types.Type, limit int64, fs []Field) topTestCase {
+func newTestCase(m *mheap.Mheap, ts []types.Type, limit int64, fs []colexec.Field) topTestCase {
 	return topTestCase{
 		types: ts,
 		proc:  process.New(m),
diff --git a/pkg/sql/colexec/top/types.go b/pkg/sql/colexec/top/types.go
index 895ca91de2d5e41590c6a460c36c750348e5330e..0dcebf890dcd32e87a38480fb21fbacb15d0f0bb 100644
--- a/pkg/sql/colexec/top/types.go
+++ b/pkg/sql/colexec/top/types.go
@@ -15,11 +15,9 @@
 package top
 
 import (
-	"fmt"
-
 	"github.com/matrixorigin/matrixone/pkg/compare"
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
-	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
 )
 
 const (
@@ -28,16 +26,6 @@ const (
 	End
 )
 
-// Direction for ordering results.
-type Direction int8
-
-// Direction values.
-const (
-	DefaultDirection Direction = iota
-	Ascending
-	Descending
-)
-
 type Container struct {
 	n     int // result vector number
 	state int
@@ -48,36 +36,10 @@ type Container struct {
 	bat *batch.Batch
 }
 
-type Field struct {
-	E    *plan.Expr
-	Type Direction
-}
-
 type Argument struct {
 	Limit int64
-	Fs    []Field
 	ctr   *Container
-}
-
-var directionName = [...]string{
-	DefaultDirection: "",
-	Ascending:        "ASC",
-	Descending:       "DESC",
-}
-
-func (n Field) String() string {
-	s := fmt.Sprintf("%v", n.E)
-	if n.Type != DefaultDirection {
-		s += " " + n.Type.String()
-	}
-	return s
-}
-
-func (i Direction) String() string {
-	if i < 0 || i > Direction(len(directionName)-1) {
-		return fmt.Sprintf("Direction(%d)", i)
-	}
-	return directionName[i]
+	Fs    []colexec.Field
 }
 
 func (ctr *Container) compare(vi, vj int, i, j int64) int {
diff --git a/pkg/sql/colexec/types.go b/pkg/sql/colexec/types.go
new file mode 100644
index 0000000000000000000000000000000000000000..5638ceb8aa8f79ed0a1bd713d4ed76571400f674
--- /dev/null
+++ b/pkg/sql/colexec/types.go
@@ -0,0 +1,66 @@
+// Copyright 2021 Matrix Origin
+//
+// Licensed 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 colexec
+
+import (
+	"fmt"
+
+	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+)
+
+type ResultPos struct {
+	Rel int32
+	Pos int32
+}
+
+func NewResultPos(rel int32, pos int32) ResultPos {
+	return ResultPos{Rel: rel, Pos: pos}
+}
+
+// Direction for ordering results.
+type Direction int8
+
+// Direction values.
+const (
+	DefaultDirection Direction = iota
+	Ascending
+	Descending
+)
+
+var directionName = [...]string{
+	DefaultDirection: "",
+	Ascending:        "ASC",
+	Descending:       "DESC",
+}
+
+type Field struct {
+	E    *plan.Expr
+	Type Direction
+}
+
+func (n Field) String() string {
+	s := fmt.Sprintf("%v", n.E)
+	if n.Type != DefaultDirection {
+		s += " " + n.Type.String()
+	}
+	return s
+}
+
+func (i Direction) String() string {
+	if i < 0 || i > Direction(len(directionName)-1) {
+		return fmt.Sprintf("Direction(%d)", i)
+	}
+	return directionName[i]
+}
diff --git a/pkg/sql/compile/cnserviceDriver.go b/pkg/sql/compile/cnserviceDriver.go
new file mode 100644
index 0000000000000000000000000000000000000000..d2f09ec66415300a8b4232c25a311d7bb7b0f347
--- /dev/null
+++ b/pkg/sql/compile/cnserviceDriver.go
@@ -0,0 +1,1049 @@
+// Copyright 2021 Matrix Origin
+//
+// Licensed 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 compile
+
+import (
+	"context"
+	"errors"
+	"github.com/matrixorigin/matrixone/pkg/cnservice/cnclient"
+	"github.com/matrixorigin/matrixone/pkg/common/moerr"
+	"github.com/matrixorigin/matrixone/pkg/common/morpc"
+	"github.com/matrixorigin/matrixone/pkg/container/batch"
+	"github.com/matrixorigin/matrixone/pkg/container/types"
+	"github.com/matrixorigin/matrixone/pkg/pb/pipeline"
+	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/aggregate"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/anti"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/connector"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/dispatch"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/group"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/intersect"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/intersectall"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/join"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/left"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/limit"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/loopanti"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/loopjoin"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/loopleft"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsemi"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/loopsingle"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mark"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/merge"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mergegroup"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mergelimit"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeoffset"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mergeorder"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/mergetop"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/minus"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/offset"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/order"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/product"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/projection"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/restrict"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/semi"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/single"
+	"github.com/matrixorigin/matrixone/pkg/sql/colexec/top"
+	"github.com/matrixorigin/matrixone/pkg/vm"
+	"github.com/matrixorigin/matrixone/pkg/vm/mheap"
+	"github.com/matrixorigin/matrixone/pkg/vm/mmu/guest"
+	"github.com/matrixorigin/matrixone/pkg/vm/mmu/host"
+	"github.com/matrixorigin/matrixone/pkg/vm/process"
+)
+
+// CnServerMessageHandler deal the client message that received at cn-server.
+// the message is always *pipeline.Message here. It's a byte array which encoded by method encodeScope.
+// write back Analysis Information and error info if error occurs to client.
+func CnServerMessageHandler(ctx context.Context, message morpc.Message, cs morpc.ClientSession) error {
+	var errCode []byte = nil
+	// decode message and run it, get final analysis information and err info.
+	analysis, err := pipelineMessageHandle(ctx, message, cs)
+	if err != nil {
+		errCode = []byte(err.Error())
+	}
+	return cs.Write(ctx, &pipeline.Message{Sid: pipeline.MessageEnd, Code: errCode, Analyse: analysis})
+}
+
+func pipelineMessageHandle(ctx context.Context, message morpc.Message, cs morpc.ClientSession) (anaData []byte, err error) {
+	m, ok := message.(*pipeline.Message)
+	if !ok {
+		panic("unexpected message type for cn-server")
+	}
+	c := newCompile(ctx)
+	var s *Scope
+	s, err = decodeScope(m.GetData(), c.proc)
+	if err != nil {
+		return nil, err
+	}
+	// refactor the last operator connect to output
+	refactorScope(ctx, s, cs)
+
+	err = s.Run(c)
+	if err != nil {
+		return nil, err
+	}
+	// get analyse related information
+	if query, ok := s.Plan.Plan.(*plan.Plan_Query); ok {
+		nodes := query.Query.GetNodes()
+		anas := &pipeline.AnalysisList{}
+		anas.List = make([]*plan.AnalyzeInfo, len(nodes))
+		for i := range anas.List {
+			anas.List[i] = nodes[i].AnalyzeInfo
+		}
+		anaData, err = anas.Marshal()
+		if err != nil {
+			return nil, err
+		}
+		return anaData, nil
+	}
+	return nil, nil
+}
+
+// remoteRun sends a scope to a remote node for execution, and wait to receive the back message.
+// the back message is always *pipeline.Message but has three cases.
+// 1. ErrMessage
+// 2. End Message with the result of analysis
+// 3. Batch Message
+func (s *Scope) remoteRun(c *Compile) error {
+	// encode the scope
+	sData, errEncode := encodeScope(s)
+	if errEncode != nil {
+		return errEncode
+	}
+
+	// send encoded message
+	message := &pipeline.Message{Data: sData}
+	r, errSend := cnclient.Client.Send(c.ctx, s.NodeInfo.Addr, message)
+	if errSend != nil {
+		return errSend
+	}
+	defer r.Close()
+
+	// range to receive.
+	arg := s.Instructions[len(s.Instructions)-1].Arg.(*connector.Argument)
+	for {
+		val, errReceive := r.Get()
+		if errReceive != nil {
+			return errReceive
+		}
+		m := val.(*pipeline.Message)
+
+		errMessage := m.GetCode()
+		if len(errMessage) > 0 {
+			return errors.New(string(errMessage))
+		}
+
+		sid := m.GetID()
+		if sid == pipeline.MessageEnd {
+			// get analyse information
+			anaData := m.GetAnalyse()
+			if len(anaData) > 0 {
+				// decode analyse
+				ana := new(pipeline.AnalysisList)
+				err := ana.Unmarshal(anaData)
+				if err != nil {
+					return err
+				}
+				mergeAnalyseInfo(c.anal, ana)
+			}
+			break
+		}
+		// decoded message
+		bat, errBatch := decodeBatch(c.proc, m)
+		if errBatch != nil {
+			return errBatch
+		}
+		sendToConnectOperator(arg, bat)
+	}
+
+	return nil
+}
+
+var _ = new(Scope).remoteRun
+
+// encodeScope generate a pipeline.Pipeline from Scope, encode pipeline, and returns.
+func encodeScope(s *Scope) ([]byte, error) {
+	p, err := fillPipeline(s)
+	if err != nil {
+		return nil, err
+	}
+	return p.Marshal()
+}
+
+// decodeScope decode a pipeline.Pipeline from bytes, and generate a Scope from it.
+func decodeScope(data []byte, proc *process.Process) (*Scope, error) {
+	// unmarshal to pipeline
+	p := &pipeline.Pipeline{}
+	err := p.Unmarshal(data)
+	if err != nil {
+		return nil, err
+	}
+	ctx := &scopeContext{
+		parent: nil,
+		id:     p.PipelineId,
+		regs:   make(map[*process.WaitRegister]int32),
+	}
+	ctx.root = ctx
+	s, err := generateScope(proc, p, ctx, nil)
+	if err != nil {
+		return nil, err
+	}
+	return s, fillInstructionsForScope(s, ctx, p)
+}
+
+func refactorScope(_ context.Context, _ *Scope, _ morpc.ClientSession) {
+	// refactor the scope
+}
+
+// fillPipeline convert the scope to pipeline.Pipeline structure through 2 iterations.
+func fillPipeline(s *Scope) (*pipeline.Pipeline, error) {
+	ctx := &scopeContext{
+		id:     0,
+		parent: nil,
+		regs:   make(map[*process.WaitRegister]int32),
+	}
+	ctx.root = ctx
+	p, ctxId, err := generatePipeline(s, ctx, 1)
+	if err != nil {
+		return nil, err
+	}
+	if _, err := fillInstructionsForPipeline(s, ctx, p, ctxId); err != nil {
+		return nil, err
+	}
+	return p, nil
+}
+
+// generatePipeline generate a base pipeline.Pipeline structure without instructions
+// according to source scope.
+func generatePipeline(s *Scope, ctx *scopeContext, ctxId int32) (*pipeline.Pipeline, int32, error) {
+	var err error
+
+	p := &pipeline.Pipeline{}
+	// Magic and IsEnd
+	p.PipelineType = pipeline.Pipeline_PipelineType(s.Magic)
+	p.PipelineId = ctx.id
+	p.IsEnd = s.IsEnd
+	p.IsJoin = s.IsJoin
+	// Plan
+	p.Qry = s.Plan
+	p.Node = &pipeline.NodeInfo{
+		Id:      s.NodeInfo.Id,
+		Addr:    s.NodeInfo.Addr,
+		Mcpu:    int32(s.NodeInfo.Mcpu),
+		Payload: make([]string, len(s.NodeInfo.Data)),
+	}
+	ctx.pipe = p
+	ctx.scope = s
+	{
+		for i := range s.NodeInfo.Data {
+			p.Node.Payload[i] = string(s.NodeInfo.Data[i])
+		}
+	}
+	p.ChildrenCount = int32(len(s.Proc.Reg.MergeReceivers))
+	{
+		for i := range s.Proc.Reg.MergeReceivers {
+			ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i)
+		}
+	}
+	// DataSource
+	if s.DataSource != nil { // if select 1, DataSource is nil
+		p.DataSource = &pipeline.Source{
+			SchemaName:   s.DataSource.SchemaName,
+			TableName:    s.DataSource.RelationName,
+			ColList:      s.DataSource.Attributes,
+			PushdownId:   s.DataSource.PushdownId,
+			PushdownAddr: s.DataSource.PushdownAddr,
+		}
+		if s.DataSource.Bat != nil {
+			data, err := types.Encode(s.DataSource.Bat)
+			if err != nil {
+				return nil, -1, err
+			}
+			p.DataSource.Block = string(data)
+		}
+	}
+	// PreScope
+	p.Children = make([]*pipeline.Pipeline, len(s.PreScopes))
+	ctx.children = make([]*scopeContext, len(s.PreScopes))
+	for i := range s.PreScopes {
+		ctx.children[i] = &scopeContext{
+			parent: ctx,
+			id:     ctxId,
+			root:   ctx.root,
+			regs:   make(map[*process.WaitRegister]int32),
+		}
+		ctxId++
+		if p.Children[i], ctxId, err = generatePipeline(s.PreScopes[i], ctx.children[i], ctxId); err != nil {
+			return nil, -1, err
+		}
+	}
+	return p, ctxId, nil
+}
+
+// fillInstructionsForPipeline fills pipeline's instructions.
+func fillInstructionsForPipeline(s *Scope, ctx *scopeContext, p *pipeline.Pipeline, ctxId int32) (int32, error) {
+	var err error
+
+	for i := range s.PreScopes {
+		if ctxId, err = fillInstructionsForPipeline(s.PreScopes[i], ctx.children[i], p.Children[i], ctxId); err != nil {
+			return ctxId, err
+		}
+	}
+	// Instructions
+	p.InstructionList = make([]*pipeline.Instruction, len(s.Instructions))
+	for i := range p.InstructionList {
+		if ctxId, p.InstructionList[i], err = convertToPipelineInstruction(&s.Instructions[i], ctx, ctxId); err != nil {
+			return ctxId, err
+		}
+	}
+	return ctxId, nil
+}
+
+// generateScope generate a scope from scope context and pipeline.
+func generateScope(proc *process.Process, p *pipeline.Pipeline, ctx *scopeContext, analNodes []*process.AnalyzeInfo) (*Scope, error) {
+	var err error
+
+	s := &Scope{
+		Magic:  int(p.GetPipelineType()),
+		IsEnd:  p.IsEnd,
+		IsJoin: p.IsJoin,
+		Plan:   p.Qry,
+	}
+	dsc := p.GetDataSource()
+	if dsc != nil {
+		s.DataSource = &Source{
+			SchemaName:   dsc.SchemaName,
+			RelationName: dsc.TableName,
+			Attributes:   dsc.ColList,
+			PushdownId:   dsc.PushdownId,
+			PushdownAddr: dsc.PushdownAddr,
+		}
+		if len(dsc.Block) > 0 {
+			bat := new(batch.Batch)
+			if err := types.Decode([]byte(dsc.Block), bat); err != nil {
+				return nil, err
+			}
+			s.DataSource.Bat = bat
+		}
+	}
+	if p.Node != nil {
+		s.NodeInfo.Id = p.Node.Id
+		s.NodeInfo.Addr = p.Node.Addr
+		s.NodeInfo.Mcpu = int(p.Node.Mcpu)
+		s.NodeInfo.Data = make([][]byte, len(p.Node.Payload))
+		for i := range p.Node.Payload {
+			s.NodeInfo.Data[i] = []byte(p.Node.Payload[i])
+		}
+	}
+	s.Proc = process.NewWithAnalyze(proc, proc.Ctx, int(p.ChildrenCount), analNodes)
+	{
+		for i := range s.Proc.Reg.MergeReceivers {
+			ctx.regs[s.Proc.Reg.MergeReceivers[i]] = int32(i)
+		}
+	}
+	s.PreScopes = make([]*Scope, len(p.Children))
+	ctx.children = make([]*scopeContext, len(s.PreScopes))
+	for i := range s.PreScopes {
+		ctx.children[i] = &scopeContext{
+			parent: ctx,
+			root:   ctx.root,
+			id:     p.Children[i].PipelineId,
+			regs:   make(map[*process.WaitRegister]int32),
+		}
+		if s.PreScopes[i], err = generateScope(s.Proc, p.Children[i], ctx.children[i], analNodes); err != nil {
+			return nil, err
+		}
+	}
+	return s, nil
+}
+
+// fillInstructionsForScope fills scope's instructions.
+func fillInstructionsForScope(s *Scope, ctx *scopeContext, p *pipeline.Pipeline) error {
+	var err error
+
+	for i := range s.PreScopes {
+		if err = fillInstructionsForScope(s.PreScopes[i], ctx.children[i], p.Children[i]); err != nil {
+			return err
+		}
+	}
+	s.Instructions = make([]vm.Instruction, len(p.InstructionList))
+	for i := range s.Instructions {
+		if s.Instructions[i], err = convertToVmInstruction(p.InstructionList[i], ctx); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+// convert vm.Instruction to pipeline.Instruction
+func convertToPipelineInstruction(opr *vm.Instruction, ctx *scopeContext, ctxId int32) (int32, *pipeline.Instruction, error) {
+	var err error
+
+	in := &pipeline.Instruction{Op: int32(opr.Op), Idx: int32(opr.Idx)}
+	switch t := opr.Arg.(type) {
+	case *anti.Argument:
+		in.Anti = &pipeline.AntiJoin{
+			Ibucket:   t.Ibucket,
+			Nbucket:   t.Nbucket,
+			Expr:      t.Cond,
+			Types:     convertToPlanTypes(t.Typs),
+			LeftCond:  t.Conditions[0],
+			RightCond: t.Conditions[1],
+			Result:    t.Result,
+		}
+	case *dispatch.Argument:
+		in.Dispatch = &pipeline.Dispatch{
+			All: t.All,
+		}
+		in.Dispatch.Connector = make([]*pipeline.Connector, len(t.Regs))
+		for i := range t.Regs {
+			idx, ctx0 := ctx.root.findRegister(t.Regs[i])
+			if ctx0.root.isRemote(ctx0, 0) && !ctx0.isDescendant(ctx) {
+				id := srv.RegistConnector(t.Regs[i])
+				if ctxId, err = ctx0.addSubPipeline(id, idx, ctxId); err != nil {
+					return ctxId, nil, err
+				}
+			}
+			in.Dispatch.Connector[i] = &pipeline.Connector{
+				ConnectorIndex: idx,
+				PipelineId:     ctx0.id,
+			}
+		}
+	case *group.Argument:
+		in.Agg = &pipeline.Group{
+			NeedEval: t.NeedEval,
+			Ibucket:  t.Ibucket,
+			Nbucket:  t.Nbucket,
+			Exprs:    t.Exprs,
+			Types:    convertToPlanTypes(t.Types),
+			Aggs:     convertToPipelineAggregates(t.Aggs),
+		}
+	case *join.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.Join = &pipeline.Join{
+			Ibucket:   t.Ibucket,
+			Nbucket:   t.Nbucket,
+			RelList:   relList,
+			ColList:   colList,
+			Expr:      t.Cond,
+			Types:     convertToPlanTypes(t.Typs),
+			LeftCond:  t.Conditions[0],
+			RightCond: t.Conditions[1],
+		}
+	case *left.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.LeftJoin = &pipeline.LeftJoin{
+			Ibucket:   t.Ibucket,
+			Nbucket:   t.Nbucket,
+			RelList:   relList,
+			ColList:   colList,
+			Expr:      t.Cond,
+			Types:     convertToPlanTypes(t.Typs),
+			LeftCond:  t.Conditions[0],
+			RightCond: t.Conditions[1],
+		}
+	case *limit.Argument:
+		in.Limit = t.Limit
+	case *loopanti.Argument:
+		in.Anti = &pipeline.AntiJoin{
+			Result: t.Result,
+			Expr:   t.Cond,
+			Types:  convertToPlanTypes(t.Typs),
+		}
+	case *loopjoin.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.Join = &pipeline.Join{
+			RelList: relList,
+			ColList: colList,
+			Expr:    t.Cond,
+			Types:   convertToPlanTypes(t.Typs),
+		}
+	case *loopleft.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.LeftJoin = &pipeline.LeftJoin{
+			RelList: relList,
+			ColList: colList,
+			Expr:    t.Cond,
+			Types:   convertToPlanTypes(t.Typs),
+		}
+	case *loopsemi.Argument:
+		in.SemiJoin = &pipeline.SemiJoin{
+			Result: t.Result,
+			Expr:   t.Cond,
+			Types:  convertToPlanTypes(t.Typs),
+		}
+	case *loopsingle.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.SingleJoin = &pipeline.SingleJoin{
+			RelList: relList,
+			ColList: colList,
+			Expr:    t.Cond,
+			Types:   convertToPlanTypes(t.Typs),
+		}
+	case *offset.Argument:
+		in.Offset = t.Offset
+	case *order.Argument:
+		in.OrderBy = convertToPlanOrderByList(t.Fs)
+	case *product.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.Product = &pipeline.Product{
+			RelList: relList,
+			ColList: colList,
+			Types:   convertToPlanTypes(t.Typs),
+		}
+	case *projection.Argument:
+		in.ProjectList = t.Es
+	case *restrict.Argument:
+		in.Filter = t.E
+	case *semi.Argument:
+		in.SemiJoin = &pipeline.SemiJoin{
+			Ibucket:   t.Ibucket,
+			Nbucket:   t.Nbucket,
+			Result:    t.Result,
+			Expr:      t.Cond,
+			Types:     convertToPlanTypes(t.Typs),
+			LeftCond:  t.Conditions[0],
+			RightCond: t.Conditions[1],
+		}
+	case *single.Argument:
+		relList, colList := getRelColList(t.Result)
+		in.SingleJoin = &pipeline.SingleJoin{
+			Ibucket:   t.Ibucket,
+			Nbucket:   t.Nbucket,
+			RelList:   relList,
+			ColList:   colList,
+			Expr:      t.Cond,
+			Types:     convertToPlanTypes(t.Typs),
+			LeftCond:  t.Conditions[0],
+			RightCond: t.Conditions[1],
+		}
+	case *top.Argument:
+		in.Limit = uint64(t.Limit)
+		in.OrderBy = convertToPlanOrderByList(t.Fs)
+	// we reused ANTI to store the information here because of the lack of related structure.
+	case *intersect.Argument: // 1
+		in.Anti = &pipeline.AntiJoin{
+			Ibucket: t.IBucket,
+			Nbucket: t.NBucket,
+		}
+	case *minus.Argument: // 2
+		in.Anti = &pipeline.AntiJoin{
+			Ibucket: t.IBucket,
+			Nbucket: t.NBucket,
+		}
+	case *intersectall.Argument:
+		in.Anti = &pipeline.AntiJoin{
+			Ibucket: t.IBucket,
+			Nbucket: t.NBucket,
+		}
+	// may useless.
+	case *merge.Argument:
+	case *mergegroup.Argument:
+		in.Agg = &pipeline.Group{
+			NeedEval: t.NeedEval,
+		}
+	case *mergelimit.Argument:
+		in.Limit = t.Limit
+	case *mergeoffset.Argument:
+		in.Offset = t.Offset
+	case *mergetop.Argument:
+		in.Limit = uint64(t.Limit)
+		in.OrderBy = convertToPlanOrderByList(t.Fs)
+	case *mergeorder.Argument:
+		in.OrderBy = convertToPlanOrderByList(t.Fs)
+	case *connector.Argument:
+		idx, ctx0 := ctx.root.findRegister(t.Reg)
+		if ctx0.root.isRemote(ctx0, 0) && !ctx0.isDescendant(ctx) {
+			id := srv.RegistConnector(t.Reg)
+			if ctxId, err = ctx0.addSubPipeline(id, idx, ctxId); err != nil {
+				return ctxId, nil, err
+			}
+		}
+		in.Connect = &pipeline.Connector{
+			PipelineId:     ctx0.id,
+			ConnectorIndex: idx, // receiver
+		}
+	case *mark.Argument:
+		in.MarkJoin = &pipeline.MarkJoin{
+			Ibucket:      t.Ibucket,
+			Nbucket:      t.Nbucket,
+			Result:       t.Result,
+			LeftCond:     t.Conditions[0],
+			RightCond:    t.Conditions[1],
+			Types:        convertToPlanTypes(t.Typs),
+			Cond:         t.Cond,
+			OnList:       t.OnList,
+			OutputNull:   t.OutputNull,
+			OutputMark:   t.OutputMark,
+			OutputAnyway: t.OutputAnyway,
+			MarkMeaning:  t.MarkMeaning,
+		}
+	default:
+		return -1, nil, moerr.New(moerr.INTERNAL_ERROR, "unexpected operator: %v", opr.Op)
+	}
+	return ctxId, in, nil
+}
+
+// convert pipeline.Instruction to vm.Instruction
+func convertToVmInstruction(opr *pipeline.Instruction, ctx *scopeContext) (vm.Instruction, error) {
+	v := vm.Instruction{Op: int(opr.Op), Idx: int(opr.Idx)}
+	switch opr.Op {
+	case vm.Anti:
+		t := opr.GetAnti()
+		v.Arg = &anti.Argument{
+			Ibucket: t.Ibucket,
+			Nbucket: t.Nbucket,
+			Cond:    t.Expr,
+			Typs:    convertToTypes(t.Types),
+			Conditions: [][]*plan.Expr{
+				t.LeftCond, t.RightCond,
+			},
+			Result: t.Result,
+		}
+	case vm.Dispatch:
+		t := opr.GetDispatch()
+		regs := make([]*process.WaitRegister, len(t.Connector))
+		for i, cp := range t.Connector {
+			regs[i] = ctx.root.getRegister(cp.PipelineId, cp.ConnectorIndex)
+		}
+		v.Arg = &dispatch.Argument{
+			Regs: regs,
+			All:  t.All,
+		}
+	case vm.Group:
+		t := opr.GetAgg()
+		v.Arg = &group.Argument{
+			NeedEval: t.NeedEval,
+			Ibucket:  t.Ibucket,
+			Nbucket:  t.Nbucket,
+			Exprs:    t.Exprs,
+			Types:    convertToTypes(t.Types),
+			Aggs:     convertToAggregates(t.Aggs),
+		}
+	case vm.Join:
+		t := opr.GetJoin()
+		v.Arg = &join.Argument{
+			Ibucket:    t.Ibucket,
+			Nbucket:    t.Nbucket,
+			Cond:       t.Expr,
+			Typs:       convertToTypes(t.Types),
+			Result:     convertToResultPos(t.RelList, t.ColList),
+			Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond},
+		}
+	case vm.Left:
+		t := opr.GetLeftJoin()
+		v.Arg = &left.Argument{
+			Ibucket:    t.Ibucket,
+			Nbucket:    t.Nbucket,
+			Cond:       t.Expr,
+			Typs:       convertToTypes(t.Types),
+			Result:     convertToResultPos(t.RelList, t.ColList),
+			Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond},
+		}
+	case vm.Limit:
+		v.Arg = &limit.Argument{Limit: opr.Limit}
+	case vm.LoopAnti:
+		t := opr.GetAnti()
+		v.Arg = &loopanti.Argument{
+			Result: t.Result,
+			Cond:   t.Expr,
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.LoopJoin:
+		t := opr.GetJoin()
+		v.Arg = &loopjoin.Argument{
+			Result: convertToResultPos(t.RelList, t.ColList),
+			Cond:   t.Expr,
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.LoopLeft:
+		t := opr.GetLeftJoin()
+		v.Arg = &loopleft.Argument{
+			Result: convertToResultPos(t.RelList, t.ColList),
+			Cond:   t.Expr,
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.LoopSemi:
+		t := opr.GetSemiJoin()
+		v.Arg = &loopsemi.Argument{
+			Result: t.Result,
+			Cond:   t.Expr,
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.LoopSingle:
+		t := opr.GetSingleJoin()
+		v.Arg = &loopsingle.Argument{
+			Result: convertToResultPos(t.RelList, t.ColList),
+			Cond:   t.Expr,
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.Offset:
+		v.Arg = &offset.Argument{Offset: opr.Offset}
+	case vm.Order:
+		v.Arg = &order.Argument{Fs: convertToColExecField(opr.OrderBy)}
+	case vm.Product:
+		t := opr.GetProduct()
+		v.Arg = &product.Argument{
+			Result: convertToResultPos(t.RelList, t.ColList),
+			Typs:   convertToTypes(t.Types),
+		}
+	case vm.Projection:
+		v.Arg = &projection.Argument{Es: opr.ProjectList}
+	case vm.Restrict:
+		v.Arg = &restrict.Argument{E: opr.Filter}
+	case vm.Semi:
+		t := opr.GetSemiJoin()
+		v.Arg = &semi.Argument{
+			Ibucket:    t.Ibucket,
+			Nbucket:    t.Nbucket,
+			Result:     t.Result,
+			Cond:       t.Expr,
+			Typs:       convertToTypes(t.Types),
+			Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond},
+		}
+	case vm.Single:
+		t := opr.GetSingleJoin()
+		v.Arg = &single.Argument{
+			Ibucket:    t.Ibucket,
+			Nbucket:    t.Nbucket,
+			Result:     convertToResultPos(t.RelList, t.ColList),
+			Cond:       t.Expr,
+			Typs:       convertToTypes(t.Types),
+			Conditions: [][]*plan.Expr{t.LeftCond, t.RightCond},
+		}
+	case vm.Mark:
+		t := opr.GetMarkJoin()
+		v.Arg = &mark.Argument{
+			Ibucket:      t.Ibucket,
+			Nbucket:      t.Nbucket,
+			Result:       t.Result,
+			Conditions:   [][]*plan.Expr{t.LeftCond, t.RightCond},
+			Typs:         convertToTypes(t.Types),
+			Cond:         t.Cond,
+			OnList:       t.OnList,
+			OutputNull:   t.OutputNull,
+			OutputMark:   t.OutputMark,
+			OutputAnyway: t.OutputAnyway,
+			MarkMeaning:  t.MarkMeaning,
+		}
+	case vm.Top:
+		v.Arg = &top.Argument{
+			Limit: int64(opr.Limit),
+			Fs:    convertToColExecField(opr.OrderBy),
+		}
+	// should change next day?
+	case vm.Intersect:
+		t := opr.GetAnti()
+		v.Arg = &intersect.Argument{
+			IBucket: t.Ibucket,
+			NBucket: t.Nbucket,
+		}
+	case vm.IntersectAll:
+		t := opr.GetAnti()
+		v.Arg = &intersectall.Argument{
+			IBucket: t.Ibucket,
+			NBucket: t.Nbucket,
+		}
+	case vm.Minus:
+		t := opr.GetAnti()
+		v.Arg = &minus.Argument{
+			IBucket: t.Ibucket,
+			NBucket: t.Nbucket,
+		}
+	case vm.Connector:
+		t := opr.GetConnect()
+		v.Arg = &connector.Argument{
+			Reg: ctx.root.getRegister(t.PipelineId, t.ConnectorIndex),
+		}
+	// may useless
+	case vm.Merge:
+		v.Arg = &merge.Argument{}
+	case vm.MergeGroup:
+		v.Arg = &mergegroup.Argument{
+			NeedEval: opr.Agg.NeedEval,
+		}
+	case vm.MergeLimit:
+		v.Arg = &mergelimit.Argument{
+			Limit: opr.Limit,
+		}
+	case vm.MergeOffset:
+		v.Arg = &mergeoffset.Argument{
+			Offset: opr.Offset,
+		}
+	case vm.MergeTop:
+		v.Arg = &mergetop.Argument{
+			Limit: int64(opr.Limit),
+			Fs:    convertToColExecField(opr.OrderBy),
+		}
+	case vm.MergeOrder:
+		v.Arg = &mergeorder.Argument{
+			Fs: convertToColExecField(opr.OrderBy),
+		}
+	default:
+		return v, moerr.New(moerr.INTERNAL_ERROR, "unexpected operator: %v", opr.Op)
+	}
+	return v, nil
+}
+
+// newCompile init a new compile for remote run.
+func newCompile(ctx context.Context) *Compile {
+	// not implement now, fill method should send by stream
+	c := &Compile{
+		ctx:  ctx,
+		proc: process.New(mheap.New(guest.New(1<<30, host.New(1<<20)))),
+	}
+	//c.fill = func(a any, b *batch.Batch) error {
+	//	stream, err := CNClient.NewStream("target")
+	//	if err != nil {
+	//		return err
+	//	}
+	//	stream.Send()
+	//}
+	return c
+}
+
+func mergeAnalyseInfo(target *anaylze, ana *pipeline.AnalysisList) {
+	source := ana.List
+	if len(target.analInfos) != len(source) {
+		return
+	}
+	for i := range target.analInfos {
+		n := source[i]
+		target.analInfos[i].OutputSize += n.OutputSize
+		target.analInfos[i].OutputRows += n.OutputRows
+		target.analInfos[i].InputRows += n.InputRows
+		target.analInfos[i].InputSize += n.InputSize
+		target.analInfos[i].MemorySize += n.MemorySize
+		target.analInfos[i].TimeConsumed += n.TimeConsumed
+	}
+}
+
+// convert []types.Type to []*plan.Type
+func convertToPlanTypes(ts []types.Type) []*plan.Type {
+	result := make([]*plan.Type, len(ts))
+	for i, t := range ts {
+		result[i] = &plan.Type{
+			Id:        int32(t.Oid),
+			Width:     t.Width,
+			Precision: t.Precision,
+			Size:      t.Size,
+			Scale:     t.Scale,
+		}
+	}
+	return result
+}
+
+// convert []*plan.Type to []types.Type
+func convertToTypes(ts []*plan.Type) []types.Type {
+	result := make([]types.Type, len(ts))
+	for i, t := range ts {
+		result[i] = types.Type{
+			Oid:       types.T(t.Id),
+			Width:     t.Width,
+			Precision: t.Precision,
+			Size:      t.Size,
+			Scale:     t.Scale,
+		}
+	}
+	return result
+}
+
+// convert []aggregate.Aggregate to []*pipeline.Aggregate
+func convertToPipelineAggregates(ags []aggregate.Aggregate) []*pipeline.Aggregate {
+	result := make([]*pipeline.Aggregate, len(ags))
+	for i, a := range ags {
+		result[i] = &pipeline.Aggregate{
+			Op:   int32(a.Op),
+			Dist: a.Dist,
+			Expr: a.E,
+		}
+	}
+	return result
+}
+
+// convert []*pipeline.Aggregate to []aggregate.Aggregate
+func convertToAggregates(ags []*pipeline.Aggregate) []aggregate.Aggregate {
+	result := make([]aggregate.Aggregate, len(ags))
+	for i, a := range ags {
+		result[i] = aggregate.Aggregate{
+			Op:   int(a.Op),
+			Dist: a.Dist,
+			E:    a.Expr,
+		}
+	}
+	return result
+}
+
+// convert []colexec.Field to []*plan.OrderBySpec
+func convertToPlanOrderByList(field []colexec.Field) []*plan.OrderBySpec {
+	// default order direction is ASC.
+	convToPlanOrderFlag := func(source colexec.Direction) plan.OrderBySpec_OrderByFlag {
+		if source == colexec.Descending {
+			return plan.OrderBySpec_DESC
+		}
+		return plan.OrderBySpec_ASC
+	}
+
+	res := make([]*plan.OrderBySpec, len(field))
+	for i, f := range field {
+		res[i] = &plan.OrderBySpec{
+			Expr: f.E,
+			Flag: convToPlanOrderFlag(f.Type),
+		}
+	}
+	return res
+}
+
+// convert []*plan.OrderBySpec to []colexec.Field
+func convertToColExecField(list []*plan.OrderBySpec) []colexec.Field {
+	convToColExecDirection := func(source plan.OrderBySpec_OrderByFlag) colexec.Direction {
+		if source == plan.OrderBySpec_ASC {
+			return colexec.Ascending
+		}
+		return colexec.Descending
+	}
+
+	res := make([]colexec.Field, len(list))
+	for i, l := range list {
+		res[i].E, res[i].Type = l.Expr, convToColExecDirection(l.Flag)
+	}
+	return res
+}
+
+// get relation list and column list from []colexec.ResultPos
+func getRelColList(resultPos []colexec.ResultPos) (relList []int32, colList []int32) {
+	relList = make([]int32, len(resultPos))
+	colList = make([]int32, len(resultPos))
+	for i := range resultPos {
+		relList[i], colList[i] = resultPos[i].Rel, resultPos[i].Pos
+	}
+	return
+}
+
+// generate []colexec.ResultPos from relation list and column list
+func convertToResultPos(relList, colList []int32) []colexec.ResultPos {
+	res := make([]colexec.ResultPos, len(relList))
+	for i := range res {
+		res[i].Rel, res[i].Pos = relList[i], colList[i]
+	}
+	return res
+}
+
+func decodeBatch(_ *process.Process, msg *pipeline.Message) (*batch.Batch, error) {
+	bat := new(batch.Batch) // TODO: allocate the memory from process may suitable.
+	err := types.Decode(msg.GetData(), bat)
+	return bat, err
+}
+
+func sendToConnectOperator(arg *connector.Argument, bat *batch.Batch) {
+	select {
+	case <-arg.Reg.Ctx.Done():
+	case arg.Reg.Ch <- bat:
+	}
+}
+
+func (ctx *scopeContext) getRegister(id, idx int32) *process.WaitRegister {
+	if ctx.id == id {
+		for k, v := range ctx.regs {
+			if v == idx {
+				return k
+			}
+		}
+	}
+	for i := range ctx.children {
+		if reg := ctx.children[i].getRegister(id, idx); reg != nil {
+			return reg
+		}
+	}
+	return nil
+}
+
+func (ctx *scopeContext) findRegister(reg *process.WaitRegister) (int32, *scopeContext) {
+	if idx, ok := ctx.regs[reg]; ok {
+		return idx, ctx
+	}
+	for i := range ctx.children {
+		if idx, ctx := ctx.children[i].findRegister(reg); idx >= 0 {
+			return idx, ctx
+		}
+	}
+	return -1, nil
+}
+
+func (ctx *scopeContext) addSubPipeline(id uint64, idx int32, ctxId int32) (int32, error) {
+	ds := &Scope{Magic: Pushdown}
+	ds.Proc = process.NewWithAnalyze(ctx.scope.Proc, ctx.scope.Proc.Ctx, 0, nil)
+	ds.DataSource = &Source{
+		PushdownId:   id,
+		PushdownAddr: cnAddr,
+	}
+	ds.appendInstruction(vm.Instruction{
+		Op: vm.Connector,
+		Arg: &connector.Argument{
+			Reg: ctx.scope.Proc.Reg.MergeReceivers[idx],
+		},
+	})
+	ctx.scope.PreScopes = append(ctx.scope.PreScopes, ds)
+	p := &pipeline.Pipeline{}
+	p.PipelineId = ctxId
+	p.PipelineType = Pushdown
+	ctxId++
+	p.DataSource = &pipeline.Source{
+		PushdownId:   id,
+		PushdownAddr: cnAddr,
+	}
+	p.InstructionList = append(p.InstructionList, &pipeline.Instruction{
+		Op: vm.Connector,
+		Connect: &pipeline.Connector{
+			ConnectorIndex: idx,
+			PipelineId:     ctx.id,
+		},
+	})
+	ctx.pipe.Children = append(ctx.pipe.Children, p)
+	return ctxId, nil
+}
+
+func (ctx *scopeContext) isDescendant(dsc *scopeContext) bool {
+	if ctx.id == dsc.id {
+		return true
+	}
+	for i := range ctx.children {
+		if ctx.children[i].isDescendant(dsc) {
+			return true
+		}
+	}
+	return false
+}
+
+func (ctx *scopeContext) isRemote(targetContext *scopeContext, depth int) bool {
+	if targetContext.scope.Magic != Remote {
+		return false
+	}
+	if ctx.id == targetContext.id && depth == 0 {
+		return true
+	}
+	for i := range ctx.children {
+		if ctx.children[i].scope.Magic == Remote {
+			if ctx.children[i].isRemote(targetContext, depth+1) {
+				return true
+			}
+		} else {
+			if ctx.children[i].isRemote(targetContext, depth) {
+				return true
+			}
+		}
+	}
+	return false
+}
diff --git a/pkg/sql/compile/compile.go b/pkg/sql/compile/compile.go
index 64a97e240483268a5c5a437c16dc9598a4f86de6..0896c875c422042ca3df5b45742f0f64b9f73885 100644
--- a/pkg/sql/compile/compile.go
+++ b/pkg/sql/compile/compile.go
@@ -88,7 +88,7 @@ func (c *Compile) Run(_ uint64) (err error) {
 		return nil
 	}
 
-	//PrintScope(nil, []*Scope{c.scope})
+	PrintScope(nil, []*Scope{c.scope})
 
 	switch c.scope.Magic {
 	case Normal:
@@ -553,33 +553,31 @@ func (c *Compile) compileProjection(n *plan.Node, ss []*Scope) []*Scope {
 
 func (c *Compile) compileUnion(n *plan.Node, ss []*Scope, children []*Scope, ns []*plan.Node) []*Scope {
 	ss = append(ss, children...)
-	rs := c.newScopeList(validScopeCount(ss))
-	j := 0
-	for i := range ss {
-		if !ss[i].IsEnd {
-			ss[i].appendInstruction(vm.Instruction{
-				Op:  vm.Dispatch,
-				Arg: constructDispatch(true, extraRegisters(rs, j)),
-			})
-			j++
-			ss[i].IsEnd = true
-		}
-	}
+	rs := c.newScopeList(1)
 	gn := new(plan.Node)
 	gn.GroupBy = make([]*plan.Expr, len(n.ProjectList))
 	copy(gn.GroupBy, n.ProjectList)
 	for i := range rs {
+		ch := c.newMergeScope(dupScopeList(ss))
+		ch.appendInstruction(vm.Instruction{
+			Op: vm.Connector,
+			Arg: &connector.Argument{
+				Reg: rs[i].Proc.Reg.MergeReceivers[0],
+			},
+		})
+		ch.IsEnd = true
+		rs[i].PreScopes = []*Scope{ch}
 		rs[i].Instructions = append(rs[i].Instructions, vm.Instruction{
 			Op:  vm.Group,
 			Idx: c.anal.curr,
 			Arg: constructGroup(gn, n, i, len(rs), true),
 		})
 	}
-	return []*Scope{c.newMergeScope(append(rs, ss...))}
+	return rs
 }
 
 func (c *Compile) compileMinusAndIntersect(n *plan.Node, ss []*Scope, children []*Scope, nodeType plan.Node_NodeType) []*Scope {
-	rs, left, right := c.newJoinScopeListWithBucket(c.newScopeList(2), ss, children)
+	rs := c.newJoinScopeListWithBucket(c.newScopeList(2), ss, children)
 	switch nodeType {
 	case plan.Node_MINUS:
 		for i := range rs {
@@ -607,8 +605,7 @@ func (c *Compile) compileMinusAndIntersect(n *plan.Node, ss []*Scope, children [
 		}
 
 	}
-
-	return []*Scope{c.newMergeScope(append(append(rs, left), right))}
+	return rs
 }
 
 func (c *Compile) compileUnionAll(n *plan.Node, ss []*Scope, children []*Scope) []*Scope {
@@ -702,20 +699,12 @@ func (c *Compile) compileJoin(n, right *plan.Node, ss []*Scope, children []*Scop
 	case plan.Node_ANTI:
 		_, conds := extraJoinConditions(n.OnList)
 		for i := range rs {
-			if isEq {
-				if len(conds) > 1 {
-					rs[i].appendInstruction(vm.Instruction{
-						Op:  vm.Mark,
-						Idx: c.anal.curr,
-						Arg: constructMark(n, typs, c.proc, n.OnList),
-					})
-				} else {
-					rs[i].appendInstruction(vm.Instruction{
-						Op:  vm.Anti,
-						Idx: c.anal.curr,
-						Arg: constructAnti(n, typs, c.proc),
-					})
-				}
+			if isEq && len(conds) == 1 {
+				rs[i].appendInstruction(vm.Instruction{
+					Op:  vm.Anti,
+					Idx: c.anal.curr,
+					Arg: constructAnti(n, typs, c.proc),
+				})
 			} else {
 				rs[i].appendInstruction(vm.Instruction{
 					Op:  vm.LoopAnti,
@@ -907,22 +896,27 @@ func (c *Compile) newScopeListWithNode(mcpu, childrenCount int) []*Scope {
 	return ss
 }
 
-func (c *Compile) newJoinScopeListWithBucket(rs, ss, children []*Scope) ([]*Scope, *Scope, *Scope) {
-	left := c.newMergeScope(ss)
-	right := c.newMergeScope(children)
-	leftRegs := extraRegisters(rs, 0)
-	left.appendInstruction(vm.Instruction{
-		Op:  vm.Dispatch,
-		Arg: constructDispatch(true, leftRegs),
-	})
-	rightRegs := extraRegisters(rs, 1)
-	right.appendInstruction(vm.Instruction{
-		Op:  vm.Dispatch,
-		Arg: constructDispatch(true, rightRegs),
-	})
-	left.IsEnd = true
-	right.IsEnd = true
-	return rs, left, right
+func (c *Compile) newJoinScopeListWithBucket(rs, ss, children []*Scope) []*Scope {
+	for i := range rs {
+		left := c.newMergeScope(dupScopeList(ss))
+		right := c.newMergeScope(dupScopeList(children))
+		rs[i].PreScopes = []*Scope{left, right}
+		left.appendInstruction(vm.Instruction{
+			Op: vm.Connector,
+			Arg: &connector.Argument{
+				Reg: rs[i].Proc.Reg.MergeReceivers[0],
+			},
+		})
+		right.appendInstruction(vm.Instruction{
+			Op: vm.Connector,
+			Arg: &connector.Argument{
+				Reg: rs[i].Proc.Reg.MergeReceivers[1],
+			},
+		})
+		left.IsEnd = true
+		right.IsEnd = true
+	}
+	return rs
 }
 
 func (c *Compile) newJoinScopeList(ss []*Scope, children []*Scope) []*Scope {
@@ -932,16 +926,13 @@ func (c *Compile) newJoinScopeList(ss []*Scope, children []*Scope) []*Scope {
 			rs[i] = ss[i]
 			continue
 		}
-		chp := c.newMergeScope(children)
+		chp := c.newMergeScope(dupScopeList(children))
 		rs[i] = new(Scope)
 		rs[i].Magic = Remote
 		rs[i].IsJoin = true
 		rs[i].NodeInfo = ss[i].NodeInfo
 		rs[i].PreScopes = []*Scope{ss[i], chp}
 		rs[i].Proc = process.NewWithAnalyze(c.proc, c.ctx, 2, c.anal.Nodes())
-		if ss[i].Magic == Remote {
-			ss[i].Magic = Parallel
-		}
 		ss[i].appendInstruction(vm.Instruction{
 			Op: vm.Connector,
 			Arg: &connector.Argument{
diff --git a/pkg/sql/compile/operator.go b/pkg/sql/compile/operator.go
index c699843e2ec577cea2b2a8f6523d7aee1e43e8f5..6858bc1e754dec81e138695ce4d7f9c5e58a84f5 100644
--- a/pkg/sql/compile/operator.go
+++ b/pkg/sql/compile/operator.go
@@ -344,11 +344,11 @@ func constructTop(n *plan.Node, proc *process.Process) *top.Argument {
 	if err != nil {
 		panic(err)
 	}
-	fs := make([]top.Field, len(n.OrderBy))
+	fs := make([]colexec.Field, len(n.OrderBy))
 	for i, e := range n.OrderBy {
 		fs[i].E = e.Expr
 		if e.Flag == plan.OrderBySpec_DESC {
-			fs[i].Type = top.Descending
+			fs[i].Type = colexec.Descending
 		}
 	}
 	return &top.Argument{
@@ -358,7 +358,7 @@ func constructTop(n *plan.Node, proc *process.Process) *top.Argument {
 }
 
 func constructJoin(n *plan.Node, typs []types.Type, proc *process.Process) *join.Argument {
-	result := make([]join.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -390,7 +390,7 @@ func constructSemi(n *plan.Node, typs []types.Type, proc *process.Process) *semi
 }
 
 func constructLeft(n *plan.Node, typs []types.Type, proc *process.Process) *left.Argument {
-	result := make([]left.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -404,7 +404,7 @@ func constructLeft(n *plan.Node, typs []types.Type, proc *process.Process) *left
 }
 
 func constructSingle(n *plan.Node, typs []types.Type, proc *process.Process) *single.Argument {
-	result := make([]single.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -418,7 +418,7 @@ func constructSingle(n *plan.Node, typs []types.Type, proc *process.Process) *si
 }
 
 func constructProduct(n *plan.Node, typs []types.Type, proc *process.Process) *product.Argument {
-	result := make([]product.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -466,12 +466,14 @@ func constructMark(n *plan.Node, typs []types.Type, proc *process.Process, onLis
 	}
 }
 
+var _ = constructMark
+
 func constructOrder(n *plan.Node, proc *process.Process) *order.Argument {
-	fs := make([]order.Field, len(n.OrderBy))
+	fs := make([]colexec.Field, len(n.OrderBy))
 	for i, e := range n.OrderBy {
 		fs[i].E = e.Expr
 		if e.Flag == plan.OrderBySpec_DESC {
-			fs[i].Type = order.Descending
+			fs[i].Type = colexec.Descending
 		}
 	}
 	return &order.Argument{
@@ -578,11 +580,11 @@ func constructMergeTop(n *plan.Node, proc *process.Process) *mergetop.Argument {
 	if err != nil {
 		panic(err)
 	}
-	fs := make([]top.Field, len(n.OrderBy))
+	fs := make([]colexec.Field, len(n.OrderBy))
 	for i, e := range n.OrderBy {
 		fs[i].E = e.Expr
 		if e.Flag == plan.OrderBySpec_DESC {
-			fs[i].Type = top.Descending
+			fs[i].Type = colexec.Descending
 		}
 	}
 	return &mergetop.Argument{
@@ -612,11 +614,11 @@ func constructMergeLimit(n *plan.Node, proc *process.Process) *mergelimit.Argume
 }
 
 func constructMergeOrder(n *plan.Node, proc *process.Process) *mergeorder.Argument {
-	fs := make([]order.Field, len(n.OrderBy))
+	fs := make([]colexec.Field, len(n.OrderBy))
 	for i, e := range n.OrderBy {
 		fs[i].E = e.Expr
 		if e.Flag == plan.OrderBySpec_DESC {
-			fs[i].Type = order.Descending
+			fs[i].Type = colexec.Descending
 		}
 	}
 	return &mergeorder.Argument{
@@ -625,7 +627,7 @@ func constructMergeOrder(n *plan.Node, proc *process.Process) *mergeorder.Argume
 }
 
 func constructLoopJoin(n *plan.Node, typs []types.Type, proc *process.Process) *loopjoin.Argument {
-	result := make([]loopjoin.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -653,7 +655,7 @@ func constructLoopSemi(n *plan.Node, typs []types.Type, proc *process.Process) *
 }
 
 func constructLoopLeft(n *plan.Node, typs []types.Type, proc *process.Process) *loopleft.Argument {
-	result := make([]loopleft.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
@@ -665,7 +667,7 @@ func constructLoopLeft(n *plan.Node, typs []types.Type, proc *process.Process) *
 }
 
 func constructLoopSingle(n *plan.Node, typs []types.Type, proc *process.Process) *loopsingle.Argument {
-	result := make([]loopsingle.ResultPos, len(n.ProjectList))
+	result := make([]colexec.ResultPos, len(n.ProjectList))
 	for i, expr := range n.ProjectList {
 		result[i].Rel, result[i].Pos = constructJoinResult(expr)
 	}
diff --git a/pkg/sql/compile/scope.go b/pkg/sql/compile/scope.go
index 45f486f56e320d28e77129f808bb851861e0f5fc..7b5a7f20584adcf8d0f69f6a6fecabba7422b1ab 100644
--- a/pkg/sql/compile/scope.go
+++ b/pkg/sql/compile/scope.go
@@ -37,12 +37,9 @@ import (
 	"github.com/matrixorigin/matrixone/pkg/vm/process"
 )
 
-// PrintScope Print is to format scope list
 func PrintScope(prefix []byte, ss []*Scope) {
 	for _, s := range ss {
-		if s.Magic == Merge || s.Magic == Remote {
-			PrintScope(append(prefix, '\t'), s.PreScopes)
-		}
+		PrintScope(append(prefix, '\t'), s.PreScopes)
 		p := pipeline.NewMerge(s.Instructions, nil)
 		logutil.Infof("%s:%v %v", prefix, s.Magic, p)
 	}
@@ -100,13 +97,20 @@ func (s *Scope) MergeRun(c *Compile) error {
 				}()
 				err = cs.ParallelRun(c)
 			}(s.PreScopes[i])
+		case Pushdown:
+			go func(cs *Scope) {
+				var err error
+				defer func() {
+					errChan <- err
+				}()
+				err = cs.PushdownRun(c)
+			}(s.PreScopes[i])
 		}
 	}
 	p := pipeline.NewMerge(s.Instructions, s.Reg)
 	if _, err := p.MergeRun(s.Proc); err != nil {
 		return err
 	}
-
 	// check sub-goroutine's error
 	for i := 0; i < len(s.PreScopes); i++ {
 		if err := <-errChan; err != nil {
@@ -118,6 +122,32 @@ func (s *Scope) MergeRun(c *Compile) error {
 
 // RemoteRun send the scope to a remote node (if target node is itself, it is same to function ParallelRun) and run it.
 func (s *Scope) RemoteRun(c *Compile) error {
+	// if address itself, just run it parallel at local.
+	//	return s.ParallelRun(c)
+	/*
+		if len(s.NodeInfo.Addr) == 0 {
+			return s.ParallelRun(c)
+		}
+		err := s.remoteRun(c)
+		// tell to connect operator that it's over
+		arg := s.Instructions[len(s.Instructions)-1].Arg.(*connector.Argument)
+		sendToConnectOperator(arg, nil)
+	*/
+	/*
+		// just for test serialization codes.
+		n := len(s.Instructions) - 1
+		in := s.Instructions[n]
+		s.Instructions = s.Instructions[:n]
+		data, err := encodeScope(s)
+		if err != nil {
+			return err
+		}
+		rs, err := decodeScope(data, s.Proc)
+		if err != nil {
+			return err
+		}
+		rs.Instructions = append(rs.Instructions, in)
+	*/
 	return s.ParallelRun(c)
 }
 
@@ -160,6 +190,29 @@ func (s *Scope) ParallelRun(c *Compile) error {
 	return s.MergeRun(c)
 }
 
+func (s *Scope) PushdownRun(c *Compile) error {
+	var end bool // exist flag
+	var err error
+
+	reg := srv.GetConnector(s.DataSource.PushdownId)
+	for {
+		bat := <-reg.Ch
+		if bat == nil {
+			s.Proc.Reg.InputBatch = bat
+			_, err = vm.Run(s.Instructions, s.Proc)
+			s.Proc.Cancel()
+			return err
+		}
+		if bat.Length() == 0 {
+			continue
+		}
+		s.Proc.Reg.InputBatch = bat
+		if end, err = vm.Run(s.Instructions, s.Proc); err != nil || end {
+			return err
+		}
+	}
+}
+
 func (s *Scope) JoinRun(c *Compile) error {
 	mcpu := s.NodeInfo.Mcpu
 	if mcpu < 1 {
@@ -348,3 +401,23 @@ func (s *Scope) appendInstruction(in vm.Instruction) {
 		s.Instructions = append(s.Instructions, in)
 	}
 }
+
+func dupScope(s *Scope) *Scope {
+	data, err := encodeScope(s)
+	if err != nil {
+		return nil
+	}
+	rs, err := decodeScope(data, s.Proc)
+	if err != nil {
+		return nil
+	}
+	return rs
+}
+
+func dupScopeList(ss []*Scope) []*Scope {
+	rs := make([]*Scope, len(ss))
+	for i := range rs {
+		rs[i] = dupScope(ss[i])
+	}
+	return rs
+}
diff --git a/pkg/sql/compile/scope_test.go b/pkg/sql/compile/scope_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..9cf75b1d00ddba5846f13ebc4265463db4da17d8
--- /dev/null
+++ b/pkg/sql/compile/scope_test.go
@@ -0,0 +1,95 @@
+// Copyright 2021 Matrix Origin
+//
+// Licensed 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 compile
+
+import (
+	"context"
+	"fmt"
+	"github.com/matrixorigin/matrixone/pkg/container/batch"
+	"github.com/matrixorigin/matrixone/pkg/pb/plan"
+	"github.com/matrixorigin/matrixone/pkg/sql/parsers/dialect/mysql"
+	plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan"
+	"github.com/matrixorigin/matrixone/pkg/testutil"
+	"github.com/matrixorigin/matrixone/pkg/vm"
+	"github.com/matrixorigin/matrixone/pkg/vm/engine/memEngine"
+	"github.com/stretchr/testify/require"
+	"testing"
+)
+
+func TestScopeSerialization(t *testing.T) {
+	testCases := []string{
+		"select 1",
+		"select * from R",
+		"select count(*) from R",
+		"select * from R limit 2, 1",
+		"select * from R left join S on R.uid = S.uid",
+	}
+
+	var sourceScopes = generateScopeCases(t, testCases)
+
+	for i, sourceScope := range sourceScopes {
+		data, errEncode := encodeScope(sourceScope)
+		require.NoError(t, errEncode)
+		targetScope, errDecode := decodeScope(data, sourceScope.Proc)
+		require.NoError(t, errDecode)
+
+		// Just do simple check
+		require.Equal(t, len(sourceScope.PreScopes), len(targetScope.PreScopes), fmt.Sprintf("related SQL is '%s'", testCases[i]))
+		require.Equal(t, len(sourceScope.Instructions), len(targetScope.Instructions), fmt.Sprintf("related SQL is '%s'", testCases[i]))
+		for j := 0; j < len(sourceScope.Instructions); j++ {
+			require.Equal(t, sourceScope.Instructions[j].Op, targetScope.Instructions[j].Op)
+		}
+		if sourceScope.DataSource == nil {
+			require.Nil(t, targetScope.DataSource)
+		} else {
+			require.Equal(t, sourceScope.DataSource.SchemaName, targetScope.DataSource.SchemaName)
+			require.Equal(t, sourceScope.DataSource.RelationName, targetScope.DataSource.RelationName)
+			require.Equal(t, sourceScope.DataSource.PushdownId, targetScope.DataSource.PushdownId)
+			require.Equal(t, sourceScope.DataSource.PushdownAddr, targetScope.DataSource.PushdownAddr)
+		}
+		require.Equal(t, sourceScope.NodeInfo.Addr, targetScope.NodeInfo.Addr)
+		require.Equal(t, sourceScope.NodeInfo.Id, targetScope.NodeInfo.Id)
+	}
+
+}
+
+func generateScopeCases(t *testing.T, testCases []string) []*Scope {
+	// getScope method generate and return the scope of a SQL string.
+	getScope := func(t1 *testing.T, sql string) *Scope {
+		proc := testutil.NewProcess()
+		e := memEngine.NewTestEngine()
+		opt := plan2.NewBaseOptimizer(e.(*memEngine.MemEngine))
+		stmts, err := mysql.Parse(sql)
+		require.NoError(t1, err)
+		qry, err := opt.Optimize(stmts[0])
+		require.NoError(t1, err)
+		c := New("test", sql, "", context.Background(), e, proc, nil)
+		err = c.Compile(&plan.Plan{Plan: &plan.Plan_Query{Query: qry}}, nil, func(a any, batch *batch.Batch) error {
+			return nil
+		})
+		require.NoError(t1, err)
+		// ignore the last operator if it's output
+		if c.scope.Instructions[len(c.scope.Instructions)-1].Op == vm.Output {
+			c.scope.Instructions = c.scope.Instructions[:len(c.scope.Instructions)-1]
+		}
+		return c.scope
+	}
+
+	result := make([]*Scope, len(testCases))
+	for i, sql := range testCases {
+		result[i] = getScope(t, sql)
+	}
+	return result
+}
diff --git a/pkg/sql/compile/server.go b/pkg/sql/compile/server.go
index 55695f5a6597c7487ad171c8dac822f2f6d66400..3b65a8d2f375606a01e3b8a6e2a23c1a80df1e8e 100644
--- a/pkg/sql/compile/server.go
+++ b/pkg/sql/compile/server.go
@@ -22,20 +22,31 @@ import (
 )
 
 var srv *Server
+var cnAddr string
 
 func NewServer() *Server {
 	if srv != nil {
 		return srv
 	}
-	srv := &Server{}
+	srv = &Server{
+		mp: make(map[uint64]*process.WaitRegister),
+	}
 	return srv
 }
 
-func (srv *Server) RegistConnector(reg *process.WaitRegister) {
+func (srv *Server) GetConnector(id uint64) *process.WaitRegister {
+	srv.Lock()
+	defer srv.Unlock()
+	defer func() { delete(srv.mp, id) }()
+	return srv.mp[id]
+}
+
+func (srv *Server) RegistConnector(reg *process.WaitRegister) uint64 {
 	srv.Lock()
 	defer srv.Unlock()
-	srv.mp[srv.curr] = reg
-	srv.curr++
+	srv.mp[srv.id] = reg
+	defer func() { srv.id++ }()
+	return srv.id
 }
 
 func (srv *Server) HandleRequest(ctx context.Context, req morpc.Message, _ uint64, cs morpc.ClientSession) error {
diff --git a/pkg/sql/compile/types.go b/pkg/sql/compile/types.go
index 24ea6cb989e14297baf3f1b56a8d60c39ebeafd8..ead5d41135d376fe703f129bd5c28c924d6ef874 100644
--- a/pkg/sql/compile/types.go
+++ b/pkg/sql/compile/types.go
@@ -20,6 +20,7 @@ import (
 
 	"github.com/matrixorigin/matrixone/pkg/container/batch"
 	"github.com/matrixorigin/matrixone/pkg/container/types"
+	"github.com/matrixorigin/matrixone/pkg/pb/pipeline"
 	"github.com/matrixorigin/matrixone/pkg/pb/plan"
 	"github.com/matrixorigin/matrixone/pkg/sql/parsers/tree"
 	plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan"
@@ -39,6 +40,7 @@ const (
 	Normal
 	Remote
 	Parallel
+	Pushdown
 	CreateDatabase
 	CreateTable
 	CreateIndex
@@ -53,6 +55,8 @@ const (
 
 // Source contains information of a relation which will be used in execution,
 type Source struct {
+	PushdownId   uint64
+	PushdownAddr string
 	SchemaName   string
 	RelationName string
 	Attributes   []string
@@ -96,6 +100,17 @@ type Scope struct {
 	Reg *process.WaitRegister
 }
 
+// scopeContext contextual information to assist in the generation of pipeline.Pipeline.
+type scopeContext struct {
+	id       int32
+	scope    *Scope
+	root     *scopeContext
+	parent   *scopeContext
+	children []*scopeContext
+	pipe     *pipeline.Pipeline
+	regs     map[*process.WaitRegister]int32
+}
+
 // anaylze information
 type anaylze struct {
 	// curr is the current index of plan
@@ -106,8 +121,8 @@ type anaylze struct {
 
 type Server struct {
 	sync.Mutex
-	curr uint64
-	mp   map[uint64]*process.WaitRegister
+	id uint64
+	mp map[uint64]*process.WaitRegister // k = id, v = reg
 }
 
 // Compile contains all the information needed for compilation.
diff --git a/pkg/vm/pipeline/pipeline.go b/pkg/vm/pipeline/pipeline.go
index c6f8d262b10c1b9a5e9075745f94d39d81376cc8..15304fbcf17c4f25f177a769eb9e1d63e5ddc1a7 100644
--- a/pkg/vm/pipeline/pipeline.go
+++ b/pkg/vm/pipeline/pipeline.go
@@ -146,6 +146,9 @@ func cleanup(p *Pipeline, proc *process.Process) {
 	for i, in := range p.instructions {
 		if in.Op == vm.Connector {
 			arg := p.instructions[i].Arg.(*connector.Argument)
+			if len(arg.Reg.Ch) > 0 {
+				break
+			}
 			select {
 			case <-arg.Reg.Ctx.Done():
 			case arg.Reg.Ch <- nil:
@@ -155,6 +158,9 @@ func cleanup(p *Pipeline, proc *process.Process) {
 		if in.Op == vm.Dispatch {
 			arg := p.instructions[i].Arg.(*dispatch.Argument)
 			for _, reg := range arg.Regs {
+				if len(reg.Ch) > 0 {
+					break
+				}
 				select {
 				case <-reg.Ctx.Done():
 				case reg.Ch <- nil:
diff --git a/proto/pipeline.proto b/proto/pipeline.proto
index 121a883a88e8b498e5bdd83b30384cb52bed2733..67a576a7ccd92b3fcf51ea816635b8ceb14f40c4 100644
--- a/proto/pipeline.proto
+++ b/proto/pipeline.proto
@@ -30,11 +30,17 @@ message Message {
     uint64  cmd = 2;
     bytes   code = 3;
     bytes   data = 4;
+    bytes   analyse = 5;
+}
+
+message Connector {
+    int32 pipeline_id = 1;
+    int32 connector_index = 2;
 }
 
 message Dispatch {
     bool all = 1;
-    repeated int32 children = 2;
+    repeated Connector connector = 2;
 }
 
 message Aggregate {
@@ -52,12 +58,25 @@ message Group {
     repeated Aggregate  aggs = 6;
 }
 
+message Join {
+    uint64 ibucket = 1;
+    uint64 nbucket = 2;
+    repeated int32 rel_list = 3;
+    repeated int32 col_list = 4;
+    plan.Expr  expr = 5;
+    repeated plan.Type  types = 6;
+    repeated plan.Expr left_cond = 7;
+    repeated plan.Expr right_cond = 8;
+}
+
 message AntiJoin{
     uint64 ibucket = 1;
     uint64 nbucket = 2;
     repeated int32 result = 3;
-    repeated plan.Expr left_cond = 4;
-    repeated plan.Expr right_cond = 5;
+    plan.Expr  expr = 4;
+    repeated plan.Type  types = 5;
+    repeated plan.Expr left_cond = 6;
+    repeated plan.Expr right_cond = 7;
 }
 
 message InnerJoin {
@@ -65,8 +84,10 @@ message InnerJoin {
     uint64 nbucket = 2;
     repeated int32 rel_list = 3;
     repeated int32 col_list = 4;
-    repeated plan.Expr left_cond = 5;
-    repeated plan.Expr right_cond = 6;
+    plan.Expr  expr = 5;
+    repeated plan.Type  types = 6;
+    repeated plan.Expr left_cond = 7;
+    repeated plan.Expr right_cond = 8;
 }
 
 message LeftJoin {
@@ -74,17 +95,20 @@ message LeftJoin {
     uint64 nbucket = 2;
     repeated int32 rel_list = 3;
     repeated int32 col_list = 4;
-    repeated plan.Type  types = 5;
-    repeated plan.Expr left_cond = 6;
-    repeated plan.Expr right_cond = 7;
+    plan.Expr  expr = 5;
+    repeated plan.Type  types = 6;
+    repeated plan.Expr left_cond = 7;
+    repeated plan.Expr right_cond = 8;
 }
 
 message SemiJoin {
     uint64 ibucket = 1;
     uint64 nbucket = 2;
     repeated int32 result = 3;
-    repeated plan.Expr left_cond = 4;
-    repeated plan.Expr right_cond = 5;
+    plan.Expr  expr = 4;
+    repeated plan.Type  types = 5;
+    repeated plan.Expr left_cond = 6;
+    repeated plan.Expr right_cond = 7;
 }
 
 message SingleJoin {
@@ -92,14 +116,31 @@ message SingleJoin {
     uint64 nbucket = 2;
     repeated int32 rel_list = 3;
     repeated int32 col_list = 4;
-    repeated plan.Type  types = 5;
-    repeated plan.Expr left_cond = 6;
-    repeated plan.Expr right_cond = 7;
+    plan.Expr  expr = 5;
+    repeated plan.Type  types = 6;
+    repeated plan.Expr left_cond = 7;
+    repeated plan.Expr right_cond = 8;
+}
+
+message MarkJoin {
+    uint64 ibucket = 1;
+    uint64 nbucket = 2;
+    repeated int32 result = 3;
+    repeated plan.Expr left_cond = 4;
+    repeated plan.Expr right_cond = 5;
+    repeated plan.Type types = 6;
+    plan.Expr cond = 7;
+    repeated plan.Expr on_list = 8;
+    bool output_null = 9;
+    bool output_mark = 10;
+    bool output_anyway = 11;
+    bool mark_meaning = 12;
 }
 
 message Product {
     repeated int32 rel_list = 1;
     repeated int32 col_list = 2;
+    repeated plan.Type  types = 3;
 }
 
 message Instruction{
@@ -108,18 +149,25 @@ message Instruction{
     // Idx specified the anaylze information index.
     int32 idx = 2;
     AntiJoin  anti = 3;
-    Dispatch  dispatch = 4;
-    Group     agg = 5;
-    InnerJoin inner_join = 6;
-    LeftJoin  left_join = 7;
-    SemiJoin  semi_join = 8;
-    SingleJoin single_join = 9;
-    Product    product = 10;
-	repeated plan.OrderBySpec order_by = 11;
-    repeated plan.Expr    project_list = 12;
-    plan.Expr    filter = 13;
-    uint64    limit = 14;
-    uint64    offset = 15;
+    Connector   connect = 4;
+    Dispatch  dispatch = 5;
+    Group     agg = 6;
+    InnerJoin inner_join = 7;
+    LeftJoin  left_join = 8;
+    SemiJoin  semi_join = 9;
+    SingleJoin single_join = 10;
+    MarkJoin    mark_join = 11;
+    Join    join = 12;
+    Product    product = 13;
+	  repeated plan.OrderBySpec order_by = 14;
+    repeated plan.Expr    project_list = 15;
+    plan.Expr    filter = 16;
+    uint64    limit = 17;
+    uint64    offset = 18;
+}
+
+message AnalysisList {
+    repeated plan.AnalyzeInfo list = 1;
 }
 
 message Source {
@@ -127,6 +175,8 @@ message Source {
     string table_name = 2;
     repeated string col_list = 3;
     string    block = 4;
+    uint64    pushdown_id = 5;
+    string    pushdown_addr = 6;
 }
 
 message NodeInfo {
@@ -152,6 +202,8 @@ message Pipeline {
     repeated Instruction instruction_list = 6;
 
     bool is_end = 7;
-    NodeInfo node = 8;
-    int32 push_down_info = 9;
+    bool is_join = 8;
+    NodeInfo node = 9;
+    int32 push_down_info = 10;
+    int32 children_count = 11;
 }