github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/process/message.go (about)

     1  // Copyright 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package process
    16  
    17  import (
    18  	"context"
    19  	"sync"
    20  )
    21  
    22  const ALLCN = "ALLCN"
    23  const CURRENTCN = "CURRENTCN"
    24  
    25  type MsgType int32
    26  
    27  const (
    28  	MsgTopValue      MsgType = 0
    29  	MsgPipelineStart MsgType = 1
    30  	MsgPipelineStop  MsgType = 2
    31  	MsgRuntimeFilter MsgType = 3
    32  	MsgHashMap       MsgType = 4
    33  	MaxMessage       MsgType = 1024
    34  )
    35  
    36  func (m MsgType) MessageName() string {
    37  	switch m {
    38  	case MsgTopValue:
    39  		return "MsgTopValue"
    40  	case MsgRuntimeFilter:
    41  		return "MsgRuntimeFilter"
    42  	}
    43  	return "unknown message type"
    44  }
    45  
    46  func NewMessageBoard() *MessageBoard {
    47  	m := &MessageBoard{
    48  		Messages: make([]*Message, 0, 1024),
    49  		Waiters:  make([]chan bool, 0, 16),
    50  		RwMutex:  &sync.RWMutex{},
    51  	}
    52  	return m
    53  }
    54  
    55  type MessageAddress struct {
    56  	CnAddr     string
    57  	OperatorID int32
    58  	ParallelID int32
    59  }
    60  
    61  type Message interface {
    62  	Serialize() []byte
    63  	Deserialize([]byte) Message
    64  	NeedBlock() bool
    65  	GetMsgTag() int32
    66  	GetReceiverAddr() MessageAddress
    67  }
    68  
    69  type MessageBoard struct {
    70  	Messages []*Message
    71  	Waiters  []chan bool
    72  	RwMutex  *sync.RWMutex // for nonblock message
    73  }
    74  
    75  func (m *MessageBoard) Reset() {
    76  	m.RwMutex.Lock()
    77  	defer m.RwMutex.Unlock()
    78  	m.Messages = m.Messages[:0]
    79  	m.Waiters = m.Waiters[:0]
    80  }
    81  
    82  type MessageReceiver struct {
    83  	offset   int32
    84  	tags     []int32
    85  	received []int32
    86  	addr     *MessageAddress
    87  	mb       *MessageBoard
    88  	waiter   chan bool
    89  }
    90  
    91  func (proc *Process) SendMessage(m Message) {
    92  	mb := proc.MessageBoard
    93  	if m.GetReceiverAddr().CnAddr == CURRENTCN { // message for current CN
    94  		mb.RwMutex.Lock()
    95  		mb.Messages = append(mb.Messages, &m)
    96  		if m.NeedBlock() {
    97  			// broadcast for block message
    98  			for _, ch := range mb.Waiters {
    99  				if ch != nil && len(ch) == 0 {
   100  					ch <- true
   101  				}
   102  			}
   103  		}
   104  		mb.RwMutex.Unlock()
   105  	} else {
   106  		//todo: send message to other CN, need to lookup cnlist
   107  		panic("unsupported message yet!")
   108  	}
   109  }
   110  
   111  func (proc *Process) NewMessageReceiver(tags []int32, addr MessageAddress) *MessageReceiver {
   112  	return &MessageReceiver{
   113  		tags: tags,
   114  		addr: &addr,
   115  		mb:   proc.MessageBoard,
   116  	}
   117  }
   118  
   119  func (mr *MessageReceiver) receiveMessageNonBlock() []Message {
   120  	mr.mb.RwMutex.RLock()
   121  	defer mr.mb.RwMutex.RUnlock()
   122  	var result []Message
   123  	lenMessages := int32(len(mr.mb.Messages))
   124  	for ; mr.offset < lenMessages; mr.offset++ {
   125  		if mr.mb.Messages[mr.offset] == nil {
   126  			continue
   127  		}
   128  		message := *mr.mb.Messages[mr.offset]
   129  		if !MatchAddress(message, mr.addr) {
   130  			continue
   131  		}
   132  		for i := range mr.tags {
   133  			if mr.tags[i] == message.GetMsgTag() {
   134  				result = append(result, message)
   135  				mr.received = append(mr.received, mr.offset)
   136  				break
   137  			}
   138  		}
   139  	}
   140  	return result
   141  }
   142  
   143  func (mr *MessageReceiver) Free() {
   144  	if len(mr.received) == 0 {
   145  		return
   146  	}
   147  	mr.mb.RwMutex.Lock()
   148  	defer mr.mb.RwMutex.Unlock()
   149  	for i := range mr.received {
   150  		mr.mb.Messages[mr.received[i]] = nil
   151  	}
   152  	mr.received = nil
   153  	mr.waiter = nil
   154  }
   155  
   156  func (mr *MessageReceiver) ReceiveMessage(needBlock bool, ctx context.Context) ([]Message, bool) {
   157  	var result = mr.receiveMessageNonBlock()
   158  	if !needBlock || len(result) > 0 {
   159  		return result, false
   160  	}
   161  	if mr.waiter == nil {
   162  		mr.waiter = make(chan bool, 1)
   163  		mr.mb.RwMutex.Lock()
   164  		mr.mb.Waiters = append(mr.mb.Waiters, mr.waiter)
   165  		mr.mb.RwMutex.Unlock()
   166  	}
   167  	for {
   168  		result = mr.receiveMessageNonBlock()
   169  		if len(result) > 0 {
   170  			break
   171  		}
   172  		select {
   173  		case <-mr.waiter:
   174  		case <-ctx.Done():
   175  			return result, true
   176  		}
   177  	}
   178  	return result, false
   179  }
   180  
   181  func MatchAddress(m Message, raddr *MessageAddress) bool {
   182  	mAddr := m.GetReceiverAddr()
   183  	if mAddr.OperatorID != raddr.OperatorID && mAddr.OperatorID != -1 {
   184  		return false
   185  	}
   186  	if mAddr.ParallelID != raddr.ParallelID && mAddr.ParallelID != -1 {
   187  		return false
   188  	}
   189  	return true
   190  }
   191  
   192  func AddrBroadCastOnCurrentCN() MessageAddress {
   193  	return MessageAddress{
   194  		CnAddr:     CURRENTCN,
   195  		OperatorID: -1,
   196  		ParallelID: -1,
   197  	}
   198  }
   199  
   200  func AddrBroadCastOnALLCN() MessageAddress {
   201  	return MessageAddress{
   202  		CnAddr:     ALLCN,
   203  		OperatorID: -1,
   204  		ParallelID: -1,
   205  	}
   206  }