mirror of
https://github.com/MontFerret/ferret.git
synced 2024-12-02 09:21:54 +02:00
Bench (#683)
* Extended public API * Refactored event loop * Refactored parser * Fixed unit tests * Added --dry-run opt * Added total alloc * Added profiler * Fixed driver registration
This commit is contained in:
parent
77569f3ced
commit
bd6463fa29
81
benchmarks/drivers/open_test.go
Normal file
81
benchmarks/drivers/open_test.go
Normal file
@ -0,0 +1,81 @@
|
||||
package drivers_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/MontFerret/ferret/pkg/drivers"
|
||||
"github.com/MontFerret/ferret/pkg/drivers/cdp"
|
||||
"testing"
|
||||
|
||||
"github.com/MontFerret/ferret"
|
||||
)
|
||||
|
||||
var c *ferret.Instance
|
||||
|
||||
func init() {
|
||||
c = ferret.New()
|
||||
c.Drivers().Register(cdp.NewDriver(), drivers.AsDefault())
|
||||
}
|
||||
|
||||
func Benchmark_Open_CDP(b *testing.B) {
|
||||
ctx := context.Background()
|
||||
|
||||
p, err := c.Compile(`
|
||||
LET doc = DOCUMENT("https://www.montferret.dev")
|
||||
|
||||
RETURN TRUE
|
||||
`)
|
||||
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
||||
for n := 0; n < b.N; n++ {
|
||||
if _, err := c.Run(ctx, p); err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func Benchmark_Navigate_CDP(b *testing.B) {
|
||||
ctx := context.Background()
|
||||
|
||||
p, err := c.Compile(`
|
||||
LET doc = DOCUMENT('https://www.theverge.com/tech', {
|
||||
driver: "cdp",
|
||||
ignore: {
|
||||
resources: [
|
||||
{
|
||||
url: "*",
|
||||
type: "image"
|
||||
}
|
||||
]
|
||||
}
|
||||
})
|
||||
|
||||
WAIT_ELEMENT(doc, '.c-compact-river__entry', 5000)
|
||||
LET articles = ELEMENTS(doc, '.c-entry-box--compact__image-wrapper')
|
||||
LET links = (
|
||||
FOR article IN articles
|
||||
FILTER article.attributes?.href LIKE 'https://www.theverge.com/*'
|
||||
RETURN article.attributes.href
|
||||
)
|
||||
|
||||
FOR link IN links
|
||||
LIMIT 10
|
||||
// The Verge has pretty heavy pages, so let's increase the navigation wait time
|
||||
NAVIGATE(doc, link, 20000)
|
||||
WAIT_ELEMENT(doc, '.c-entry-content', 15000)
|
||||
LET texter = ELEMENT(doc, '.c-entry-content')
|
||||
RETURN texter.innerText
|
||||
`)
|
||||
|
||||
if err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
|
||||
for n := 0; n < b.N; n++ {
|
||||
if _, err := c.Run(ctx, p); err != nil {
|
||||
b.Fatal(err)
|
||||
}
|
||||
}
|
||||
}
|
193
e2e/cli.go
193
e2e/cli.go
@ -2,16 +2,21 @@ package main
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"flag"
|
||||
"fmt"
|
||||
"github.com/rs/zerolog"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"os/signal"
|
||||
"path/filepath"
|
||||
rt "runtime"
|
||||
"runtime/pprof"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/rs/zerolog"
|
||||
|
||||
"github.com/MontFerret/ferret"
|
||||
"github.com/MontFerret/ferret/pkg/drivers/cdp"
|
||||
@ -21,6 +26,129 @@ import (
|
||||
"github.com/MontFerret/ferret/pkg/runtime/logging"
|
||||
)
|
||||
|
||||
type (
|
||||
Timer struct {
|
||||
start time.Time
|
||||
end time.Time
|
||||
}
|
||||
|
||||
Profiler struct {
|
||||
labels []string
|
||||
timers map[string]*Timer
|
||||
allocs map[string]*rt.MemStats
|
||||
cpus map[string]*bytes.Buffer
|
||||
heaps map[string]*bytes.Buffer
|
||||
}
|
||||
)
|
||||
|
||||
func NewProfiler() *Profiler {
|
||||
return &Profiler{
|
||||
labels: make([]string, 0, 10),
|
||||
timers: make(map[string]*Timer),
|
||||
allocs: make(map[string]*rt.MemStats),
|
||||
cpus: make(map[string]*bytes.Buffer),
|
||||
heaps: make(map[string]*bytes.Buffer),
|
||||
}
|
||||
}
|
||||
|
||||
func (p *Profiler) StartTimer(label string) {
|
||||
timer := &Timer{
|
||||
start: time.Now(),
|
||||
}
|
||||
|
||||
p.timers[label] = timer
|
||||
p.labels = append(p.labels, label)
|
||||
}
|
||||
|
||||
func (p *Profiler) StopTimer(label string) {
|
||||
timer, found := p.timers[label]
|
||||
|
||||
if !found {
|
||||
panic(fmt.Sprintf("Timer not found: %s", label))
|
||||
}
|
||||
|
||||
timer.end = time.Now()
|
||||
}
|
||||
|
||||
func (p *Profiler) HeapSnapshot(label string) {
|
||||
heap := &bytes.Buffer{}
|
||||
|
||||
err := pprof.WriteHeapProfile(heap)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
p.heaps[label] = heap
|
||||
p.labels = append(p.labels, label)
|
||||
}
|
||||
|
||||
func (p *Profiler) Allocations(label string) {
|
||||
stats := &rt.MemStats{}
|
||||
|
||||
rt.ReadMemStats(stats)
|
||||
|
||||
p.allocs[label] = stats
|
||||
p.labels = append(p.labels, label)
|
||||
}
|
||||
|
||||
func (p *Profiler) StartCPU(label string) {
|
||||
b := &bytes.Buffer{}
|
||||
|
||||
if err := pprof.StartCPUProfile(b); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
p.cpus[label] = b
|
||||
p.labels = append(p.labels, label)
|
||||
}
|
||||
|
||||
func (p *Profiler) StopCPU() {
|
||||
pprof.StopCPUProfile()
|
||||
}
|
||||
|
||||
func (p *Profiler) Print(label string) {
|
||||
writer := &bytes.Buffer{}
|
||||
|
||||
timer, found := p.timers[label]
|
||||
|
||||
if found {
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Time: %s", timer.end.Sub(timer.start)))
|
||||
}
|
||||
|
||||
stats, found := p.allocs[label]
|
||||
|
||||
if found {
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Alloc: %s", byteCountDecimal(stats.Alloc)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Frees: %s", byteCountDecimal(stats.Frees)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Total Alloc: %s", byteCountDecimal(stats.TotalAlloc)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap Alloc: %s", byteCountDecimal(stats.HeapAlloc)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap Sys: %s", byteCountDecimal(stats.HeapSys)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap Idle: %s", byteCountDecimal(stats.HeapIdle)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap In Use: %s", byteCountDecimal(stats.HeapInuse)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap Released: %s", byteCountDecimal(stats.HeapReleased)))
|
||||
fmt.Fprintln(writer, fmt.Sprintf("Heap Objects: %d", stats.HeapObjects))
|
||||
}
|
||||
|
||||
cpu, found := p.cpus[label]
|
||||
|
||||
if found {
|
||||
fmt.Fprintln(writer, cpu.String())
|
||||
}
|
||||
|
||||
if writer.Len() > 0 {
|
||||
fmt.Println(fmt.Sprintf("%s:", label))
|
||||
fmt.Println("-----")
|
||||
fmt.Println(writer.String())
|
||||
}
|
||||
}
|
||||
|
||||
func (p *Profiler) PrintAll() {
|
||||
for _, label := range p.labels {
|
||||
p.Print(label)
|
||||
}
|
||||
}
|
||||
|
||||
type Params []string
|
||||
|
||||
func (p *Params) String() string {
|
||||
@ -65,6 +193,12 @@ var (
|
||||
"set CDP address",
|
||||
)
|
||||
|
||||
dryRun = flag.Bool(
|
||||
"dry-run",
|
||||
false,
|
||||
"compiles a given query, but does not execute",
|
||||
)
|
||||
|
||||
logLevel = flag.String(
|
||||
"log-level",
|
||||
logging.ErrorLevel.String(),
|
||||
@ -151,7 +285,7 @@ func main() {
|
||||
}()
|
||||
|
||||
if query != "" {
|
||||
err = execQuery(ctx, engine, opts, query)
|
||||
err = runQuery(ctx, engine, opts, query)
|
||||
} else {
|
||||
err = execFiles(ctx, engine, opts, files)
|
||||
}
|
||||
@ -228,7 +362,7 @@ func execFiles(ctx context.Context, engine *ferret.Instance, opts []runtime.Opti
|
||||
|
||||
log.Debug().Msg("successfully read file")
|
||||
log.Debug().Msg("executing file...")
|
||||
err = execQuery(ctx, engine, opts, string(out))
|
||||
err = runQuery(ctx, engine, opts, string(out))
|
||||
|
||||
if err != nil {
|
||||
log.Debug().Err(err).Msg("failed to execute file")
|
||||
@ -253,6 +387,14 @@ func execFiles(ctx context.Context, engine *ferret.Instance, opts []runtime.Opti
|
||||
return nil
|
||||
}
|
||||
|
||||
func runQuery(ctx context.Context, engine *ferret.Instance, opts []runtime.Option, query string) error {
|
||||
if !(*dryRun) {
|
||||
return execQuery(ctx, engine, opts, query)
|
||||
}
|
||||
|
||||
return analyzeQuery(ctx, engine, opts, query)
|
||||
}
|
||||
|
||||
func execQuery(ctx context.Context, engine *ferret.Instance, opts []runtime.Option, query string) error {
|
||||
out, err := engine.Exec(ctx, query, opts...)
|
||||
|
||||
@ -264,3 +406,48 @@ func execQuery(ctx context.Context, engine *ferret.Instance, opts []runtime.Opti
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func analyzeQuery(ctx context.Context, engine *ferret.Instance, opts []runtime.Option, query string) error {
|
||||
compilation := "Compilation"
|
||||
beforeCompilation := "Before compilation"
|
||||
afterCompilation := "After compilation"
|
||||
prof := NewProfiler()
|
||||
|
||||
prof.Allocations(beforeCompilation)
|
||||
prof.StartTimer(compilation)
|
||||
program := engine.MustCompile(query)
|
||||
|
||||
prof.StopTimer(compilation)
|
||||
prof.Allocations(afterCompilation)
|
||||
|
||||
exec := "Execution"
|
||||
beforeExec := "Before execution"
|
||||
afterExec := "After execution"
|
||||
|
||||
prof.Allocations(beforeExec)
|
||||
prof.StartTimer(exec)
|
||||
|
||||
engine.MustRun(ctx, program, opts...)
|
||||
|
||||
prof.StopTimer(exec)
|
||||
prof.Allocations(afterExec)
|
||||
|
||||
prof.PrintAll()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func byteCountDecimal(b uint64) string {
|
||||
const unit = 1000
|
||||
|
||||
if b < unit {
|
||||
return fmt.Sprintf("%d B", b)
|
||||
}
|
||||
|
||||
div, exp := int64(unit), 0
|
||||
for n := b / unit; n >= unit; n /= unit {
|
||||
div *= unit
|
||||
exp++
|
||||
}
|
||||
return fmt.Sprintf("%.1f %cB", float64(b)/float64(div), "kMGTPE"[exp])
|
||||
}
|
||||
|
24
ferret.go
24
ferret.go
@ -46,9 +46,7 @@ func (i *Instance) Exec(ctx context.Context, query string, opts ...runtime.Optio
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, drv := range i.drivers.GetAll() {
|
||||
ctx = drivers.WithContext(ctx, drv)
|
||||
}
|
||||
ctx = i.drivers.WithContext(ctx)
|
||||
|
||||
return p.Run(ctx, opts...)
|
||||
}
|
||||
@ -62,3 +60,23 @@ func (i *Instance) MustExec(ctx context.Context, query string, opts ...runtime.O
|
||||
|
||||
return out
|
||||
}
|
||||
|
||||
func (i *Instance) Run(ctx context.Context, program *runtime.Program, opts ...runtime.Option) ([]byte, error) {
|
||||
if program == nil {
|
||||
return nil, core.Error(core.ErrInvalidArgument, "program")
|
||||
}
|
||||
|
||||
ctx = i.drivers.WithContext(ctx)
|
||||
|
||||
return program.Run(ctx, opts...)
|
||||
}
|
||||
|
||||
func (i *Instance) MustRun(ctx context.Context, program *runtime.Program, opts ...runtime.Option) []byte {
|
||||
out, err := i.Run(ctx, program, opts...)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return out
|
||||
}
|
||||
|
@ -8,5 +8,6 @@ var (
|
||||
ErrVariableNotFound = errors.New("variable not found")
|
||||
ErrVariableNotUnique = errors.New("variable is already defined")
|
||||
ErrInvalidToken = errors.New("invalid token")
|
||||
ErrUnexpectedToken = errors.New("unexpected token")
|
||||
ErrInvalidDataSource = errors.New("invalid data source")
|
||||
)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -151,11 +151,22 @@ func (drv *Driver) init(ctx context.Context) error {
|
||||
return errors.Wrap(err, "failed to initialize driver")
|
||||
}
|
||||
|
||||
dialOpts := make([]rpcc.DialOption, 0, 2)
|
||||
|
||||
if drv.options.Connection != nil {
|
||||
if drv.options.Connection.BufferSize > 0 {
|
||||
dialOpts = append(dialOpts, rpcc.WithWriteBufferSize(drv.options.Connection.BufferSize))
|
||||
}
|
||||
|
||||
if drv.options.Connection.Compression {
|
||||
dialOpts = append(dialOpts, rpcc.WithCompression())
|
||||
}
|
||||
}
|
||||
|
||||
bconn, err := rpcc.DialContext(
|
||||
ctx,
|
||||
ver.WebSocketDebuggerURL,
|
||||
rpcc.WithWriteBufferSize(104857586),
|
||||
rpcc.WithCompression(),
|
||||
dialOpts...,
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
|
@ -1,74 +0,0 @@
|
||||
package events
|
||||
|
||||
import "sync"
|
||||
|
||||
type ListenerCollection struct {
|
||||
mu sync.RWMutex
|
||||
values map[ID]map[ListenerID]Listener
|
||||
}
|
||||
|
||||
func NewListenerCollection() *ListenerCollection {
|
||||
lc := new(ListenerCollection)
|
||||
lc.values = make(map[ID]map[ListenerID]Listener)
|
||||
|
||||
return lc
|
||||
}
|
||||
|
||||
func (lc *ListenerCollection) Size(eventID ID) int {
|
||||
lc.mu.RLock()
|
||||
defer lc.mu.RUnlock()
|
||||
|
||||
bucket, exists := lc.values[eventID]
|
||||
|
||||
if !exists {
|
||||
return 0
|
||||
}
|
||||
|
||||
return len(bucket)
|
||||
}
|
||||
|
||||
func (lc *ListenerCollection) Add(listener Listener) {
|
||||
lc.mu.Lock()
|
||||
defer lc.mu.Unlock()
|
||||
|
||||
bucket, exists := lc.values[listener.EventID]
|
||||
|
||||
if !exists {
|
||||
bucket = make(map[ListenerID]Listener)
|
||||
lc.values[listener.EventID] = bucket
|
||||
}
|
||||
|
||||
bucket[listener.ID] = listener
|
||||
}
|
||||
|
||||
func (lc *ListenerCollection) Remove(eventID ID, listenerID ListenerID) {
|
||||
lc.mu.Lock()
|
||||
defer lc.mu.Unlock()
|
||||
|
||||
bucket, exists := lc.values[eventID]
|
||||
|
||||
if !exists {
|
||||
return
|
||||
}
|
||||
|
||||
delete(bucket, listenerID)
|
||||
}
|
||||
|
||||
func (lc *ListenerCollection) Values(eventID ID) []Listener {
|
||||
lc.mu.RLock()
|
||||
defer lc.mu.RUnlock()
|
||||
|
||||
bucket, exists := lc.values[eventID]
|
||||
|
||||
if !exists {
|
||||
return []Listener{}
|
||||
}
|
||||
|
||||
snapshot := make([]Listener, 0, len(bucket))
|
||||
|
||||
for _, listener := range bucket {
|
||||
snapshot = append(snapshot, listener)
|
||||
}
|
||||
|
||||
return snapshot
|
||||
}
|
@ -2,74 +2,71 @@ package events
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/core"
|
||||
"math/rand"
|
||||
"sync"
|
||||
)
|
||||
|
||||
type Loop struct {
|
||||
mu sync.RWMutex
|
||||
sources *SourceCollection
|
||||
listeners *ListenerCollection
|
||||
cancel context.CancelFunc
|
||||
listeners map[ID]map[ListenerID]Listener
|
||||
sources []SourceFactory
|
||||
}
|
||||
|
||||
func NewLoop() *Loop {
|
||||
func NewLoop(sources ...SourceFactory) *Loop {
|
||||
loop := new(Loop)
|
||||
loop.sources = NewSourceCollection()
|
||||
loop.listeners = NewListenerCollection()
|
||||
loop.listeners = make(map[ID]map[ListenerID]Listener)
|
||||
loop.sources = sources
|
||||
|
||||
return loop
|
||||
}
|
||||
|
||||
func (loop *Loop) Run(ctx context.Context) error {
|
||||
loop.mu.Lock()
|
||||
defer loop.mu.Unlock()
|
||||
func (loop *Loop) Run(ctx context.Context) (context.CancelFunc, error) {
|
||||
var err error
|
||||
sources := make([]Source, 0, len(loop.sources))
|
||||
|
||||
if loop.cancel != nil {
|
||||
return core.Error(core.ErrInvalidOperation, "loop is already running")
|
||||
// create new sources
|
||||
for _, factory := range loop.sources {
|
||||
src, e := factory(ctx)
|
||||
|
||||
if e != nil {
|
||||
err = e
|
||||
|
||||
break
|
||||
}
|
||||
|
||||
sources = append(sources, src)
|
||||
}
|
||||
|
||||
childCtx, cancel := context.WithCancel(ctx)
|
||||
// if error occurred
|
||||
if err != nil {
|
||||
// clean up the open ones
|
||||
for _, src := range sources {
|
||||
src.Close()
|
||||
}
|
||||
|
||||
loop.cancel = cancel
|
||||
|
||||
go loop.run(childCtx)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (loop *Loop) Close() error {
|
||||
loop.mu.Lock()
|
||||
defer loop.mu.Unlock()
|
||||
|
||||
if loop.cancel != nil {
|
||||
loop.cancel()
|
||||
loop.cancel = nil
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return loop.sources.Close()
|
||||
}
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
|
||||
func (loop *Loop) AddSource(source Source) {
|
||||
loop.mu.RLock()
|
||||
defer loop.mu.RUnlock()
|
||||
for _, src := range sources {
|
||||
loop.consume(ctx, src)
|
||||
}
|
||||
|
||||
loop.sources.Add(source)
|
||||
}
|
||||
|
||||
func (loop *Loop) RemoveSource(source Source) {
|
||||
loop.mu.RLock()
|
||||
defer loop.mu.RUnlock()
|
||||
|
||||
loop.sources.Remove(source)
|
||||
return cancel, nil
|
||||
}
|
||||
|
||||
func (loop *Loop) Listeners(eventID ID) int {
|
||||
loop.mu.RLock()
|
||||
defer loop.mu.RUnlock()
|
||||
|
||||
return loop.listeners.Size(eventID)
|
||||
bucket, exists := loop.listeners[eventID]
|
||||
|
||||
if !exists {
|
||||
return 0
|
||||
}
|
||||
|
||||
return len(bucket)
|
||||
}
|
||||
|
||||
func (loop *Loop) AddListener(eventID ID, handler Handler) ListenerID {
|
||||
@ -82,7 +79,14 @@ func (loop *Loop) AddListener(eventID ID, handler Handler) ListenerID {
|
||||
Handler: handler,
|
||||
}
|
||||
|
||||
loop.listeners.Add(listener)
|
||||
bucket, exists := loop.listeners[listener.EventID]
|
||||
|
||||
if !exists {
|
||||
bucket = make(map[ListenerID]Listener)
|
||||
loop.listeners[listener.EventID] = bucket
|
||||
}
|
||||
|
||||
bucket[listener.ID] = listener
|
||||
|
||||
return listener.ID
|
||||
}
|
||||
@ -91,82 +95,64 @@ func (loop *Loop) RemoveListener(eventID ID, listenerID ListenerID) {
|
||||
loop.mu.RLock()
|
||||
defer loop.mu.RUnlock()
|
||||
|
||||
loop.listeners.Remove(eventID, listenerID)
|
||||
bucket, exists := loop.listeners[eventID]
|
||||
|
||||
if !exists {
|
||||
return
|
||||
}
|
||||
|
||||
delete(bucket, listenerID)
|
||||
}
|
||||
|
||||
// run starts running an event loop.
|
||||
// It constantly iterates over each event source.
|
||||
func (loop *Loop) run(ctx context.Context) {
|
||||
sources := loop.sources
|
||||
size := sources.Size()
|
||||
counter := -1
|
||||
|
||||
for {
|
||||
if isCtxDone(ctx) {
|
||||
break
|
||||
}
|
||||
|
||||
counter++
|
||||
|
||||
if counter >= size {
|
||||
// reset the counter
|
||||
size = sources.Size()
|
||||
counter = 0
|
||||
}
|
||||
|
||||
var source Source
|
||||
|
||||
if size > 0 {
|
||||
found, err := sources.Get(counter)
|
||||
|
||||
if err == nil {
|
||||
source = found
|
||||
} else {
|
||||
// force to reset counter
|
||||
counter = size
|
||||
continue
|
||||
func (loop *Loop) consume(ctx context.Context, src Source) {
|
||||
go func() {
|
||||
defer func() {
|
||||
if err := src.Close(); err != nil {
|
||||
loop.emit(ctx, Error, err)
|
||||
}
|
||||
} else {
|
||||
continue
|
||||
}
|
||||
}()
|
||||
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-source.Ready():
|
||||
if isCtxDone(ctx) {
|
||||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case <-src.Ready():
|
||||
if isCtxDone(ctx) {
|
||||
return
|
||||
}
|
||||
|
||||
event, err := src.Recv()
|
||||
|
||||
if err != nil {
|
||||
loop.emit(ctx, Error, err)
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
loop.emit(ctx, event.ID, event.Data)
|
||||
}
|
||||
|
||||
event, err := source.Recv()
|
||||
|
||||
loop.emit(ctx, event.ID, event.Data, err)
|
||||
default:
|
||||
continue
|
||||
}
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (loop *Loop) emit(ctx context.Context, eventID ID, message interface{}, err error) {
|
||||
if err != nil {
|
||||
eventID = Error
|
||||
message = err
|
||||
func (loop *Loop) emit(ctx context.Context, eventID ID, message interface{}) {
|
||||
loop.mu.Lock()
|
||||
defer loop.mu.Unlock()
|
||||
|
||||
listeners, exist := loop.listeners[eventID]
|
||||
|
||||
if !exist {
|
||||
return
|
||||
}
|
||||
|
||||
loop.mu.RLock()
|
||||
snapshot := loop.listeners.Values(eventID)
|
||||
loop.mu.RUnlock()
|
||||
|
||||
for _, listener := range snapshot {
|
||||
for _, listener := range listeners {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
default:
|
||||
// if returned false, it means the loops should not call the handler anymore
|
||||
if !listener.Handler(ctx, message) {
|
||||
loop.mu.RLock()
|
||||
loop.listeners.Remove(eventID, listener.ID)
|
||||
loop.mu.RUnlock()
|
||||
delete(listeners, listener.ID)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -3,49 +3,19 @@ package events_test
|
||||
import (
|
||||
"context"
|
||||
"github.com/MontFerret/ferret/pkg/drivers/cdp/events"
|
||||
"github.com/mafredri/cdp/protocol/dom"
|
||||
"github.com/mafredri/cdp/protocol/page"
|
||||
"github.com/mafredri/cdp/rpcc"
|
||||
. "github.com/smartystreets/goconvey/convey"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
type (
|
||||
TestEventStream struct {
|
||||
ready chan struct{}
|
||||
message chan interface{}
|
||||
}
|
||||
|
||||
TestLoadEventFiredClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestDocumentUpdatedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestAttributeModifiedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestAttributeRemovedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestChildNodeCountUpdatedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestChildNodeInsertedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
|
||||
TestChildNodeRemovedClient struct {
|
||||
*TestEventStream
|
||||
}
|
||||
)
|
||||
type TestEventStream struct {
|
||||
closed atomic.Value
|
||||
ready chan struct{}
|
||||
messages chan string
|
||||
}
|
||||
|
||||
var TestEvent = events.New("test_event")
|
||||
|
||||
@ -56,11 +26,16 @@ func NewTestEventStream() *TestEventStream {
|
||||
func NewBufferedTestEventStream(buffer int) *TestEventStream {
|
||||
es := new(TestEventStream)
|
||||
es.ready = make(chan struct{}, buffer)
|
||||
es.message = make(chan interface{}, buffer)
|
||||
es.messages = make(chan string, buffer)
|
||||
es.closed.Store(false)
|
||||
|
||||
return es
|
||||
}
|
||||
|
||||
func (es *TestEventStream) IsClosed() bool {
|
||||
return es.closed.Load().(bool)
|
||||
}
|
||||
|
||||
func (es *TestEventStream) Ready() <-chan struct{} {
|
||||
return es.ready
|
||||
}
|
||||
@ -70,68 +45,38 @@ func (es *TestEventStream) RecvMsg(i interface{}) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (es *TestEventStream) Recv() (interface{}, error) {
|
||||
msg := <-es.messages
|
||||
|
||||
return msg, nil
|
||||
}
|
||||
|
||||
func (es *TestEventStream) Close() error {
|
||||
close(es.message)
|
||||
es.closed.Store(true)
|
||||
close(es.messages)
|
||||
close(es.ready)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (es *TestEventStream) Emit(msg interface{}) {
|
||||
func (es *TestEventStream) EmitP(msg string, skipCheck bool) {
|
||||
if !skipCheck {
|
||||
isClosed := es.closed.Load().(bool)
|
||||
|
||||
if isClosed {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
es.ready <- struct{}{}
|
||||
es.message <- msg
|
||||
es.messages <- msg
|
||||
}
|
||||
|
||||
func (es *TestLoadEventFiredClient) Recv() (*page.LoadEventFiredReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*page.LoadEventFiredReply)
|
||||
|
||||
return reply, nil
|
||||
func (es *TestEventStream) Emit(msg string) {
|
||||
es.EmitP(msg, false)
|
||||
}
|
||||
|
||||
func (es *TestLoadEventFiredClient) EmitDefault() {
|
||||
es.TestEventStream.Emit(&page.LoadEventFiredReply{})
|
||||
}
|
||||
|
||||
func (es *TestDocumentUpdatedClient) Recv() (*dom.DocumentUpdatedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.DocumentUpdatedReply)
|
||||
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (es *TestAttributeModifiedClient) Recv() (*dom.AttributeModifiedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.AttributeModifiedReply)
|
||||
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (es *TestAttributeRemovedClient) Recv() (*dom.AttributeRemovedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.AttributeRemovedReply)
|
||||
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (es *TestChildNodeCountUpdatedClient) Recv() (*dom.ChildNodeCountUpdatedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.ChildNodeCountUpdatedReply)
|
||||
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (es *TestChildNodeInsertedClient) Recv() (*dom.ChildNodeInsertedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.ChildNodeInsertedReply)
|
||||
|
||||
return reply, nil
|
||||
}
|
||||
|
||||
func (es *TestChildNodeRemovedClient) Recv() (*dom.ChildNodeRemovedReply, error) {
|
||||
r := <-es.message
|
||||
reply := r.(*dom.ChildNodeRemovedReply)
|
||||
|
||||
return reply, nil
|
||||
func (es *TestEventStream) EmitDefault() {
|
||||
es.Emit("")
|
||||
}
|
||||
|
||||
func wait() {
|
||||
@ -175,22 +120,25 @@ func (c *Counter) Value() int64 {
|
||||
func TestLoop(t *testing.T) {
|
||||
Convey(".AddListener", t, func() {
|
||||
Convey("Should add a new listener", func() {
|
||||
loop := events.NewLoop()
|
||||
counter := NewCounter()
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
src := events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
})
|
||||
var tes *TestEventStream
|
||||
|
||||
loop.AddSource(src)
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewTestEventStream()
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
loop.Run(ctx)
|
||||
_, err := loop.Run(ctx)
|
||||
defer cancel()
|
||||
|
||||
onLoad.EmitDefault()
|
||||
So(err, ShouldBeNil)
|
||||
|
||||
tes.EmitDefault()
|
||||
|
||||
wait()
|
||||
|
||||
@ -202,7 +150,7 @@ func TestLoop(t *testing.T) {
|
||||
|
||||
wait()
|
||||
|
||||
onLoad.EmitDefault()
|
||||
tes.EmitDefault()
|
||||
|
||||
wait()
|
||||
|
||||
@ -213,25 +161,30 @@ func TestLoop(t *testing.T) {
|
||||
Convey(".RemoveListener", t, func() {
|
||||
Convey("Should remove a listener", func() {
|
||||
Convey("Should add a new listener", func() {
|
||||
loop := events.NewLoop()
|
||||
counter := NewCounter()
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
src := events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
})
|
||||
var test *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
test = NewTestEventStream()
|
||||
return test, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
loop.AddSource(src)
|
||||
id := loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
counter.Increase()
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
loop.Run(ctx)
|
||||
c, err := loop.Run(ctx)
|
||||
defer cancel()
|
||||
|
||||
onLoad.EmitDefault()
|
||||
So(err, ShouldBeNil)
|
||||
So(c, ShouldHaveSameTypeAs, cancel)
|
||||
|
||||
test.EmitDefault()
|
||||
|
||||
wait()
|
||||
|
||||
@ -243,7 +196,7 @@ func TestLoop(t *testing.T) {
|
||||
|
||||
wait()
|
||||
|
||||
onLoad.EmitDefault()
|
||||
test.EmitDefault()
|
||||
|
||||
wait()
|
||||
|
||||
@ -252,87 +205,16 @@ func TestLoop(t *testing.T) {
|
||||
})
|
||||
})
|
||||
|
||||
Convey(".AddSource", t, func() {
|
||||
Convey("Should add a new event source when not started", func() {
|
||||
loop := events.NewLoop()
|
||||
counter := NewCounter()
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
counter.Increase()
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
loop.Run(ctx)
|
||||
defer cancel()
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
|
||||
go func() {
|
||||
onLoad.EmitDefault()
|
||||
}()
|
||||
|
||||
wait()
|
||||
|
||||
So(counter.Value(), ShouldEqual, 0)
|
||||
|
||||
src := events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
})
|
||||
|
||||
loop.AddSource(src)
|
||||
|
||||
wait()
|
||||
|
||||
So(counter.Value(), ShouldEqual, 1)
|
||||
})
|
||||
})
|
||||
|
||||
Convey(".RemoveSource", t, func() {
|
||||
Convey("Should remove a source", func() {
|
||||
loop := events.NewLoop()
|
||||
counter := NewCounter()
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
So(loop.Run(ctx), ShouldBeNil)
|
||||
defer cancel()
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
counter.Increase()
|
||||
}))
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
src := events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
})
|
||||
|
||||
loop.AddSource(src)
|
||||
|
||||
wait()
|
||||
|
||||
onLoad.EmitDefault()
|
||||
|
||||
wait()
|
||||
|
||||
So(counter.Value(), ShouldEqual, 1)
|
||||
|
||||
loop.RemoveSource(src)
|
||||
|
||||
wait()
|
||||
|
||||
go func() {
|
||||
onLoad.EmitDefault()
|
||||
}()
|
||||
|
||||
wait()
|
||||
|
||||
So(counter.Value(), ShouldEqual, 1)
|
||||
})
|
||||
})
|
||||
|
||||
Convey("Should not call listener once it was removed", t, func() {
|
||||
loop := events.NewLoop()
|
||||
var tes *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewTestEventStream()
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
onEvent := make(chan struct{})
|
||||
|
||||
counter := NewCounter()
|
||||
@ -349,34 +231,32 @@ func TestLoop(t *testing.T) {
|
||||
loop.RemoveListener(TestEvent, id)
|
||||
}()
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
|
||||
loop.AddSource(events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
So(loop.Run(ctx), ShouldBeNil)
|
||||
_, err := loop.Run(ctx)
|
||||
So(err, ShouldBeNil)
|
||||
defer cancel()
|
||||
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
onLoad.Emit(&page.LoadEventFiredReply{})
|
||||
tes.EmitDefault()
|
||||
|
||||
time.Sleep(time.Duration(10) * time.Millisecond)
|
||||
|
||||
So(counter.Value(), ShouldEqual, 1)
|
||||
})
|
||||
|
||||
SkipConvey("Should stop on Context.Done", t, func() {
|
||||
loop := events.NewLoop()
|
||||
Convey("Should stop on Context.Done", t, func() {
|
||||
eventsToFire := 5
|
||||
counter := NewCounter()
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewBufferedTestEventStream(10)}
|
||||
loop.AddSource(events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
var tes *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewTestEventStream()
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
@ -384,30 +264,55 @@ func TestLoop(t *testing.T) {
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
So(loop.Run(ctx), ShouldBeNil)
|
||||
_, err := loop.Run(ctx)
|
||||
So(err, ShouldBeNil)
|
||||
|
||||
for i := 0; i <= eventsToFire; i++ {
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
onLoad.Emit(&page.LoadEventFiredReply{})
|
||||
tes.EmitDefault()
|
||||
}
|
||||
|
||||
// Stop the loop
|
||||
cancel()
|
||||
|
||||
time.Sleep(time.Duration(500) * time.Millisecond)
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
onLoad.Emit(&page.LoadEventFiredReply{})
|
||||
So(tes.IsClosed(), ShouldBeTrue)
|
||||
})
|
||||
|
||||
Convey("Should stop on nested Context.Done", t, func() {
|
||||
eventsToFire := 5
|
||||
counter := NewCounter()
|
||||
|
||||
var tes *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewTestEventStream()
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
counter.Increase()
|
||||
}))
|
||||
|
||||
cancel, err := loop.Run(context.Background())
|
||||
So(err, ShouldBeNil)
|
||||
|
||||
for i := 0; i <= eventsToFire; i++ {
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
onLoad.Emit(&page.LoadEventFiredReply{})
|
||||
tes.EmitDefault()
|
||||
}
|
||||
|
||||
time.Sleep(time.Duration(500) * time.Millisecond)
|
||||
// Stop the loop
|
||||
cancel()
|
||||
|
||||
So(counter.Value(), ShouldEqual, eventsToFire)
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
So(tes.IsClosed(), ShouldBeTrue)
|
||||
})
|
||||
}
|
||||
|
||||
@ -446,7 +351,14 @@ func BenchmarkLoop_AddListenerAsync2(b *testing.B) {
|
||||
}
|
||||
|
||||
func BenchmarkLoop_Start(b *testing.B) {
|
||||
loop := events.NewLoop()
|
||||
var tes *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewTestEventStream()
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
@ -471,18 +383,67 @@ func BenchmarkLoop_Start(b *testing.B) {
|
||||
|
||||
}))
|
||||
|
||||
onLoad := &TestLoadEventFiredClient{NewTestEventStream()}
|
||||
|
||||
loop.AddSource(events.NewSource(TestEvent, onLoad, func(_ rpcc.Stream) (i interface{}, e error) {
|
||||
return onLoad.Recv()
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
loop.Run(ctx)
|
||||
_, err := loop.Run(ctx)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
defer cancel()
|
||||
|
||||
for n := 0; n < b.N; n++ {
|
||||
onLoad.Emit(&page.LoadEventFiredReply{})
|
||||
tes.EmitP("", true)
|
||||
}
|
||||
}
|
||||
|
||||
func BenchmarkLoop_StartAsync(b *testing.B) {
|
||||
var tes *TestEventStream
|
||||
|
||||
loop := events.NewLoop(events.NewStreamSourceFactory(TestEvent, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
tes = NewBufferedTestEventStream(b.N)
|
||||
return tes, nil
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(*TestEventStream).Recv()
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
|
||||
loop.AddListener(TestEvent, events.Always(func(ctx context.Context, message interface{}) {
|
||||
|
||||
}))
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
_, err := loop.Run(ctx)
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
defer cancel()
|
||||
|
||||
b.RunParallel(func(pb *testing.PB) {
|
||||
for pb.Next() {
|
||||
tes.EmitP("", true)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
package events
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/mafredri/cdp/rpcc"
|
||||
)
|
||||
|
||||
@ -20,11 +21,18 @@ type (
|
||||
Recv() (Event, error)
|
||||
}
|
||||
|
||||
// GenericSource represents a helper struct for generating custom event sources
|
||||
GenericSource struct {
|
||||
eventID ID
|
||||
stream rpcc.Stream
|
||||
recv func(stream rpcc.Stream) (interface{}, error)
|
||||
// SourceFactory represents a function that creates a new instance of Source.
|
||||
SourceFactory func(ctx context.Context) (Source, error)
|
||||
|
||||
StreamFactory func(ctx context.Context) (rpcc.Stream, error)
|
||||
|
||||
DataStreamReceiver func(stream rpcc.Stream) (interface{}, error)
|
||||
|
||||
// StreamSource represents a helper struct for generating custom event sources
|
||||
StreamSource struct {
|
||||
eventID ID
|
||||
stream rpcc.Stream
|
||||
receiver DataStreamReceiver
|
||||
}
|
||||
)
|
||||
|
||||
@ -32,36 +40,36 @@ var (
|
||||
Error = New("error")
|
||||
)
|
||||
|
||||
// NewSource create a new custom event source
|
||||
// NewStreamSource create a new custom event source based on rpcc.Stream
|
||||
// eventID - is a unique event ID
|
||||
// stream - is a custom event stream
|
||||
// recv - is a value conversion function
|
||||
func NewSource(
|
||||
// receiver - is a value conversion function
|
||||
func NewStreamSource(
|
||||
eventID ID,
|
||||
stream rpcc.Stream,
|
||||
recv func(stream rpcc.Stream) (interface{}, error),
|
||||
receiver DataStreamReceiver,
|
||||
) Source {
|
||||
return &GenericSource{eventID, stream, recv}
|
||||
return &StreamSource{eventID, stream, receiver}
|
||||
}
|
||||
|
||||
func (src *GenericSource) EventID() ID {
|
||||
func (src *StreamSource) ID() ID {
|
||||
return src.eventID
|
||||
}
|
||||
|
||||
func (src *GenericSource) Ready() <-chan struct{} {
|
||||
func (src *StreamSource) Ready() <-chan struct{} {
|
||||
return src.stream.Ready()
|
||||
}
|
||||
|
||||
func (src *GenericSource) RecvMsg(m interface{}) error {
|
||||
func (src *StreamSource) RecvMsg(m interface{}) error {
|
||||
return src.stream.RecvMsg(m)
|
||||
}
|
||||
|
||||
func (src *GenericSource) Close() error {
|
||||
func (src *StreamSource) Close() error {
|
||||
return src.stream.Close()
|
||||
}
|
||||
|
||||
func (src *GenericSource) Recv() (Event, error) {
|
||||
data, err := src.recv(src.stream)
|
||||
func (src *StreamSource) Recv() (Event, error) {
|
||||
data, err := src.receiver(src.stream)
|
||||
|
||||
if err != nil {
|
||||
return Event{}, err
|
||||
@ -72,3 +80,15 @@ func (src *GenericSource) Recv() (Event, error) {
|
||||
Data: data,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func NewStreamSourceFactory(eventID ID, factory StreamFactory, receiver DataStreamReceiver) SourceFactory {
|
||||
return func(ctx context.Context) (Source, error) {
|
||||
stream, err := factory(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return NewStreamSource(eventID, stream, receiver), nil
|
||||
}
|
||||
}
|
||||
|
@ -1,90 +0,0 @@
|
||||
package events
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"github.com/MontFerret/ferret/pkg/runtime/core"
|
||||
)
|
||||
|
||||
type SourceCollection struct {
|
||||
mu sync.RWMutex
|
||||
values []Source
|
||||
}
|
||||
|
||||
func NewSourceCollection() *SourceCollection {
|
||||
sc := new(SourceCollection)
|
||||
sc.values = make([]Source, 0, 10)
|
||||
|
||||
return sc
|
||||
}
|
||||
|
||||
func (sc *SourceCollection) Close() error {
|
||||
sc.mu.Lock()
|
||||
defer sc.mu.Unlock()
|
||||
|
||||
if sc.values == nil {
|
||||
return errors.New("sources are already closed")
|
||||
}
|
||||
|
||||
errs := make([]error, 0, len(sc.values))
|
||||
|
||||
for _, e := range sc.values {
|
||||
if err := e.Close(); err != nil {
|
||||
errs = append(errs, err)
|
||||
}
|
||||
}
|
||||
|
||||
sc.values = nil
|
||||
|
||||
if len(errs) > 0 {
|
||||
return core.Errors(errs...)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (sc *SourceCollection) Size() int {
|
||||
sc.mu.RLock()
|
||||
defer sc.mu.RUnlock()
|
||||
|
||||
return len(sc.values)
|
||||
}
|
||||
|
||||
func (sc *SourceCollection) Get(idx int) (Source, error) {
|
||||
sc.mu.RLock()
|
||||
defer sc.mu.RUnlock()
|
||||
|
||||
if len(sc.values) <= idx {
|
||||
return nil, core.ErrNotFound
|
||||
}
|
||||
|
||||
return sc.values[idx], nil
|
||||
}
|
||||
|
||||
func (sc *SourceCollection) Add(source Source) {
|
||||
sc.mu.Lock()
|
||||
defer sc.mu.Unlock()
|
||||
|
||||
sc.values = append(sc.values, source)
|
||||
}
|
||||
|
||||
func (sc *SourceCollection) Remove(source Source) bool {
|
||||
sc.mu.Lock()
|
||||
defer sc.mu.Unlock()
|
||||
|
||||
idx := -1
|
||||
|
||||
for i, current := range sc.values {
|
||||
if current == source {
|
||||
idx = i
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if idx > -1 {
|
||||
sc.values = append(sc.values[:idx], sc.values[idx+1:]...)
|
||||
}
|
||||
|
||||
return idx > -1
|
||||
}
|
@ -3,7 +3,6 @@ package network
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/logging"
|
||||
"regexp"
|
||||
"sync"
|
||||
|
||||
@ -21,6 +20,7 @@ import (
|
||||
"github.com/MontFerret/ferret/pkg/drivers/cdp/events"
|
||||
"github.com/MontFerret/ferret/pkg/drivers/cdp/templates"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/core"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/logging"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/values"
|
||||
)
|
||||
|
||||
@ -35,7 +35,9 @@ type (
|
||||
client *cdp.Client
|
||||
headers *drivers.HTTPHeaders
|
||||
foregroundLoop *events.Loop
|
||||
stopForegroundLoop context.CancelFunc
|
||||
backgroundLoop *events.Loop
|
||||
stopBackgroundLoop context.CancelFunc
|
||||
cancel context.CancelFunc
|
||||
responseListenerID events.ListenerID
|
||||
filterListenerID events.ListenerID
|
||||
@ -54,7 +56,6 @@ func New(
|
||||
m.logger = logging.WithName(logger.With(), "network_manager").Logger()
|
||||
m.client = client
|
||||
m.headers = drivers.NewHTTPHeaders()
|
||||
m.foregroundLoop = events.NewLoop()
|
||||
m.cancel = cancel
|
||||
m.response = new(sync.Map)
|
||||
|
||||
@ -62,60 +63,43 @@ func New(
|
||||
|
||||
defer func() {
|
||||
if err != nil {
|
||||
cancel()
|
||||
m.cancel()
|
||||
|
||||
if m.foregroundLoop != nil {
|
||||
if err := m.foregroundLoop.Close(); err != nil {
|
||||
m.logger.Trace().Err(err).Msg("failed to close the foreground loop during cleanup")
|
||||
}
|
||||
if m.stopForegroundLoop != nil {
|
||||
m.stopForegroundLoop()
|
||||
}
|
||||
|
||||
if m.backgroundLoop != nil {
|
||||
if err := m.backgroundLoop.Close(); err != nil {
|
||||
m.logger.Trace().Err(err).Msg("failed to close the background loop during cleanup")
|
||||
}
|
||||
if m.stopBackgroundLoop != nil {
|
||||
m.stopBackgroundLoop()
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
frameNavigatedStream, err := m.client.Page.FrameNavigated(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.foregroundLoop.AddSource(events.NewSource(eventFrameLoad, frameNavigatedStream, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(page.FrameNavigatedClient).Recv()
|
||||
}))
|
||||
|
||||
responseReceivedStream, err := m.client.Network.ResponseReceived(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.foregroundLoop.AddSource(events.NewSource(responseReceived, responseReceivedStream, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(network.ResponseReceivedClient).Recv()
|
||||
}))
|
||||
m.foregroundLoop = events.NewLoop(
|
||||
events.NewStreamSourceFactory(eventFrameLoad, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
return m.client.Page.FrameNavigated(ctx)
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(page.FrameNavigatedClient).Recv()
|
||||
}),
|
||||
events.NewStreamSourceFactory(responseReceived, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
return m.client.Network.ResponseReceived(ctx)
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(network.ResponseReceivedClient).Recv()
|
||||
}),
|
||||
)
|
||||
|
||||
m.responseListenerID = m.foregroundLoop.AddListener(responseReceived, m.onResponse)
|
||||
|
||||
if options.Filter != nil && len(options.Filter.Patterns) > 0 {
|
||||
m.backgroundLoop = events.NewLoop()
|
||||
|
||||
err = m.client.Fetch.Enable(ctx, toFetchArgs(options.Filter.Patterns))
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
requestPausedStream, err := m.client.Fetch.RequestPaused(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.backgroundLoop.AddSource(events.NewSource(requestPaused, requestPausedStream, func(stream rpcc.Stream) (interface{}, error) {
|
||||
m.backgroundLoop = events.NewLoop(events.NewStreamSourceFactory(requestPaused, func(ctx context.Context) (rpcc.Stream, error) {
|
||||
return m.client.Fetch.RequestPaused(ctx)
|
||||
}, func(stream rpcc.Stream) (interface{}, error) {
|
||||
return stream.(fetch.RequestPausedClient).Recv()
|
||||
}))
|
||||
|
||||
@ -140,20 +124,24 @@ func New(
|
||||
}
|
||||
}
|
||||
|
||||
err = m.foregroundLoop.Run(ctx)
|
||||
cancel, err = m.foregroundLoop.Run(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.stopForegroundLoop = cancel
|
||||
|
||||
if m.backgroundLoop != nil {
|
||||
// run in a separate loop in order to get higher priority
|
||||
// TODO: Consider adding support of event priorities to EventLoop
|
||||
err = m.backgroundLoop.Run(ctx)
|
||||
cancel, err = m.backgroundLoop.Run(ctx)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
m.stopBackgroundLoop = cancel
|
||||
}
|
||||
|
||||
return m, nil
|
||||
@ -170,10 +158,12 @@ func (m *Manager) Close() error {
|
||||
m.cancel = nil
|
||||
}
|
||||
|
||||
_ = m.foregroundLoop.Close()
|
||||
if m.stopForegroundLoop != nil {
|
||||
m.stopForegroundLoop()
|
||||
}
|
||||
|
||||
if m.backgroundLoop != nil {
|
||||
_ = m.backgroundLoop.Close()
|
||||
if m.stopBackgroundLoop != nil {
|
||||
m.stopBackgroundLoop()
|
||||
}
|
||||
|
||||
return nil
|
||||
|
@ -4,6 +4,7 @@ import (
|
||||
"context"
|
||||
"os"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/mafredri/cdp"
|
||||
"github.com/mafredri/cdp/protocol/fetch"
|
||||
@ -183,19 +184,16 @@ func TestManager(t *testing.T) {
|
||||
return frameNavigatedClient, nil
|
||||
}
|
||||
|
||||
responseReceivedClient := NewResponseReceivedClient()
|
||||
responseReceivedClient.On("Close", mock.Anything).Once().Return(nil)
|
||||
setExtraHTTPHeadersErr := errors.New("test error")
|
||||
responseReceivedErr := errors.New("test error")
|
||||
networkAPI := new(NetworkAPI)
|
||||
networkAPI.responseReceived = func(ctx context.Context) (network2.ResponseReceivedClient, error) {
|
||||
return responseReceivedClient, nil
|
||||
return nil, responseReceivedErr
|
||||
}
|
||||
networkAPI.setExtraHTTPHeaders = func(ctx context.Context, args *network2.SetExtraHTTPHeadersArgs) error {
|
||||
return setExtraHTTPHeadersErr
|
||||
return nil
|
||||
}
|
||||
|
||||
requestPausedClient := NewRequestPausedClient()
|
||||
requestPausedClient.On("Close", mock.Anything).Once().Return(nil)
|
||||
fetchAPI := new(FetchAPI)
|
||||
fetchAPI.enable = func(ctx context.Context, args *fetch.EnableArgs) error {
|
||||
return nil
|
||||
@ -228,8 +226,6 @@ func TestManager(t *testing.T) {
|
||||
|
||||
So(err, ShouldNotBeNil)
|
||||
frameNavigatedClient.AssertExpectations(t)
|
||||
responseReceivedClient.AssertExpectations(t)
|
||||
requestPausedClient.AssertExpectations(t)
|
||||
})
|
||||
|
||||
Convey("Should close all resources on Close", func() {
|
||||
@ -286,6 +282,8 @@ func TestManager(t *testing.T) {
|
||||
So(err, ShouldBeNil)
|
||||
So(mgr.Close(), ShouldBeNil)
|
||||
|
||||
time.Sleep(time.Duration(100) * time.Millisecond)
|
||||
|
||||
frameNavigatedClient.AssertExpectations(t)
|
||||
responseReceivedClient.AssertExpectations(t)
|
||||
requestPausedClient.AssertExpectations(t)
|
||||
|
@ -7,18 +7,31 @@ type (
|
||||
*drivers.Options
|
||||
Address string
|
||||
KeepCookies bool
|
||||
Connection *ConnectionOptions
|
||||
}
|
||||
|
||||
ConnectionOptions struct {
|
||||
BufferSize int
|
||||
Compression bool
|
||||
}
|
||||
|
||||
Option func(opts *Options)
|
||||
)
|
||||
|
||||
const DefaultAddress = "http://127.0.0.1:9222"
|
||||
const (
|
||||
DefaultAddress = "http://127.0.0.1:9222"
|
||||
DefaultBufferSize = 1048562
|
||||
)
|
||||
|
||||
func NewOptions(setters []Option) *Options {
|
||||
opts := new(Options)
|
||||
opts.Options = new(drivers.Options)
|
||||
opts.Name = DriverName
|
||||
opts.Address = DefaultAddress
|
||||
opts.Connection = &ConnectionOptions{
|
||||
BufferSize: DefaultBufferSize,
|
||||
Compression: true,
|
||||
}
|
||||
|
||||
for _, setter := range setters {
|
||||
setter(opts)
|
||||
@ -82,3 +95,21 @@ func WithCookies(cookies []drivers.HTTPCookie) Option {
|
||||
drivers.WithCookies(cookies)(opts.Options)
|
||||
}
|
||||
}
|
||||
|
||||
func WithBufferSize(size int) Option {
|
||||
return func(opts *Options) {
|
||||
opts.Connection.BufferSize = size
|
||||
}
|
||||
}
|
||||
|
||||
func WithCompression() Option {
|
||||
return func(opts *Options) {
|
||||
opts.Connection.Compression = true
|
||||
}
|
||||
}
|
||||
|
||||
func WithNoCompression() Option {
|
||||
return func(opts *Options) {
|
||||
opts.Connection.Compression = false
|
||||
}
|
||||
}
|
||||
|
@ -1,16 +1,24 @@
|
||||
package drivers
|
||||
|
||||
import (
|
||||
"context"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/core"
|
||||
)
|
||||
|
||||
type Container struct {
|
||||
drivers map[string]Driver
|
||||
}
|
||||
type (
|
||||
DriverEntry struct {
|
||||
Driver Driver
|
||||
Options []GlobalOption
|
||||
}
|
||||
|
||||
Container struct {
|
||||
drivers map[string]DriverEntry
|
||||
}
|
||||
)
|
||||
|
||||
func NewContainer() *Container {
|
||||
return &Container{
|
||||
drivers: map[string]Driver{},
|
||||
drivers: map[string]DriverEntry{},
|
||||
}
|
||||
}
|
||||
|
||||
@ -20,7 +28,7 @@ func (c *Container) Has(name string) bool {
|
||||
return exists
|
||||
}
|
||||
|
||||
func (c *Container) Register(drv Driver) error {
|
||||
func (c *Container) Register(drv Driver, opts ...GlobalOption) error {
|
||||
if drv == nil {
|
||||
return core.Error(core.ErrMissedArgument, "driver")
|
||||
}
|
||||
@ -32,7 +40,10 @@ func (c *Container) Register(drv Driver) error {
|
||||
return core.Errorf(core.ErrNotUnique, "driver: %s", name)
|
||||
}
|
||||
|
||||
c.drivers[name] = drv
|
||||
c.drivers[name] = DriverEntry{
|
||||
Driver: drv,
|
||||
Options: opts,
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
@ -44,15 +55,25 @@ func (c *Container) Remove(name string) {
|
||||
func (c *Container) Get(name string) (Driver, bool) {
|
||||
found, exists := c.drivers[name]
|
||||
|
||||
return found, exists
|
||||
return found.Driver, exists
|
||||
}
|
||||
|
||||
func (c *Container) GetAll() []Driver {
|
||||
res := make([]Driver, 0, len(c.drivers))
|
||||
|
||||
for _, drv := range c.drivers {
|
||||
res = append(res, drv)
|
||||
for _, entry := range c.drivers {
|
||||
res = append(res, entry.Driver)
|
||||
}
|
||||
|
||||
return res
|
||||
}
|
||||
|
||||
func (c *Container) WithContext(ctx context.Context) context.Context {
|
||||
next := ctx
|
||||
|
||||
for _, entry := range c.drivers {
|
||||
next = withContext(next, entry.Driver, entry.Options)
|
||||
}
|
||||
|
||||
return next
|
||||
}
|
||||
|
@ -24,20 +24,7 @@ type (
|
||||
)
|
||||
|
||||
func WithContext(ctx context.Context, drv Driver, opts ...GlobalOption) context.Context {
|
||||
ctx, value := resolveValue(ctx)
|
||||
|
||||
value.drivers[drv.Name()] = drv
|
||||
|
||||
for _, opt := range opts {
|
||||
opt(drv, value.opts)
|
||||
}
|
||||
|
||||
// set first registered driver as a default one
|
||||
if value.opts.defaultDriver == "" {
|
||||
value.opts.defaultDriver = drv.Name()
|
||||
}
|
||||
|
||||
return ctx
|
||||
return withContext(ctx, drv, opts)
|
||||
}
|
||||
|
||||
func FromContext(ctx context.Context, name string) (Driver, error) {
|
||||
@ -56,6 +43,23 @@ func FromContext(ctx context.Context, name string) (Driver, error) {
|
||||
return drv, nil
|
||||
}
|
||||
|
||||
func withContext(ctx context.Context, drv Driver, opts []GlobalOption) context.Context {
|
||||
ctx, value := resolveValue(ctx)
|
||||
|
||||
value.drivers[drv.Name()] = drv
|
||||
|
||||
for _, opt := range opts {
|
||||
opt(drv, value.opts)
|
||||
}
|
||||
|
||||
// set first registered driver as a default one
|
||||
if value.opts.defaultDriver == "" {
|
||||
value.opts.defaultDriver = drv.Name()
|
||||
}
|
||||
|
||||
return ctx
|
||||
}
|
||||
|
||||
func resolveValue(ctx context.Context) (context.Context, *ctxValue) {
|
||||
key := ctxKey{}
|
||||
v := ctx.Value(key)
|
||||
|
@ -5,7 +5,7 @@ parser grammar FqlParser;
|
||||
options { tokenVocab=FqlLexer; }
|
||||
|
||||
program
|
||||
: head* body EOF
|
||||
: head* body
|
||||
;
|
||||
|
||||
head
|
||||
@ -20,10 +20,6 @@ use
|
||||
: Use namespaceIdentifier
|
||||
;
|
||||
|
||||
namespaceIdentifier
|
||||
: namespace Identifier
|
||||
;
|
||||
|
||||
body
|
||||
: bodyStatement* bodyExpression
|
||||
;
|
||||
@ -39,29 +35,15 @@ bodyExpression
|
||||
| forExpression
|
||||
;
|
||||
|
||||
variableDeclaration
|
||||
: Let Identifier Assign expression
|
||||
| Let IgnoreIdentifier Assign expression
|
||||
;
|
||||
|
||||
returnExpression
|
||||
: Return Distinct? expression
|
||||
;
|
||||
|
||||
inlineHighLevelExpression
|
||||
: OpenParen highLevelExpression CloseParen errorOperator?
|
||||
;
|
||||
|
||||
highLevelExpression
|
||||
: forExpression
|
||||
| waitForExpression
|
||||
: Return (Distinct)? expression
|
||||
;
|
||||
|
||||
forExpression
|
||||
: For Identifier (Comma Identifier)? In forExpressionSource
|
||||
: For valueVariable=(Identifier | IgnoreIdentifier) (Comma counterVariable=Identifier)? In forExpressionSource
|
||||
forExpressionBody*
|
||||
forExpressionReturn
|
||||
| For Identifier Do? While expression
|
||||
| For counterVariable=(Identifier | IgnoreIdentifier) Do? While expression
|
||||
forExpressionBody*
|
||||
forExpressionReturn
|
||||
;
|
||||
@ -153,21 +135,10 @@ collectCounter
|
||||
: With Count Into Identifier
|
||||
;
|
||||
|
||||
optionsClause
|
||||
: Options objectLiteral
|
||||
| Options variable
|
||||
;
|
||||
|
||||
waitForExpression
|
||||
: Waitfor Event waitForEventName In waitForEventSource (optionsClause)? (waitForTimeout)?
|
||||
;
|
||||
|
||||
waitForTimeout
|
||||
: integerLiteral
|
||||
| variable
|
||||
| param
|
||||
;
|
||||
|
||||
waitForEventName
|
||||
: stringLiteral
|
||||
| variable
|
||||
@ -182,34 +153,47 @@ waitForEventSource
|
||||
| memberExpression
|
||||
;
|
||||
|
||||
rangeOperator
|
||||
: (integerLiteral | variable | param) Range (integerLiteral | variable | param)
|
||||
optionsClause
|
||||
: Options objectLiteral
|
||||
;
|
||||
|
||||
waitForTimeout
|
||||
: integerLiteral
|
||||
| variable
|
||||
| param
|
||||
;
|
||||
|
||||
variableDeclaration
|
||||
: Let Identifier Assign expression
|
||||
| Let IgnoreIdentifier Assign expression
|
||||
;
|
||||
|
||||
param
|
||||
: Param Identifier
|
||||
;
|
||||
|
||||
variable
|
||||
: Identifier
|
||||
;
|
||||
|
||||
literal
|
||||
: arrayLiteral
|
||||
| objectLiteral
|
||||
| booleanLiteral
|
||||
| stringLiteral
|
||||
| floatLiteral
|
||||
| integerLiteral
|
||||
| noneLiteral
|
||||
;
|
||||
|
||||
arrayLiteral
|
||||
: OpenBracket (expression (Comma expression)* Comma?)? CloseBracket
|
||||
: OpenBracket argumentList? CloseBracket
|
||||
;
|
||||
|
||||
objectLiteral
|
||||
: OpenBrace (propertyAssignment (Comma propertyAssignment)* Comma?)? CloseBrace
|
||||
;
|
||||
|
||||
propertyAssignment
|
||||
: propertyName Colon expression
|
||||
| computedPropertyName Colon expression
|
||||
| variable
|
||||
;
|
||||
|
||||
computedPropertyName
|
||||
: OpenBracket expression CloseBracket
|
||||
;
|
||||
|
||||
propertyName
|
||||
: Identifier
|
||||
| stringLiteral
|
||||
| param
|
||||
;
|
||||
|
||||
booleanLiteral
|
||||
: BooleanLiteral
|
||||
;
|
||||
@ -231,36 +215,28 @@ noneLiteral
|
||||
| None
|
||||
;
|
||||
|
||||
expression
|
||||
: unaryOperator expression
|
||||
| expression multiplicativeOperator expression
|
||||
| expression additiveOperator expression
|
||||
| expression arrayOperator (inOperator | equalityOperator) expression
|
||||
| expression inOperator expression
|
||||
| expression likeOperator expression
|
||||
| expression equalityOperator expression
|
||||
| expression regexpOperator expression
|
||||
| expression logicalAndOperator expression
|
||||
| expression logicalOrOperator expression
|
||||
| expression QuestionMark expression? Colon expression
|
||||
| rangeOperator
|
||||
| stringLiteral
|
||||
| floatLiteral
|
||||
| integerLiteral
|
||||
| booleanLiteral
|
||||
| arrayLiteral
|
||||
| objectLiteral
|
||||
| memberExpression
|
||||
| functionCallExpression
|
||||
| param
|
||||
propertyAssignment
|
||||
: propertyName Colon expression
|
||||
| computedPropertyName Colon expression
|
||||
| variable
|
||||
| noneLiteral
|
||||
| expressionGroup
|
||||
| inlineHighLevelExpression
|
||||
;
|
||||
|
||||
expressionGroup
|
||||
: OpenParen expression CloseParen errorOperator?
|
||||
computedPropertyName
|
||||
: OpenBracket expression CloseBracket
|
||||
;
|
||||
|
||||
propertyName
|
||||
: Identifier
|
||||
| stringLiteral
|
||||
| param
|
||||
;
|
||||
|
||||
namespaceIdentifier
|
||||
: namespace Identifier
|
||||
;
|
||||
|
||||
namespace
|
||||
: NamespaceSegment*
|
||||
;
|
||||
|
||||
memberExpression
|
||||
@ -275,21 +251,21 @@ memberExpressionSource
|
||||
| functionCall
|
||||
;
|
||||
|
||||
functionCall
|
||||
: namespace functionIdentifier arguments
|
||||
;
|
||||
|
||||
functionCallExpression
|
||||
: functionCall errorOperator?
|
||||
;
|
||||
|
||||
memberExpressionPath
|
||||
: QuestionMark? Dot propertyName
|
||||
| (QuestionMark Dot)? computedPropertyName
|
||||
functionCall
|
||||
: namespace functionIdentifier OpenParen argumentList? CloseParen
|
||||
;
|
||||
|
||||
errorOperator
|
||||
: QuestionMark
|
||||
argumentList
|
||||
: expression (Comma expression)* Comma?
|
||||
;
|
||||
|
||||
memberExpressionPath
|
||||
: errorOperator? Dot propertyName
|
||||
| (errorOperator Dot)? computedPropertyName
|
||||
;
|
||||
|
||||
functionIdentifier
|
||||
@ -323,28 +299,47 @@ functionIdentifier
|
||||
| Event
|
||||
;
|
||||
|
||||
namespace
|
||||
: NamespaceSegment*
|
||||
rangeOperator
|
||||
: left=rangeOperand Range right=rangeOperand
|
||||
;
|
||||
|
||||
arguments
|
||||
: OpenParen (expression (Comma expression)*)? CloseParen
|
||||
rangeOperand
|
||||
: integerLiteral
|
||||
| variable
|
||||
| param
|
||||
;
|
||||
|
||||
expression
|
||||
: unaryOperator right=expression
|
||||
| condition=expression ternaryOperator=QuestionMark onTrue=expression? Colon onFalse=expression
|
||||
| left=expression logicalAndOperator right=expression
|
||||
| left=expression logicalOrOperator right=expression
|
||||
| predicate
|
||||
;
|
||||
|
||||
predicate
|
||||
: left=predicate equalityOperator right=predicate
|
||||
| left=predicate arrayOperator right=predicate
|
||||
| left=predicate inOperator right=predicate
|
||||
| left=predicate likeOperator right=predicate
|
||||
| expressionAtom
|
||||
;
|
||||
|
||||
expressionAtom
|
||||
: left=expressionAtom multiplicativeOperator right=expressionAtom
|
||||
| left=expressionAtom additiveOperator right=expressionAtom
|
||||
| left=expressionAtom regexpOperator right=expressionAtom
|
||||
| functionCallExpression
|
||||
| rangeOperator
|
||||
| literal
|
||||
| variable
|
||||
| memberExpression
|
||||
| param
|
||||
| OpenParen (forExpression | waitForExpression | expression) CloseParen errorOperator?
|
||||
;
|
||||
|
||||
arrayOperator
|
||||
: All
|
||||
| Any
|
||||
| None
|
||||
;
|
||||
|
||||
inOperator
|
||||
: In
|
||||
| Not In
|
||||
;
|
||||
|
||||
likeOperator
|
||||
: Like
|
||||
| Not Like
|
||||
: operator=(All | Any | None) (inOperator | equalityOperator)
|
||||
;
|
||||
|
||||
equalityOperator
|
||||
@ -356,6 +351,20 @@ equalityOperator
|
||||
| Neq
|
||||
;
|
||||
|
||||
inOperator
|
||||
: Not? In
|
||||
;
|
||||
|
||||
likeOperator
|
||||
: Not? Like
|
||||
;
|
||||
|
||||
unaryOperator
|
||||
: Not
|
||||
| Plus
|
||||
| Minus
|
||||
;
|
||||
|
||||
regexpOperator
|
||||
: RegexMatch
|
||||
| RegexNotMatch
|
||||
@ -380,16 +389,6 @@ additiveOperator
|
||||
| Minus
|
||||
;
|
||||
|
||||
unaryOperator
|
||||
: Not
|
||||
| Plus
|
||||
| Minus
|
||||
;
|
||||
|
||||
param
|
||||
: Param Identifier
|
||||
;
|
||||
|
||||
variable
|
||||
: Identifier
|
||||
errorOperator
|
||||
: QuestionMark
|
||||
;
|
37
pkg/parser/case_changing_stream.go
Normal file
37
pkg/parser/case_changing_stream.go
Normal file
@ -0,0 +1,37 @@
|
||||
package parser
|
||||
|
||||
import (
|
||||
"unicode"
|
||||
|
||||
"github.com/antlr/antlr4/runtime/Go/antlr"
|
||||
)
|
||||
|
||||
// CaseChangingStream wraps an existing CharStream, but upper cases, or
|
||||
// lower cases the input before it is tokenized.
|
||||
type CaseChangingStream struct {
|
||||
antlr.CharStream
|
||||
|
||||
upper bool
|
||||
}
|
||||
|
||||
// newCaseChangingStream returns a new CaseChangingStream that forces
|
||||
// all tokens read from the underlying stream to be either upper case
|
||||
// or lower case based on the upper argument.
|
||||
func newCaseChangingStream(in antlr.CharStream, upper bool) *CaseChangingStream {
|
||||
return &CaseChangingStream{in, upper}
|
||||
}
|
||||
|
||||
// LA gets the value of the symbol at offset from the current position
|
||||
// from the underlying CharStream and converts it to either upper case
|
||||
// or lower case.
|
||||
func (is *CaseChangingStream) LA(offset int) int {
|
||||
in := is.CharStream.LA(offset)
|
||||
if in < 0 {
|
||||
// Such as antlr.TokenEOF which is -1
|
||||
return in
|
||||
}
|
||||
if is.upper {
|
||||
return int(unicode.ToUpper(rune(in)))
|
||||
}
|
||||
return int(unicode.ToLower(rune(in)))
|
||||
}
|
File diff suppressed because one or more lines are too long
File diff suppressed because it is too large
Load Diff
@ -44,12 +44,6 @@ func (s *BaseFqlParserListener) EnterUse(ctx *UseContext) {}
|
||||
// ExitUse is called when production use is exited.
|
||||
func (s *BaseFqlParserListener) ExitUse(ctx *UseContext) {}
|
||||
|
||||
// EnterNamespaceIdentifier is called when production namespaceIdentifier is entered.
|
||||
func (s *BaseFqlParserListener) EnterNamespaceIdentifier(ctx *NamespaceIdentifierContext) {}
|
||||
|
||||
// ExitNamespaceIdentifier is called when production namespaceIdentifier is exited.
|
||||
func (s *BaseFqlParserListener) ExitNamespaceIdentifier(ctx *NamespaceIdentifierContext) {}
|
||||
|
||||
// EnterBody is called when production body is entered.
|
||||
func (s *BaseFqlParserListener) EnterBody(ctx *BodyContext) {}
|
||||
|
||||
@ -68,32 +62,12 @@ func (s *BaseFqlParserListener) EnterBodyExpression(ctx *BodyExpressionContext)
|
||||
// ExitBodyExpression is called when production bodyExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitBodyExpression(ctx *BodyExpressionContext) {}
|
||||
|
||||
// EnterVariableDeclaration is called when production variableDeclaration is entered.
|
||||
func (s *BaseFqlParserListener) EnterVariableDeclaration(ctx *VariableDeclarationContext) {}
|
||||
|
||||
// ExitVariableDeclaration is called when production variableDeclaration is exited.
|
||||
func (s *BaseFqlParserListener) ExitVariableDeclaration(ctx *VariableDeclarationContext) {}
|
||||
|
||||
// EnterReturnExpression is called when production returnExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterReturnExpression(ctx *ReturnExpressionContext) {}
|
||||
|
||||
// ExitReturnExpression is called when production returnExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitReturnExpression(ctx *ReturnExpressionContext) {}
|
||||
|
||||
// EnterInlineHighLevelExpression is called when production inlineHighLevelExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterInlineHighLevelExpression(ctx *InlineHighLevelExpressionContext) {
|
||||
}
|
||||
|
||||
// ExitInlineHighLevelExpression is called when production inlineHighLevelExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitInlineHighLevelExpression(ctx *InlineHighLevelExpressionContext) {
|
||||
}
|
||||
|
||||
// EnterHighLevelExpression is called when production highLevelExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterHighLevelExpression(ctx *HighLevelExpressionContext) {}
|
||||
|
||||
// ExitHighLevelExpression is called when production highLevelExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitHighLevelExpression(ctx *HighLevelExpressionContext) {}
|
||||
|
||||
// EnterForExpression is called when production forExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterForExpression(ctx *ForExpressionContext) {}
|
||||
|
||||
@ -202,24 +176,12 @@ func (s *BaseFqlParserListener) EnterCollectCounter(ctx *CollectCounterContext)
|
||||
// ExitCollectCounter is called when production collectCounter is exited.
|
||||
func (s *BaseFqlParserListener) ExitCollectCounter(ctx *CollectCounterContext) {}
|
||||
|
||||
// EnterOptionsClause is called when production optionsClause is entered.
|
||||
func (s *BaseFqlParserListener) EnterOptionsClause(ctx *OptionsClauseContext) {}
|
||||
|
||||
// ExitOptionsClause is called when production optionsClause is exited.
|
||||
func (s *BaseFqlParserListener) ExitOptionsClause(ctx *OptionsClauseContext) {}
|
||||
|
||||
// EnterWaitForExpression is called when production waitForExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterWaitForExpression(ctx *WaitForExpressionContext) {}
|
||||
|
||||
// ExitWaitForExpression is called when production waitForExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitWaitForExpression(ctx *WaitForExpressionContext) {}
|
||||
|
||||
// EnterWaitForTimeout is called when production waitForTimeout is entered.
|
||||
func (s *BaseFqlParserListener) EnterWaitForTimeout(ctx *WaitForTimeoutContext) {}
|
||||
|
||||
// ExitWaitForTimeout is called when production waitForTimeout is exited.
|
||||
func (s *BaseFqlParserListener) ExitWaitForTimeout(ctx *WaitForTimeoutContext) {}
|
||||
|
||||
// EnterWaitForEventName is called when production waitForEventName is entered.
|
||||
func (s *BaseFqlParserListener) EnterWaitForEventName(ctx *WaitForEventNameContext) {}
|
||||
|
||||
@ -232,11 +194,41 @@ func (s *BaseFqlParserListener) EnterWaitForEventSource(ctx *WaitForEventSourceC
|
||||
// ExitWaitForEventSource is called when production waitForEventSource is exited.
|
||||
func (s *BaseFqlParserListener) ExitWaitForEventSource(ctx *WaitForEventSourceContext) {}
|
||||
|
||||
// EnterRangeOperator is called when production rangeOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterRangeOperator(ctx *RangeOperatorContext) {}
|
||||
// EnterOptionsClause is called when production optionsClause is entered.
|
||||
func (s *BaseFqlParserListener) EnterOptionsClause(ctx *OptionsClauseContext) {}
|
||||
|
||||
// ExitRangeOperator is called when production rangeOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitRangeOperator(ctx *RangeOperatorContext) {}
|
||||
// ExitOptionsClause is called when production optionsClause is exited.
|
||||
func (s *BaseFqlParserListener) ExitOptionsClause(ctx *OptionsClauseContext) {}
|
||||
|
||||
// EnterWaitForTimeout is called when production waitForTimeout is entered.
|
||||
func (s *BaseFqlParserListener) EnterWaitForTimeout(ctx *WaitForTimeoutContext) {}
|
||||
|
||||
// ExitWaitForTimeout is called when production waitForTimeout is exited.
|
||||
func (s *BaseFqlParserListener) ExitWaitForTimeout(ctx *WaitForTimeoutContext) {}
|
||||
|
||||
// EnterVariableDeclaration is called when production variableDeclaration is entered.
|
||||
func (s *BaseFqlParserListener) EnterVariableDeclaration(ctx *VariableDeclarationContext) {}
|
||||
|
||||
// ExitVariableDeclaration is called when production variableDeclaration is exited.
|
||||
func (s *BaseFqlParserListener) ExitVariableDeclaration(ctx *VariableDeclarationContext) {}
|
||||
|
||||
// EnterParam is called when production param is entered.
|
||||
func (s *BaseFqlParserListener) EnterParam(ctx *ParamContext) {}
|
||||
|
||||
// ExitParam is called when production param is exited.
|
||||
func (s *BaseFqlParserListener) ExitParam(ctx *ParamContext) {}
|
||||
|
||||
// EnterVariable is called when production variable is entered.
|
||||
func (s *BaseFqlParserListener) EnterVariable(ctx *VariableContext) {}
|
||||
|
||||
// ExitVariable is called when production variable is exited.
|
||||
func (s *BaseFqlParserListener) ExitVariable(ctx *VariableContext) {}
|
||||
|
||||
// EnterLiteral is called when production literal is entered.
|
||||
func (s *BaseFqlParserListener) EnterLiteral(ctx *LiteralContext) {}
|
||||
|
||||
// ExitLiteral is called when production literal is exited.
|
||||
func (s *BaseFqlParserListener) ExitLiteral(ctx *LiteralContext) {}
|
||||
|
||||
// EnterArrayLiteral is called when production arrayLiteral is entered.
|
||||
func (s *BaseFqlParserListener) EnterArrayLiteral(ctx *ArrayLiteralContext) {}
|
||||
@ -250,24 +242,6 @@ func (s *BaseFqlParserListener) EnterObjectLiteral(ctx *ObjectLiteralContext) {}
|
||||
// ExitObjectLiteral is called when production objectLiteral is exited.
|
||||
func (s *BaseFqlParserListener) ExitObjectLiteral(ctx *ObjectLiteralContext) {}
|
||||
|
||||
// EnterPropertyAssignment is called when production propertyAssignment is entered.
|
||||
func (s *BaseFqlParserListener) EnterPropertyAssignment(ctx *PropertyAssignmentContext) {}
|
||||
|
||||
// ExitPropertyAssignment is called when production propertyAssignment is exited.
|
||||
func (s *BaseFqlParserListener) ExitPropertyAssignment(ctx *PropertyAssignmentContext) {}
|
||||
|
||||
// EnterComputedPropertyName is called when production computedPropertyName is entered.
|
||||
func (s *BaseFqlParserListener) EnterComputedPropertyName(ctx *ComputedPropertyNameContext) {}
|
||||
|
||||
// ExitComputedPropertyName is called when production computedPropertyName is exited.
|
||||
func (s *BaseFqlParserListener) ExitComputedPropertyName(ctx *ComputedPropertyNameContext) {}
|
||||
|
||||
// EnterPropertyName is called when production propertyName is entered.
|
||||
func (s *BaseFqlParserListener) EnterPropertyName(ctx *PropertyNameContext) {}
|
||||
|
||||
// ExitPropertyName is called when production propertyName is exited.
|
||||
func (s *BaseFqlParserListener) ExitPropertyName(ctx *PropertyNameContext) {}
|
||||
|
||||
// EnterBooleanLiteral is called when production booleanLiteral is entered.
|
||||
func (s *BaseFqlParserListener) EnterBooleanLiteral(ctx *BooleanLiteralContext) {}
|
||||
|
||||
@ -298,17 +272,35 @@ func (s *BaseFqlParserListener) EnterNoneLiteral(ctx *NoneLiteralContext) {}
|
||||
// ExitNoneLiteral is called when production noneLiteral is exited.
|
||||
func (s *BaseFqlParserListener) ExitNoneLiteral(ctx *NoneLiteralContext) {}
|
||||
|
||||
// EnterExpression is called when production expression is entered.
|
||||
func (s *BaseFqlParserListener) EnterExpression(ctx *ExpressionContext) {}
|
||||
// EnterPropertyAssignment is called when production propertyAssignment is entered.
|
||||
func (s *BaseFqlParserListener) EnterPropertyAssignment(ctx *PropertyAssignmentContext) {}
|
||||
|
||||
// ExitExpression is called when production expression is exited.
|
||||
func (s *BaseFqlParserListener) ExitExpression(ctx *ExpressionContext) {}
|
||||
// ExitPropertyAssignment is called when production propertyAssignment is exited.
|
||||
func (s *BaseFqlParserListener) ExitPropertyAssignment(ctx *PropertyAssignmentContext) {}
|
||||
|
||||
// EnterExpressionGroup is called when production expressionGroup is entered.
|
||||
func (s *BaseFqlParserListener) EnterExpressionGroup(ctx *ExpressionGroupContext) {}
|
||||
// EnterComputedPropertyName is called when production computedPropertyName is entered.
|
||||
func (s *BaseFqlParserListener) EnterComputedPropertyName(ctx *ComputedPropertyNameContext) {}
|
||||
|
||||
// ExitExpressionGroup is called when production expressionGroup is exited.
|
||||
func (s *BaseFqlParserListener) ExitExpressionGroup(ctx *ExpressionGroupContext) {}
|
||||
// ExitComputedPropertyName is called when production computedPropertyName is exited.
|
||||
func (s *BaseFqlParserListener) ExitComputedPropertyName(ctx *ComputedPropertyNameContext) {}
|
||||
|
||||
// EnterPropertyName is called when production propertyName is entered.
|
||||
func (s *BaseFqlParserListener) EnterPropertyName(ctx *PropertyNameContext) {}
|
||||
|
||||
// ExitPropertyName is called when production propertyName is exited.
|
||||
func (s *BaseFqlParserListener) ExitPropertyName(ctx *PropertyNameContext) {}
|
||||
|
||||
// EnterNamespaceIdentifier is called when production namespaceIdentifier is entered.
|
||||
func (s *BaseFqlParserListener) EnterNamespaceIdentifier(ctx *NamespaceIdentifierContext) {}
|
||||
|
||||
// ExitNamespaceIdentifier is called when production namespaceIdentifier is exited.
|
||||
func (s *BaseFqlParserListener) ExitNamespaceIdentifier(ctx *NamespaceIdentifierContext) {}
|
||||
|
||||
// EnterNamespace is called when production namespace is entered.
|
||||
func (s *BaseFqlParserListener) EnterNamespace(ctx *NamespaceContext) {}
|
||||
|
||||
// ExitNamespace is called when production namespace is exited.
|
||||
func (s *BaseFqlParserListener) ExitNamespace(ctx *NamespaceContext) {}
|
||||
|
||||
// EnterMemberExpression is called when production memberExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterMemberExpression(ctx *MemberExpressionContext) {}
|
||||
@ -322,17 +314,23 @@ func (s *BaseFqlParserListener) EnterMemberExpressionSource(ctx *MemberExpressio
|
||||
// ExitMemberExpressionSource is called when production memberExpressionSource is exited.
|
||||
func (s *BaseFqlParserListener) ExitMemberExpressionSource(ctx *MemberExpressionSourceContext) {}
|
||||
|
||||
// EnterFunctionCallExpression is called when production functionCallExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterFunctionCallExpression(ctx *FunctionCallExpressionContext) {}
|
||||
|
||||
// ExitFunctionCallExpression is called when production functionCallExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitFunctionCallExpression(ctx *FunctionCallExpressionContext) {}
|
||||
|
||||
// EnterFunctionCall is called when production functionCall is entered.
|
||||
func (s *BaseFqlParserListener) EnterFunctionCall(ctx *FunctionCallContext) {}
|
||||
|
||||
// ExitFunctionCall is called when production functionCall is exited.
|
||||
func (s *BaseFqlParserListener) ExitFunctionCall(ctx *FunctionCallContext) {}
|
||||
|
||||
// EnterFunctionCallExpression is called when production functionCallExpression is entered.
|
||||
func (s *BaseFqlParserListener) EnterFunctionCallExpression(ctx *FunctionCallExpressionContext) {}
|
||||
// EnterArgumentList is called when production argumentList is entered.
|
||||
func (s *BaseFqlParserListener) EnterArgumentList(ctx *ArgumentListContext) {}
|
||||
|
||||
// ExitFunctionCallExpression is called when production functionCallExpression is exited.
|
||||
func (s *BaseFqlParserListener) ExitFunctionCallExpression(ctx *FunctionCallExpressionContext) {}
|
||||
// ExitArgumentList is called when production argumentList is exited.
|
||||
func (s *BaseFqlParserListener) ExitArgumentList(ctx *ArgumentListContext) {}
|
||||
|
||||
// EnterMemberExpressionPath is called when production memberExpressionPath is entered.
|
||||
func (s *BaseFqlParserListener) EnterMemberExpressionPath(ctx *MemberExpressionPathContext) {}
|
||||
@ -340,29 +338,41 @@ func (s *BaseFqlParserListener) EnterMemberExpressionPath(ctx *MemberExpressionP
|
||||
// ExitMemberExpressionPath is called when production memberExpressionPath is exited.
|
||||
func (s *BaseFqlParserListener) ExitMemberExpressionPath(ctx *MemberExpressionPathContext) {}
|
||||
|
||||
// EnterErrorOperator is called when production errorOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterErrorOperator(ctx *ErrorOperatorContext) {}
|
||||
|
||||
// ExitErrorOperator is called when production errorOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitErrorOperator(ctx *ErrorOperatorContext) {}
|
||||
|
||||
// EnterFunctionIdentifier is called when production functionIdentifier is entered.
|
||||
func (s *BaseFqlParserListener) EnterFunctionIdentifier(ctx *FunctionIdentifierContext) {}
|
||||
|
||||
// ExitFunctionIdentifier is called when production functionIdentifier is exited.
|
||||
func (s *BaseFqlParserListener) ExitFunctionIdentifier(ctx *FunctionIdentifierContext) {}
|
||||
|
||||
// EnterNamespace is called when production namespace is entered.
|
||||
func (s *BaseFqlParserListener) EnterNamespace(ctx *NamespaceContext) {}
|
||||
// EnterRangeOperator is called when production rangeOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterRangeOperator(ctx *RangeOperatorContext) {}
|
||||
|
||||
// ExitNamespace is called when production namespace is exited.
|
||||
func (s *BaseFqlParserListener) ExitNamespace(ctx *NamespaceContext) {}
|
||||
// ExitRangeOperator is called when production rangeOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitRangeOperator(ctx *RangeOperatorContext) {}
|
||||
|
||||
// EnterArguments is called when production arguments is entered.
|
||||
func (s *BaseFqlParserListener) EnterArguments(ctx *ArgumentsContext) {}
|
||||
// EnterRangeOperand is called when production rangeOperand is entered.
|
||||
func (s *BaseFqlParserListener) EnterRangeOperand(ctx *RangeOperandContext) {}
|
||||
|
||||
// ExitArguments is called when production arguments is exited.
|
||||
func (s *BaseFqlParserListener) ExitArguments(ctx *ArgumentsContext) {}
|
||||
// ExitRangeOperand is called when production rangeOperand is exited.
|
||||
func (s *BaseFqlParserListener) ExitRangeOperand(ctx *RangeOperandContext) {}
|
||||
|
||||
// EnterExpression is called when production expression is entered.
|
||||
func (s *BaseFqlParserListener) EnterExpression(ctx *ExpressionContext) {}
|
||||
|
||||
// ExitExpression is called when production expression is exited.
|
||||
func (s *BaseFqlParserListener) ExitExpression(ctx *ExpressionContext) {}
|
||||
|
||||
// EnterPredicate is called when production predicate is entered.
|
||||
func (s *BaseFqlParserListener) EnterPredicate(ctx *PredicateContext) {}
|
||||
|
||||
// ExitPredicate is called when production predicate is exited.
|
||||
func (s *BaseFqlParserListener) ExitPredicate(ctx *PredicateContext) {}
|
||||
|
||||
// EnterExpressionAtom is called when production expressionAtom is entered.
|
||||
func (s *BaseFqlParserListener) EnterExpressionAtom(ctx *ExpressionAtomContext) {}
|
||||
|
||||
// ExitExpressionAtom is called when production expressionAtom is exited.
|
||||
func (s *BaseFqlParserListener) ExitExpressionAtom(ctx *ExpressionAtomContext) {}
|
||||
|
||||
// EnterArrayOperator is called when production arrayOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterArrayOperator(ctx *ArrayOperatorContext) {}
|
||||
@ -370,6 +380,12 @@ func (s *BaseFqlParserListener) EnterArrayOperator(ctx *ArrayOperatorContext) {}
|
||||
// ExitArrayOperator is called when production arrayOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitArrayOperator(ctx *ArrayOperatorContext) {}
|
||||
|
||||
// EnterEqualityOperator is called when production equalityOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterEqualityOperator(ctx *EqualityOperatorContext) {}
|
||||
|
||||
// ExitEqualityOperator is called when production equalityOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitEqualityOperator(ctx *EqualityOperatorContext) {}
|
||||
|
||||
// EnterInOperator is called when production inOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterInOperator(ctx *InOperatorContext) {}
|
||||
|
||||
@ -382,11 +398,11 @@ func (s *BaseFqlParserListener) EnterLikeOperator(ctx *LikeOperatorContext) {}
|
||||
// ExitLikeOperator is called when production likeOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitLikeOperator(ctx *LikeOperatorContext) {}
|
||||
|
||||
// EnterEqualityOperator is called when production equalityOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterEqualityOperator(ctx *EqualityOperatorContext) {}
|
||||
// EnterUnaryOperator is called when production unaryOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterUnaryOperator(ctx *UnaryOperatorContext) {}
|
||||
|
||||
// ExitEqualityOperator is called when production equalityOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitEqualityOperator(ctx *EqualityOperatorContext) {}
|
||||
// ExitUnaryOperator is called when production unaryOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitUnaryOperator(ctx *UnaryOperatorContext) {}
|
||||
|
||||
// EnterRegexpOperator is called when production regexpOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterRegexpOperator(ctx *RegexpOperatorContext) {}
|
||||
@ -418,20 +434,8 @@ func (s *BaseFqlParserListener) EnterAdditiveOperator(ctx *AdditiveOperatorConte
|
||||
// ExitAdditiveOperator is called when production additiveOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitAdditiveOperator(ctx *AdditiveOperatorContext) {}
|
||||
|
||||
// EnterUnaryOperator is called when production unaryOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterUnaryOperator(ctx *UnaryOperatorContext) {}
|
||||
// EnterErrorOperator is called when production errorOperator is entered.
|
||||
func (s *BaseFqlParserListener) EnterErrorOperator(ctx *ErrorOperatorContext) {}
|
||||
|
||||
// ExitUnaryOperator is called when production unaryOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitUnaryOperator(ctx *UnaryOperatorContext) {}
|
||||
|
||||
// EnterParam is called when production param is entered.
|
||||
func (s *BaseFqlParserListener) EnterParam(ctx *ParamContext) {}
|
||||
|
||||
// ExitParam is called when production param is exited.
|
||||
func (s *BaseFqlParserListener) ExitParam(ctx *ParamContext) {}
|
||||
|
||||
// EnterVariable is called when production variable is entered.
|
||||
func (s *BaseFqlParserListener) EnterVariable(ctx *VariableContext) {}
|
||||
|
||||
// ExitVariable is called when production variable is exited.
|
||||
func (s *BaseFqlParserListener) ExitVariable(ctx *VariableContext) {}
|
||||
// ExitErrorOperator is called when production errorOperator is exited.
|
||||
func (s *BaseFqlParserListener) ExitErrorOperator(ctx *ErrorOperatorContext) {}
|
||||
|
@ -23,10 +23,6 @@ func (v *BaseFqlParserVisitor) VisitUse(ctx *UseContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitNamespaceIdentifier(ctx *NamespaceIdentifierContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitBody(ctx *BodyContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
@ -39,22 +35,10 @@ func (v *BaseFqlParserVisitor) VisitBodyExpression(ctx *BodyExpressionContext) i
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitVariableDeclaration(ctx *VariableDeclarationContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitReturnExpression(ctx *ReturnExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitInlineHighLevelExpression(ctx *InlineHighLevelExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitHighLevelExpression(ctx *HighLevelExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitForExpression(ctx *ForExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
@ -127,18 +111,10 @@ func (v *BaseFqlParserVisitor) VisitCollectCounter(ctx *CollectCounterContext) i
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitOptionsClause(ctx *OptionsClauseContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitWaitForExpression(ctx *WaitForExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitWaitForTimeout(ctx *WaitForTimeoutContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitWaitForEventName(ctx *WaitForEventNameContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
@ -147,7 +123,27 @@ func (v *BaseFqlParserVisitor) VisitWaitForEventSource(ctx *WaitForEventSourceCo
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitRangeOperator(ctx *RangeOperatorContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitOptionsClause(ctx *OptionsClauseContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitWaitForTimeout(ctx *WaitForTimeoutContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitVariableDeclaration(ctx *VariableDeclarationContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitParam(ctx *ParamContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitVariable(ctx *VariableContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitLiteral(ctx *LiteralContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
@ -159,18 +155,6 @@ func (v *BaseFqlParserVisitor) VisitObjectLiteral(ctx *ObjectLiteralContext) int
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitPropertyAssignment(ctx *PropertyAssignmentContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitComputedPropertyName(ctx *ComputedPropertyNameContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitPropertyName(ctx *PropertyNameContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitBooleanLiteral(ctx *BooleanLiteralContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
@ -191,11 +175,23 @@ func (v *BaseFqlParserVisitor) VisitNoneLiteral(ctx *NoneLiteralContext) interfa
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitExpression(ctx *ExpressionContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitPropertyAssignment(ctx *PropertyAssignmentContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitExpressionGroup(ctx *ExpressionGroupContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitComputedPropertyName(ctx *ComputedPropertyNameContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitPropertyName(ctx *PropertyNameContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitNamespaceIdentifier(ctx *NamespaceIdentifierContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitNamespace(ctx *NamespaceContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
@ -207,11 +203,15 @@ func (v *BaseFqlParserVisitor) VisitMemberExpressionSource(ctx *MemberExpression
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitFunctionCallExpression(ctx *FunctionCallExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitFunctionCall(ctx *FunctionCallContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitFunctionCallExpression(ctx *FunctionCallExpressionContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitArgumentList(ctx *ArgumentListContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
@ -219,19 +219,27 @@ func (v *BaseFqlParserVisitor) VisitMemberExpressionPath(ctx *MemberExpressionPa
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitErrorOperator(ctx *ErrorOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitFunctionIdentifier(ctx *FunctionIdentifierContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitNamespace(ctx *NamespaceContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitRangeOperator(ctx *RangeOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitArguments(ctx *ArgumentsContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitRangeOperand(ctx *RangeOperandContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitExpression(ctx *ExpressionContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitPredicate(ctx *PredicateContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitExpressionAtom(ctx *ExpressionAtomContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
@ -239,6 +247,10 @@ func (v *BaseFqlParserVisitor) VisitArrayOperator(ctx *ArrayOperatorContext) int
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitEqualityOperator(ctx *EqualityOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitInOperator(ctx *InOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
@ -247,7 +259,7 @@ func (v *BaseFqlParserVisitor) VisitLikeOperator(ctx *LikeOperatorContext) inter
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitEqualityOperator(ctx *EqualityOperatorContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitUnaryOperator(ctx *UnaryOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
@ -271,14 +283,6 @@ func (v *BaseFqlParserVisitor) VisitAdditiveOperator(ctx *AdditiveOperatorContex
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitUnaryOperator(ctx *UnaryOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitParam(ctx *ParamContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
||||
func (v *BaseFqlParserVisitor) VisitVariable(ctx *VariableContext) interface{} {
|
||||
func (v *BaseFqlParserVisitor) VisitErrorOperator(ctx *ErrorOperatorContext) interface{} {
|
||||
return v.VisitChildren(ctx)
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ type FqlParserListener interface {
|
||||
// EnterUse is called when entering the use production.
|
||||
EnterUse(c *UseContext)
|
||||
|
||||
// EnterNamespaceIdentifier is called when entering the namespaceIdentifier production.
|
||||
EnterNamespaceIdentifier(c *NamespaceIdentifierContext)
|
||||
|
||||
// EnterBody is called when entering the body production.
|
||||
EnterBody(c *BodyContext)
|
||||
|
||||
@ -31,18 +28,9 @@ type FqlParserListener interface {
|
||||
// EnterBodyExpression is called when entering the bodyExpression production.
|
||||
EnterBodyExpression(c *BodyExpressionContext)
|
||||
|
||||
// EnterVariableDeclaration is called when entering the variableDeclaration production.
|
||||
EnterVariableDeclaration(c *VariableDeclarationContext)
|
||||
|
||||
// EnterReturnExpression is called when entering the returnExpression production.
|
||||
EnterReturnExpression(c *ReturnExpressionContext)
|
||||
|
||||
// EnterInlineHighLevelExpression is called when entering the inlineHighLevelExpression production.
|
||||
EnterInlineHighLevelExpression(c *InlineHighLevelExpressionContext)
|
||||
|
||||
// EnterHighLevelExpression is called when entering the highLevelExpression production.
|
||||
EnterHighLevelExpression(c *HighLevelExpressionContext)
|
||||
|
||||
// EnterForExpression is called when entering the forExpression production.
|
||||
EnterForExpression(c *ForExpressionContext)
|
||||
|
||||
@ -97,23 +85,32 @@ type FqlParserListener interface {
|
||||
// EnterCollectCounter is called when entering the collectCounter production.
|
||||
EnterCollectCounter(c *CollectCounterContext)
|
||||
|
||||
// EnterOptionsClause is called when entering the optionsClause production.
|
||||
EnterOptionsClause(c *OptionsClauseContext)
|
||||
|
||||
// EnterWaitForExpression is called when entering the waitForExpression production.
|
||||
EnterWaitForExpression(c *WaitForExpressionContext)
|
||||
|
||||
// EnterWaitForTimeout is called when entering the waitForTimeout production.
|
||||
EnterWaitForTimeout(c *WaitForTimeoutContext)
|
||||
|
||||
// EnterWaitForEventName is called when entering the waitForEventName production.
|
||||
EnterWaitForEventName(c *WaitForEventNameContext)
|
||||
|
||||
// EnterWaitForEventSource is called when entering the waitForEventSource production.
|
||||
EnterWaitForEventSource(c *WaitForEventSourceContext)
|
||||
|
||||
// EnterRangeOperator is called when entering the rangeOperator production.
|
||||
EnterRangeOperator(c *RangeOperatorContext)
|
||||
// EnterOptionsClause is called when entering the optionsClause production.
|
||||
EnterOptionsClause(c *OptionsClauseContext)
|
||||
|
||||
// EnterWaitForTimeout is called when entering the waitForTimeout production.
|
||||
EnterWaitForTimeout(c *WaitForTimeoutContext)
|
||||
|
||||
// EnterVariableDeclaration is called when entering the variableDeclaration production.
|
||||
EnterVariableDeclaration(c *VariableDeclarationContext)
|
||||
|
||||
// EnterParam is called when entering the param production.
|
||||
EnterParam(c *ParamContext)
|
||||
|
||||
// EnterVariable is called when entering the variable production.
|
||||
EnterVariable(c *VariableContext)
|
||||
|
||||
// EnterLiteral is called when entering the literal production.
|
||||
EnterLiteral(c *LiteralContext)
|
||||
|
||||
// EnterArrayLiteral is called when entering the arrayLiteral production.
|
||||
EnterArrayLiteral(c *ArrayLiteralContext)
|
||||
@ -121,15 +118,6 @@ type FqlParserListener interface {
|
||||
// EnterObjectLiteral is called when entering the objectLiteral production.
|
||||
EnterObjectLiteral(c *ObjectLiteralContext)
|
||||
|
||||
// EnterPropertyAssignment is called when entering the propertyAssignment production.
|
||||
EnterPropertyAssignment(c *PropertyAssignmentContext)
|
||||
|
||||
// EnterComputedPropertyName is called when entering the computedPropertyName production.
|
||||
EnterComputedPropertyName(c *ComputedPropertyNameContext)
|
||||
|
||||
// EnterPropertyName is called when entering the propertyName production.
|
||||
EnterPropertyName(c *PropertyNameContext)
|
||||
|
||||
// EnterBooleanLiteral is called when entering the booleanLiteral production.
|
||||
EnterBooleanLiteral(c *BooleanLiteralContext)
|
||||
|
||||
@ -145,11 +133,20 @@ type FqlParserListener interface {
|
||||
// EnterNoneLiteral is called when entering the noneLiteral production.
|
||||
EnterNoneLiteral(c *NoneLiteralContext)
|
||||
|
||||
// EnterExpression is called when entering the expression production.
|
||||
EnterExpression(c *ExpressionContext)
|
||||
// EnterPropertyAssignment is called when entering the propertyAssignment production.
|
||||
EnterPropertyAssignment(c *PropertyAssignmentContext)
|
||||
|
||||
// EnterExpressionGroup is called when entering the expressionGroup production.
|
||||
EnterExpressionGroup(c *ExpressionGroupContext)
|
||||
// EnterComputedPropertyName is called when entering the computedPropertyName production.
|
||||
EnterComputedPropertyName(c *ComputedPropertyNameContext)
|
||||
|
||||
// EnterPropertyName is called when entering the propertyName production.
|
||||
EnterPropertyName(c *PropertyNameContext)
|
||||
|
||||
// EnterNamespaceIdentifier is called when entering the namespaceIdentifier production.
|
||||
EnterNamespaceIdentifier(c *NamespaceIdentifierContext)
|
||||
|
||||
// EnterNamespace is called when entering the namespace production.
|
||||
EnterNamespace(c *NamespaceContext)
|
||||
|
||||
// EnterMemberExpression is called when entering the memberExpression production.
|
||||
EnterMemberExpression(c *MemberExpressionContext)
|
||||
@ -157,38 +154,50 @@ type FqlParserListener interface {
|
||||
// EnterMemberExpressionSource is called when entering the memberExpressionSource production.
|
||||
EnterMemberExpressionSource(c *MemberExpressionSourceContext)
|
||||
|
||||
// EnterFunctionCallExpression is called when entering the functionCallExpression production.
|
||||
EnterFunctionCallExpression(c *FunctionCallExpressionContext)
|
||||
|
||||
// EnterFunctionCall is called when entering the functionCall production.
|
||||
EnterFunctionCall(c *FunctionCallContext)
|
||||
|
||||
// EnterFunctionCallExpression is called when entering the functionCallExpression production.
|
||||
EnterFunctionCallExpression(c *FunctionCallExpressionContext)
|
||||
// EnterArgumentList is called when entering the argumentList production.
|
||||
EnterArgumentList(c *ArgumentListContext)
|
||||
|
||||
// EnterMemberExpressionPath is called when entering the memberExpressionPath production.
|
||||
EnterMemberExpressionPath(c *MemberExpressionPathContext)
|
||||
|
||||
// EnterErrorOperator is called when entering the errorOperator production.
|
||||
EnterErrorOperator(c *ErrorOperatorContext)
|
||||
|
||||
// EnterFunctionIdentifier is called when entering the functionIdentifier production.
|
||||
EnterFunctionIdentifier(c *FunctionIdentifierContext)
|
||||
|
||||
// EnterNamespace is called when entering the namespace production.
|
||||
EnterNamespace(c *NamespaceContext)
|
||||
// EnterRangeOperator is called when entering the rangeOperator production.
|
||||
EnterRangeOperator(c *RangeOperatorContext)
|
||||
|
||||
// EnterArguments is called when entering the arguments production.
|
||||
EnterArguments(c *ArgumentsContext)
|
||||
// EnterRangeOperand is called when entering the rangeOperand production.
|
||||
EnterRangeOperand(c *RangeOperandContext)
|
||||
|
||||
// EnterExpression is called when entering the expression production.
|
||||
EnterExpression(c *ExpressionContext)
|
||||
|
||||
// EnterPredicate is called when entering the predicate production.
|
||||
EnterPredicate(c *PredicateContext)
|
||||
|
||||
// EnterExpressionAtom is called when entering the expressionAtom production.
|
||||
EnterExpressionAtom(c *ExpressionAtomContext)
|
||||
|
||||
// EnterArrayOperator is called when entering the arrayOperator production.
|
||||
EnterArrayOperator(c *ArrayOperatorContext)
|
||||
|
||||
// EnterEqualityOperator is called when entering the equalityOperator production.
|
||||
EnterEqualityOperator(c *EqualityOperatorContext)
|
||||
|
||||
// EnterInOperator is called when entering the inOperator production.
|
||||
EnterInOperator(c *InOperatorContext)
|
||||
|
||||
// EnterLikeOperator is called when entering the likeOperator production.
|
||||
EnterLikeOperator(c *LikeOperatorContext)
|
||||
|
||||
// EnterEqualityOperator is called when entering the equalityOperator production.
|
||||
EnterEqualityOperator(c *EqualityOperatorContext)
|
||||
// EnterUnaryOperator is called when entering the unaryOperator production.
|
||||
EnterUnaryOperator(c *UnaryOperatorContext)
|
||||
|
||||
// EnterRegexpOperator is called when entering the regexpOperator production.
|
||||
EnterRegexpOperator(c *RegexpOperatorContext)
|
||||
@ -205,14 +214,8 @@ type FqlParserListener interface {
|
||||
// EnterAdditiveOperator is called when entering the additiveOperator production.
|
||||
EnterAdditiveOperator(c *AdditiveOperatorContext)
|
||||
|
||||
// EnterUnaryOperator is called when entering the unaryOperator production.
|
||||
EnterUnaryOperator(c *UnaryOperatorContext)
|
||||
|
||||
// EnterParam is called when entering the param production.
|
||||
EnterParam(c *ParamContext)
|
||||
|
||||
// EnterVariable is called when entering the variable production.
|
||||
EnterVariable(c *VariableContext)
|
||||
// EnterErrorOperator is called when entering the errorOperator production.
|
||||
EnterErrorOperator(c *ErrorOperatorContext)
|
||||
|
||||
// ExitProgram is called when exiting the program production.
|
||||
ExitProgram(c *ProgramContext)
|
||||
@ -226,9 +229,6 @@ type FqlParserListener interface {
|
||||
// ExitUse is called when exiting the use production.
|
||||
ExitUse(c *UseContext)
|
||||
|
||||
// ExitNamespaceIdentifier is called when exiting the namespaceIdentifier production.
|
||||
ExitNamespaceIdentifier(c *NamespaceIdentifierContext)
|
||||
|
||||
// ExitBody is called when exiting the body production.
|
||||
ExitBody(c *BodyContext)
|
||||
|
||||
@ -238,18 +238,9 @@ type FqlParserListener interface {
|
||||
// ExitBodyExpression is called when exiting the bodyExpression production.
|
||||
ExitBodyExpression(c *BodyExpressionContext)
|
||||
|
||||
// ExitVariableDeclaration is called when exiting the variableDeclaration production.
|
||||
ExitVariableDeclaration(c *VariableDeclarationContext)
|
||||
|
||||
// ExitReturnExpression is called when exiting the returnExpression production.
|
||||
ExitReturnExpression(c *ReturnExpressionContext)
|
||||
|
||||
// ExitInlineHighLevelExpression is called when exiting the inlineHighLevelExpression production.
|
||||
ExitInlineHighLevelExpression(c *InlineHighLevelExpressionContext)
|
||||
|
||||
// ExitHighLevelExpression is called when exiting the highLevelExpression production.
|
||||
ExitHighLevelExpression(c *HighLevelExpressionContext)
|
||||
|
||||
// ExitForExpression is called when exiting the forExpression production.
|
||||
ExitForExpression(c *ForExpressionContext)
|
||||
|
||||
@ -304,23 +295,32 @@ type FqlParserListener interface {
|
||||
// ExitCollectCounter is called when exiting the collectCounter production.
|
||||
ExitCollectCounter(c *CollectCounterContext)
|
||||
|
||||
// ExitOptionsClause is called when exiting the optionsClause production.
|
||||
ExitOptionsClause(c *OptionsClauseContext)
|
||||
|
||||
// ExitWaitForExpression is called when exiting the waitForExpression production.
|
||||
ExitWaitForExpression(c *WaitForExpressionContext)
|
||||
|
||||
// ExitWaitForTimeout is called when exiting the waitForTimeout production.
|
||||
ExitWaitForTimeout(c *WaitForTimeoutContext)
|
||||
|
||||
// ExitWaitForEventName is called when exiting the waitForEventName production.
|
||||
ExitWaitForEventName(c *WaitForEventNameContext)
|
||||
|
||||
// ExitWaitForEventSource is called when exiting the waitForEventSource production.
|
||||
ExitWaitForEventSource(c *WaitForEventSourceContext)
|
||||
|
||||
// ExitRangeOperator is called when exiting the rangeOperator production.
|
||||
ExitRangeOperator(c *RangeOperatorContext)
|
||||
// ExitOptionsClause is called when exiting the optionsClause production.
|
||||
ExitOptionsClause(c *OptionsClauseContext)
|
||||
|
||||
// ExitWaitForTimeout is called when exiting the waitForTimeout production.
|
||||
ExitWaitForTimeout(c *WaitForTimeoutContext)
|
||||
|
||||
// ExitVariableDeclaration is called when exiting the variableDeclaration production.
|
||||
ExitVariableDeclaration(c *VariableDeclarationContext)
|
||||
|
||||
// ExitParam is called when exiting the param production.
|
||||
ExitParam(c *ParamContext)
|
||||
|
||||
// ExitVariable is called when exiting the variable production.
|
||||
ExitVariable(c *VariableContext)
|
||||
|
||||
// ExitLiteral is called when exiting the literal production.
|
||||
ExitLiteral(c *LiteralContext)
|
||||
|
||||
// ExitArrayLiteral is called when exiting the arrayLiteral production.
|
||||
ExitArrayLiteral(c *ArrayLiteralContext)
|
||||
@ -328,15 +328,6 @@ type FqlParserListener interface {
|
||||
// ExitObjectLiteral is called when exiting the objectLiteral production.
|
||||
ExitObjectLiteral(c *ObjectLiteralContext)
|
||||
|
||||
// ExitPropertyAssignment is called when exiting the propertyAssignment production.
|
||||
ExitPropertyAssignment(c *PropertyAssignmentContext)
|
||||
|
||||
// ExitComputedPropertyName is called when exiting the computedPropertyName production.
|
||||
ExitComputedPropertyName(c *ComputedPropertyNameContext)
|
||||
|
||||
// ExitPropertyName is called when exiting the propertyName production.
|
||||
ExitPropertyName(c *PropertyNameContext)
|
||||
|
||||
// ExitBooleanLiteral is called when exiting the booleanLiteral production.
|
||||
ExitBooleanLiteral(c *BooleanLiteralContext)
|
||||
|
||||
@ -352,11 +343,20 @@ type FqlParserListener interface {
|
||||
// ExitNoneLiteral is called when exiting the noneLiteral production.
|
||||
ExitNoneLiteral(c *NoneLiteralContext)
|
||||
|
||||
// ExitExpression is called when exiting the expression production.
|
||||
ExitExpression(c *ExpressionContext)
|
||||
// ExitPropertyAssignment is called when exiting the propertyAssignment production.
|
||||
ExitPropertyAssignment(c *PropertyAssignmentContext)
|
||||
|
||||
// ExitExpressionGroup is called when exiting the expressionGroup production.
|
||||
ExitExpressionGroup(c *ExpressionGroupContext)
|
||||
// ExitComputedPropertyName is called when exiting the computedPropertyName production.
|
||||
ExitComputedPropertyName(c *ComputedPropertyNameContext)
|
||||
|
||||
// ExitPropertyName is called when exiting the propertyName production.
|
||||
ExitPropertyName(c *PropertyNameContext)
|
||||
|
||||
// ExitNamespaceIdentifier is called when exiting the namespaceIdentifier production.
|
||||
ExitNamespaceIdentifier(c *NamespaceIdentifierContext)
|
||||
|
||||
// ExitNamespace is called when exiting the namespace production.
|
||||
ExitNamespace(c *NamespaceContext)
|
||||
|
||||
// ExitMemberExpression is called when exiting the memberExpression production.
|
||||
ExitMemberExpression(c *MemberExpressionContext)
|
||||
@ -364,38 +364,50 @@ type FqlParserListener interface {
|
||||
// ExitMemberExpressionSource is called when exiting the memberExpressionSource production.
|
||||
ExitMemberExpressionSource(c *MemberExpressionSourceContext)
|
||||
|
||||
// ExitFunctionCallExpression is called when exiting the functionCallExpression production.
|
||||
ExitFunctionCallExpression(c *FunctionCallExpressionContext)
|
||||
|
||||
// ExitFunctionCall is called when exiting the functionCall production.
|
||||
ExitFunctionCall(c *FunctionCallContext)
|
||||
|
||||
// ExitFunctionCallExpression is called when exiting the functionCallExpression production.
|
||||
ExitFunctionCallExpression(c *FunctionCallExpressionContext)
|
||||
// ExitArgumentList is called when exiting the argumentList production.
|
||||
ExitArgumentList(c *ArgumentListContext)
|
||||
|
||||
// ExitMemberExpressionPath is called when exiting the memberExpressionPath production.
|
||||
ExitMemberExpressionPath(c *MemberExpressionPathContext)
|
||||
|
||||
// ExitErrorOperator is called when exiting the errorOperator production.
|
||||
ExitErrorOperator(c *ErrorOperatorContext)
|
||||
|
||||
// ExitFunctionIdentifier is called when exiting the functionIdentifier production.
|
||||
ExitFunctionIdentifier(c *FunctionIdentifierContext)
|
||||
|
||||
// ExitNamespace is called when exiting the namespace production.
|
||||
ExitNamespace(c *NamespaceContext)
|
||||
// ExitRangeOperator is called when exiting the rangeOperator production.
|
||||
ExitRangeOperator(c *RangeOperatorContext)
|
||||
|
||||
// ExitArguments is called when exiting the arguments production.
|
||||
ExitArguments(c *ArgumentsContext)
|
||||
// ExitRangeOperand is called when exiting the rangeOperand production.
|
||||
ExitRangeOperand(c *RangeOperandContext)
|
||||
|
||||
// ExitExpression is called when exiting the expression production.
|
||||
ExitExpression(c *ExpressionContext)
|
||||
|
||||
// ExitPredicate is called when exiting the predicate production.
|
||||
ExitPredicate(c *PredicateContext)
|
||||
|
||||
// ExitExpressionAtom is called when exiting the expressionAtom production.
|
||||
ExitExpressionAtom(c *ExpressionAtomContext)
|
||||
|
||||
// ExitArrayOperator is called when exiting the arrayOperator production.
|
||||
ExitArrayOperator(c *ArrayOperatorContext)
|
||||
|
||||
// ExitEqualityOperator is called when exiting the equalityOperator production.
|
||||
ExitEqualityOperator(c *EqualityOperatorContext)
|
||||
|
||||
// ExitInOperator is called when exiting the inOperator production.
|
||||
ExitInOperator(c *InOperatorContext)
|
||||
|
||||
// ExitLikeOperator is called when exiting the likeOperator production.
|
||||
ExitLikeOperator(c *LikeOperatorContext)
|
||||
|
||||
// ExitEqualityOperator is called when exiting the equalityOperator production.
|
||||
ExitEqualityOperator(c *EqualityOperatorContext)
|
||||
// ExitUnaryOperator is called when exiting the unaryOperator production.
|
||||
ExitUnaryOperator(c *UnaryOperatorContext)
|
||||
|
||||
// ExitRegexpOperator is called when exiting the regexpOperator production.
|
||||
ExitRegexpOperator(c *RegexpOperatorContext)
|
||||
@ -412,12 +424,6 @@ type FqlParserListener interface {
|
||||
// ExitAdditiveOperator is called when exiting the additiveOperator production.
|
||||
ExitAdditiveOperator(c *AdditiveOperatorContext)
|
||||
|
||||
// ExitUnaryOperator is called when exiting the unaryOperator production.
|
||||
ExitUnaryOperator(c *UnaryOperatorContext)
|
||||
|
||||
// ExitParam is called when exiting the param production.
|
||||
ExitParam(c *ParamContext)
|
||||
|
||||
// ExitVariable is called when exiting the variable production.
|
||||
ExitVariable(c *VariableContext)
|
||||
// ExitErrorOperator is called when exiting the errorOperator production.
|
||||
ExitErrorOperator(c *ErrorOperatorContext)
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#use.
|
||||
VisitUse(ctx *UseContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#namespaceIdentifier.
|
||||
VisitNamespaceIdentifier(ctx *NamespaceIdentifierContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#body.
|
||||
VisitBody(ctx *BodyContext) interface{}
|
||||
|
||||
@ -31,18 +28,9 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#bodyExpression.
|
||||
VisitBodyExpression(ctx *BodyExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#variableDeclaration.
|
||||
VisitVariableDeclaration(ctx *VariableDeclarationContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#returnExpression.
|
||||
VisitReturnExpression(ctx *ReturnExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#inlineHighLevelExpression.
|
||||
VisitInlineHighLevelExpression(ctx *InlineHighLevelExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#highLevelExpression.
|
||||
VisitHighLevelExpression(ctx *HighLevelExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#forExpression.
|
||||
VisitForExpression(ctx *ForExpressionContext) interface{}
|
||||
|
||||
@ -97,23 +85,32 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#collectCounter.
|
||||
VisitCollectCounter(ctx *CollectCounterContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#optionsClause.
|
||||
VisitOptionsClause(ctx *OptionsClauseContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#waitForExpression.
|
||||
VisitWaitForExpression(ctx *WaitForExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#waitForTimeout.
|
||||
VisitWaitForTimeout(ctx *WaitForTimeoutContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#waitForEventName.
|
||||
VisitWaitForEventName(ctx *WaitForEventNameContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#waitForEventSource.
|
||||
VisitWaitForEventSource(ctx *WaitForEventSourceContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#rangeOperator.
|
||||
VisitRangeOperator(ctx *RangeOperatorContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#optionsClause.
|
||||
VisitOptionsClause(ctx *OptionsClauseContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#waitForTimeout.
|
||||
VisitWaitForTimeout(ctx *WaitForTimeoutContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#variableDeclaration.
|
||||
VisitVariableDeclaration(ctx *VariableDeclarationContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#param.
|
||||
VisitParam(ctx *ParamContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#variable.
|
||||
VisitVariable(ctx *VariableContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#literal.
|
||||
VisitLiteral(ctx *LiteralContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#arrayLiteral.
|
||||
VisitArrayLiteral(ctx *ArrayLiteralContext) interface{}
|
||||
@ -121,15 +118,6 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#objectLiteral.
|
||||
VisitObjectLiteral(ctx *ObjectLiteralContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#propertyAssignment.
|
||||
VisitPropertyAssignment(ctx *PropertyAssignmentContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#computedPropertyName.
|
||||
VisitComputedPropertyName(ctx *ComputedPropertyNameContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#propertyName.
|
||||
VisitPropertyName(ctx *PropertyNameContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#booleanLiteral.
|
||||
VisitBooleanLiteral(ctx *BooleanLiteralContext) interface{}
|
||||
|
||||
@ -145,11 +133,20 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#noneLiteral.
|
||||
VisitNoneLiteral(ctx *NoneLiteralContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#expression.
|
||||
VisitExpression(ctx *ExpressionContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#propertyAssignment.
|
||||
VisitPropertyAssignment(ctx *PropertyAssignmentContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#expressionGroup.
|
||||
VisitExpressionGroup(ctx *ExpressionGroupContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#computedPropertyName.
|
||||
VisitComputedPropertyName(ctx *ComputedPropertyNameContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#propertyName.
|
||||
VisitPropertyName(ctx *PropertyNameContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#namespaceIdentifier.
|
||||
VisitNamespaceIdentifier(ctx *NamespaceIdentifierContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#namespace.
|
||||
VisitNamespace(ctx *NamespaceContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#memberExpression.
|
||||
VisitMemberExpression(ctx *MemberExpressionContext) interface{}
|
||||
@ -157,38 +154,50 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#memberExpressionSource.
|
||||
VisitMemberExpressionSource(ctx *MemberExpressionSourceContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#functionCallExpression.
|
||||
VisitFunctionCallExpression(ctx *FunctionCallExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#functionCall.
|
||||
VisitFunctionCall(ctx *FunctionCallContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#functionCallExpression.
|
||||
VisitFunctionCallExpression(ctx *FunctionCallExpressionContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#argumentList.
|
||||
VisitArgumentList(ctx *ArgumentListContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#memberExpressionPath.
|
||||
VisitMemberExpressionPath(ctx *MemberExpressionPathContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#errorOperator.
|
||||
VisitErrorOperator(ctx *ErrorOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#functionIdentifier.
|
||||
VisitFunctionIdentifier(ctx *FunctionIdentifierContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#namespace.
|
||||
VisitNamespace(ctx *NamespaceContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#rangeOperator.
|
||||
VisitRangeOperator(ctx *RangeOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#arguments.
|
||||
VisitArguments(ctx *ArgumentsContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#rangeOperand.
|
||||
VisitRangeOperand(ctx *RangeOperandContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#expression.
|
||||
VisitExpression(ctx *ExpressionContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#predicate.
|
||||
VisitPredicate(ctx *PredicateContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#expressionAtom.
|
||||
VisitExpressionAtom(ctx *ExpressionAtomContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#arrayOperator.
|
||||
VisitArrayOperator(ctx *ArrayOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#equalityOperator.
|
||||
VisitEqualityOperator(ctx *EqualityOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#inOperator.
|
||||
VisitInOperator(ctx *InOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#likeOperator.
|
||||
VisitLikeOperator(ctx *LikeOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#equalityOperator.
|
||||
VisitEqualityOperator(ctx *EqualityOperatorContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#unaryOperator.
|
||||
VisitUnaryOperator(ctx *UnaryOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#regexpOperator.
|
||||
VisitRegexpOperator(ctx *RegexpOperatorContext) interface{}
|
||||
@ -205,12 +214,6 @@ type FqlParserVisitor interface {
|
||||
// Visit a parse tree produced by FqlParser#additiveOperator.
|
||||
VisitAdditiveOperator(ctx *AdditiveOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#unaryOperator.
|
||||
VisitUnaryOperator(ctx *UnaryOperatorContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#param.
|
||||
VisitParam(ctx *ParamContext) interface{}
|
||||
|
||||
// Visit a parse tree produced by FqlParser#variable.
|
||||
VisitVariable(ctx *VariableContext) interface{}
|
||||
// Visit a parse tree produced by FqlParser#errorOperator.
|
||||
VisitErrorOperator(ctx *ErrorOperatorContext) interface{}
|
||||
}
|
||||
|
@ -3,7 +3,6 @@ package parser
|
||||
|
||||
import (
|
||||
"github.com/MontFerret/ferret/pkg/parser/fql"
|
||||
resources "github.com/antlr/antlr4/doc/resources"
|
||||
"github.com/antlr/antlr4/runtime/Go/antlr"
|
||||
)
|
||||
|
||||
@ -15,7 +14,7 @@ func New(query string) *Parser {
|
||||
input := antlr.NewInputStream(query)
|
||||
// converts tokens to upper case, so now it doesn't matter
|
||||
// in which case the tokens were entered
|
||||
upper := resources.NewCaseChangingStream(input, true)
|
||||
upper := newCaseChangingStream(input, true)
|
||||
|
||||
lexer := fql.NewFqlLexer(upper)
|
||||
stream := antlr.NewCommonTokenStream(lexer, antlr.TokenDefaultChannel)
|
||||
|
@ -2,7 +2,7 @@ package core
|
||||
|
||||
import "context"
|
||||
|
||||
type OperatorExpression interface {
|
||||
type Predicate interface {
|
||||
Expression
|
||||
Eval(ctx context.Context, left, right Value) (Value, error)
|
||||
}
|
@ -15,7 +15,7 @@ type FunctionCallExpression struct {
|
||||
func NewFunctionCallExpression(
|
||||
src core.SourceMap,
|
||||
fun core.Function,
|
||||
args ...core.Expression,
|
||||
args []core.Expression,
|
||||
) (*FunctionCallExpression, error) {
|
||||
if fun == nil {
|
||||
return nil, core.Error(core.ErrMissedArgument, "function")
|
||||
@ -24,6 +24,14 @@ func NewFunctionCallExpression(
|
||||
return &FunctionCallExpression{src, fun, args}, nil
|
||||
}
|
||||
|
||||
func NewFunctionCallExpressionWith(
|
||||
src core.SourceMap,
|
||||
fun core.Function,
|
||||
args ...core.Expression,
|
||||
) (*FunctionCallExpression, error) {
|
||||
return NewFunctionCallExpression(src, fun, args)
|
||||
}
|
||||
|
||||
func (e *FunctionCallExpression) Arguments() []core.Expression {
|
||||
return e.args
|
||||
}
|
||||
|
@ -15,7 +15,7 @@ import (
|
||||
func TestFunctionCallExpression(t *testing.T) {
|
||||
Convey(".Exec", t, func() {
|
||||
Convey("Should execute an underlying function without arguments", func() {
|
||||
f, err := expressions.NewFunctionCallExpression(
|
||||
f, err := expressions.NewFunctionCallExpressionWith(
|
||||
core.SourceMap{},
|
||||
func(ctx context.Context, args ...core.Value) (value core.Value, e error) {
|
||||
So(args, ShouldHaveLength, 0)
|
||||
@ -40,7 +40,7 @@ func TestFunctionCallExpression(t *testing.T) {
|
||||
literals.NewStringLiteral("foo"),
|
||||
}
|
||||
|
||||
f, err := expressions.NewFunctionCallExpression(
|
||||
f, err := expressions.NewFunctionCallExpressionWith(
|
||||
core.SourceMap{},
|
||||
func(ctx context.Context, args ...core.Value) (value core.Value, e error) {
|
||||
So(args, ShouldHaveLength, len(args))
|
||||
@ -66,7 +66,7 @@ func TestFunctionCallExpression(t *testing.T) {
|
||||
literals.NewStringLiteral("foo"),
|
||||
}
|
||||
|
||||
f, err := expressions.NewFunctionCallExpression(
|
||||
f, err := expressions.NewFunctionCallExpressionWith(
|
||||
core.SourceMap{},
|
||||
func(ctx context.Context, args ...core.Value) (value core.Value, e error) {
|
||||
So(args, ShouldHaveLength, len(args))
|
||||
@ -88,7 +88,7 @@ func TestFunctionCallExpression(t *testing.T) {
|
||||
})
|
||||
|
||||
Convey("Should ignore errors and return NONE", func() {
|
||||
f, err := expressions.NewFunctionCallExpression(
|
||||
f, err := expressions.NewFunctionCallExpressionWith(
|
||||
core.SourceMap{},
|
||||
func(ctx context.Context, args ...core.Value) (value core.Value, e error) {
|
||||
return values.NewString("booo"), core.ErrNotImplemented
|
||||
|
@ -14,7 +14,7 @@ func NewArrayLiteral(size int) *ArrayLiteral {
|
||||
return &ArrayLiteral{make([]core.Expression, 0, size)}
|
||||
}
|
||||
|
||||
func NewArrayLiteralWith(elements ...core.Expression) *ArrayLiteral {
|
||||
func NewArrayLiteralWith(elements []core.Expression) *ArrayLiteral {
|
||||
return &ArrayLiteral{elements}
|
||||
}
|
||||
|
||||
|
@ -31,10 +31,14 @@ func NewObjectPropertyAssignment(name, value core.Expression) (*ObjectPropertyAs
|
||||
return &ObjectPropertyAssignment{name, value}, nil
|
||||
}
|
||||
|
||||
func NewObjectLiteralWith(props ...*ObjectPropertyAssignment) *ObjectLiteral {
|
||||
func NewObjectLiteral(props []*ObjectPropertyAssignment) *ObjectLiteral {
|
||||
return &ObjectLiteral{props}
|
||||
}
|
||||
|
||||
func NewObjectLiteralWith(props ...*ObjectPropertyAssignment) *ObjectLiteral {
|
||||
return NewObjectLiteral(props)
|
||||
}
|
||||
|
||||
func (l *ObjectLiteral) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
obj := values.NewObject()
|
||||
|
||||
|
@ -2,6 +2,7 @@ package operators
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
|
||||
"github.com/MontFerret/ferret/pkg/runtime/core"
|
||||
"github.com/MontFerret/ferret/pkg/runtime/values"
|
||||
@ -9,39 +10,31 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
ArrayOperatorType int
|
||||
ArrayOperator struct {
|
||||
ArrayOperatorVariant int
|
||||
|
||||
ArrayOperator struct {
|
||||
*baseOperator
|
||||
aotype ArrayOperatorType
|
||||
comparator core.OperatorExpression
|
||||
variant ArrayOperatorVariant
|
||||
comparator core.Predicate
|
||||
}
|
||||
)
|
||||
|
||||
const (
|
||||
ArrayOperatorTypeAll ArrayOperatorType = 0
|
||||
ArrayOperatorTypeAny ArrayOperatorType = 1
|
||||
ArrayOperatorTypeNone ArrayOperatorType = 2
|
||||
ArrayOperatorVariantAll ArrayOperatorVariant = iota
|
||||
ArrayOperatorVariantAny
|
||||
ArrayOperatorVariantNone
|
||||
)
|
||||
|
||||
func IsValidArrayOperatorType(aotype ArrayOperatorType) bool {
|
||||
switch aotype {
|
||||
case ArrayOperatorTypeAll, ArrayOperatorTypeAny, ArrayOperatorTypeNone:
|
||||
return true
|
||||
default:
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
func ToIsValidArrayOperatorType(stype string) (ArrayOperatorType, error) {
|
||||
switch stype {
|
||||
func ToArrayOperatorVariant(name string) (ArrayOperatorVariant, error) {
|
||||
switch strings.ToUpper(name) {
|
||||
case "ALL":
|
||||
return ArrayOperatorTypeAll, nil
|
||||
return ArrayOperatorVariantAll, nil
|
||||
case "ANY":
|
||||
return ArrayOperatorTypeAny, nil
|
||||
return ArrayOperatorVariantAny, nil
|
||||
case "NONE":
|
||||
return ArrayOperatorTypeNone, nil
|
||||
return ArrayOperatorVariantNone, nil
|
||||
default:
|
||||
return ArrayOperatorType(-1), core.Error(core.ErrInvalidArgument, stype)
|
||||
return ArrayOperatorVariant(-1), core.Error(core.ErrInvalidArgument, name)
|
||||
}
|
||||
}
|
||||
|
||||
@ -49,8 +42,8 @@ func NewArrayOperator(
|
||||
src core.SourceMap,
|
||||
left core.Expression,
|
||||
right core.Expression,
|
||||
aotype ArrayOperatorType,
|
||||
comparator core.OperatorExpression,
|
||||
variantStr string,
|
||||
comparator core.Predicate,
|
||||
) (*ArrayOperator, error) {
|
||||
if left == nil {
|
||||
return nil, core.Error(core.ErrMissedArgument, "left expression")
|
||||
@ -60,8 +53,10 @@ func NewArrayOperator(
|
||||
return nil, core.Error(core.ErrMissedArgument, "right expression")
|
||||
}
|
||||
|
||||
if !IsValidArrayOperatorType(aotype) {
|
||||
return nil, core.Error(core.ErrInvalidArgument, "operator")
|
||||
variant, err := ToArrayOperatorVariant(variantStr)
|
||||
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if comparator == nil {
|
||||
@ -70,7 +65,7 @@ func NewArrayOperator(
|
||||
|
||||
base := &baseOperator{src, left, right}
|
||||
|
||||
return &ArrayOperator{base, aotype, comparator}, nil
|
||||
return &ArrayOperator{base, variant, comparator}, nil
|
||||
}
|
||||
|
||||
func (operator *ArrayOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
@ -99,10 +94,10 @@ func (operator *ArrayOperator) Eval(ctx context.Context, left, right core.Value)
|
||||
|
||||
arr := left.(*values.Array)
|
||||
|
||||
switch operator.aotype {
|
||||
case ArrayOperatorTypeAll:
|
||||
switch operator.variant {
|
||||
case ArrayOperatorVariantAll:
|
||||
return operator.all(ctx, arr, right)
|
||||
case ArrayOperatorTypeAny:
|
||||
case ArrayOperatorVariantAny:
|
||||
return operator.any(ctx, arr, right)
|
||||
default:
|
||||
return operator.none(ctx, arr, right)
|
||||
|
@ -10,14 +10,14 @@ import (
|
||||
|
||||
type InOperator struct {
|
||||
*baseOperator
|
||||
not bool
|
||||
negate bool
|
||||
}
|
||||
|
||||
func NewInOperator(
|
||||
src core.SourceMap,
|
||||
left core.Expression,
|
||||
right core.Expression,
|
||||
not bool,
|
||||
negate bool,
|
||||
) (*InOperator, error) {
|
||||
if left == nil {
|
||||
return nil, core.Error(core.ErrMissedArgument, "left expression")
|
||||
@ -27,7 +27,7 @@ func NewInOperator(
|
||||
return nil, core.Error(core.ErrMissedArgument, "right expression")
|
||||
}
|
||||
|
||||
return &InOperator{&baseOperator{src, left, right}, not}, nil
|
||||
return &InOperator{&baseOperator{src, left, right}, negate}, nil
|
||||
}
|
||||
|
||||
func (operator *InOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
@ -57,7 +57,7 @@ func (operator *InOperator) Eval(_ context.Context, left, right core.Value) (cor
|
||||
arr := right.(*values.Array)
|
||||
found := arr.IndexOf(left) > -1
|
||||
|
||||
if operator.not {
|
||||
if operator.negate {
|
||||
return values.NewBoolean(!found), nil
|
||||
}
|
||||
|
||||
|
@ -13,14 +13,14 @@ import (
|
||||
|
||||
type LikeOperator struct {
|
||||
*baseOperator
|
||||
not bool
|
||||
negate bool
|
||||
}
|
||||
|
||||
func NewLikeOperator(
|
||||
src core.SourceMap,
|
||||
left core.Expression,
|
||||
right core.Expression,
|
||||
not bool,
|
||||
negate bool,
|
||||
) (*LikeOperator, error) {
|
||||
if left == nil {
|
||||
return nil, core.Error(core.ErrMissedArgument, "left expression")
|
||||
@ -30,7 +30,7 @@ func NewLikeOperator(
|
||||
return nil, core.Error(core.ErrMissedArgument, "right expression")
|
||||
}
|
||||
|
||||
return &LikeOperator{&baseOperator{src, left, right}, not}, nil
|
||||
return &LikeOperator{&baseOperator{src, left, right}, negate}, nil
|
||||
}
|
||||
|
||||
func (operator *LikeOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
@ -72,7 +72,7 @@ func (operator *LikeOperator) Eval(_ context.Context, left, right core.Value) (c
|
||||
|
||||
result := r.Match(left.String())
|
||||
|
||||
if operator.not {
|
||||
if operator.negate {
|
||||
return values.NewBoolean(!result), nil
|
||||
}
|
||||
|
||||
|
@ -9,25 +9,26 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
LogicalOperatorType int
|
||||
LogicalOperator struct {
|
||||
LogicalOperatorVariant int
|
||||
|
||||
LogicalOperator struct {
|
||||
*baseOperator
|
||||
value LogicalOperatorType
|
||||
variant LogicalOperatorVariant
|
||||
}
|
||||
)
|
||||
|
||||
const (
|
||||
LogicalOperatorTypeAnd LogicalOperatorType = 0
|
||||
LogicalOperatorTypeOr LogicalOperatorType = 1
|
||||
LogicalOperatorTypeNot LogicalOperatorType = 2
|
||||
LogicalOperatorVariantAnd LogicalOperatorVariant = 0
|
||||
LogicalOperatorVariantOr LogicalOperatorVariant = 1
|
||||
LogicalOperatorVariantNot LogicalOperatorVariant = 2
|
||||
)
|
||||
|
||||
var logicalOperators = map[string]LogicalOperatorType{
|
||||
"&&": LogicalOperatorTypeAnd,
|
||||
"AND": LogicalOperatorTypeAnd,
|
||||
"||": LogicalOperatorTypeOr,
|
||||
"OR": LogicalOperatorTypeOr,
|
||||
"NOT": LogicalOperatorTypeNot,
|
||||
var logicalOperators = map[string]LogicalOperatorVariant{
|
||||
"&&": LogicalOperatorVariantAnd,
|
||||
"AND": LogicalOperatorVariantAnd,
|
||||
"||": LogicalOperatorVariantOr,
|
||||
"OR": LogicalOperatorVariantOr,
|
||||
"NOT": LogicalOperatorVariantNot,
|
||||
}
|
||||
|
||||
func NewLogicalOperator(
|
||||
@ -53,7 +54,7 @@ func NewLogicalOperator(
|
||||
}
|
||||
|
||||
func (operator *LogicalOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
if operator.value == LogicalOperatorTypeNot {
|
||||
if operator.variant == LogicalOperatorVariantNot {
|
||||
val, err := operator.right.Exec(ctx, scope)
|
||||
|
||||
if err != nil {
|
||||
@ -71,7 +72,7 @@ func (operator *LogicalOperator) Exec(ctx context.Context, scope *core.Scope) (c
|
||||
|
||||
leftBool := values.ToBoolean(left)
|
||||
|
||||
if operator.value == LogicalOperatorTypeAnd && leftBool == values.False {
|
||||
if operator.variant == LogicalOperatorVariantAnd && leftBool == values.False {
|
||||
if left.Type() == types.Boolean {
|
||||
return values.False, nil
|
||||
}
|
||||
@ -79,7 +80,7 @@ func (operator *LogicalOperator) Exec(ctx context.Context, scope *core.Scope) (c
|
||||
return left, nil
|
||||
}
|
||||
|
||||
if operator.value == LogicalOperatorTypeOr && leftBool == values.True {
|
||||
if operator.variant == LogicalOperatorVariantOr && leftBool == values.True {
|
||||
return left, nil
|
||||
}
|
||||
|
||||
@ -93,13 +94,13 @@ func (operator *LogicalOperator) Exec(ctx context.Context, scope *core.Scope) (c
|
||||
}
|
||||
|
||||
func (operator *LogicalOperator) Eval(_ context.Context, left, right core.Value) (core.Value, error) {
|
||||
if operator.value == LogicalOperatorTypeNot {
|
||||
if operator.variant == LogicalOperatorVariantNot {
|
||||
return Not(right, values.None), nil
|
||||
}
|
||||
|
||||
leftBool := values.ToBoolean(left)
|
||||
|
||||
if operator.value == LogicalOperatorTypeAnd && leftBool == values.False {
|
||||
if operator.variant == LogicalOperatorVariantAnd && leftBool == values.False {
|
||||
if left.Type() == types.Boolean {
|
||||
return values.False, nil
|
||||
}
|
||||
@ -107,7 +108,7 @@ func (operator *LogicalOperator) Eval(_ context.Context, left, right core.Value)
|
||||
return left, nil
|
||||
}
|
||||
|
||||
if operator.value == LogicalOperatorTypeOr && leftBool == values.True {
|
||||
if operator.variant == LogicalOperatorVariantOr && leftBool == values.True {
|
||||
return left, nil
|
||||
}
|
||||
|
||||
|
@ -7,43 +7,44 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
MathOperatorType string
|
||||
MathOperatorVariant string
|
||||
|
||||
MathOperator struct {
|
||||
*baseOperator
|
||||
opType MathOperatorType
|
||||
variant MathOperatorVariant
|
||||
fn OperatorFunc
|
||||
leftOnly bool
|
||||
}
|
||||
)
|
||||
|
||||
const (
|
||||
MathOperatorTypeAdd MathOperatorType = "+"
|
||||
MathOperatorTypeSubtract MathOperatorType = "-"
|
||||
MathOperatorTypeMultiply MathOperatorType = "*"
|
||||
MathOperatorTypeDivide MathOperatorType = "/"
|
||||
MathOperatorTypeModulus MathOperatorType = "%"
|
||||
MathOperatorTypeIncrement MathOperatorType = "++"
|
||||
MathOperatorTypeDecrement MathOperatorType = "--"
|
||||
MathOperatorVariantAdd MathOperatorVariant = "+"
|
||||
MathOperatorVariantSubtract MathOperatorVariant = "-"
|
||||
MathOperatorVariantMultiply MathOperatorVariant = "*"
|
||||
MathOperatorVariantDivide MathOperatorVariant = "/"
|
||||
MathOperatorVariantModulus MathOperatorVariant = "%"
|
||||
MathOperatorVariantIncrement MathOperatorVariant = "++"
|
||||
MathOperatorVariantDecrement MathOperatorVariant = "--"
|
||||
)
|
||||
|
||||
var mathOperators = map[MathOperatorType]OperatorFunc{
|
||||
MathOperatorTypeAdd: Add,
|
||||
MathOperatorTypeSubtract: Subtract,
|
||||
MathOperatorTypeMultiply: Multiply,
|
||||
MathOperatorTypeDivide: Divide,
|
||||
MathOperatorTypeModulus: Modulus,
|
||||
MathOperatorTypeIncrement: Increment,
|
||||
MathOperatorTypeDecrement: Decrement,
|
||||
var mathOperatorVariants = map[MathOperatorVariant]OperatorFunc{
|
||||
MathOperatorVariantAdd: Add,
|
||||
MathOperatorVariantSubtract: Subtract,
|
||||
MathOperatorVariantMultiply: Multiply,
|
||||
MathOperatorVariantDivide: Divide,
|
||||
MathOperatorVariantModulus: Modulus,
|
||||
MathOperatorVariantIncrement: Increment,
|
||||
MathOperatorVariantDecrement: Decrement,
|
||||
}
|
||||
|
||||
func NewMathOperator(
|
||||
src core.SourceMap,
|
||||
left core.Expression,
|
||||
right core.Expression,
|
||||
operator MathOperatorType,
|
||||
variantStr string,
|
||||
) (*MathOperator, error) {
|
||||
fn, exists := mathOperators[operator]
|
||||
variant := MathOperatorVariant(variantStr)
|
||||
fn, exists := mathOperatorVariants[variant]
|
||||
|
||||
if !exists {
|
||||
return nil, core.Error(core.ErrInvalidArgument, "operator type")
|
||||
@ -51,20 +52,20 @@ func NewMathOperator(
|
||||
|
||||
var leftOnly bool
|
||||
|
||||
if operator == "++" || operator == "--" {
|
||||
if variant == "++" || variant == "--" {
|
||||
leftOnly = true
|
||||
}
|
||||
|
||||
return &MathOperator{
|
||||
&baseOperator{src, left, right},
|
||||
operator,
|
||||
variant,
|
||||
fn,
|
||||
leftOnly,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (operator *MathOperator) Type() MathOperatorType {
|
||||
return operator.opType
|
||||
func (operator *MathOperator) Type() MathOperatorVariant {
|
||||
return operator.variant
|
||||
}
|
||||
|
||||
func (operator *MathOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
|
@ -8,30 +8,31 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
RegexpOperatorType int
|
||||
RegexpOperator struct {
|
||||
RegexpOperatorVariant int
|
||||
|
||||
RegexpOperator struct {
|
||||
*baseOperator
|
||||
opType RegexpOperatorType
|
||||
variant RegexpOperatorVariant
|
||||
}
|
||||
)
|
||||
|
||||
const (
|
||||
RegexpOperatorTypeNegative RegexpOperatorType = 0
|
||||
RegexpOperatorTypePositive RegexpOperatorType = 1
|
||||
RegexpOperatorVariantNegative RegexpOperatorVariant = 0
|
||||
RegexpOperatorVariantPositive RegexpOperatorVariant = 1
|
||||
)
|
||||
|
||||
var regexpOperators = map[string]RegexpOperatorType{
|
||||
"!~": RegexpOperatorTypeNegative,
|
||||
"=~": RegexpOperatorTypePositive,
|
||||
var regexpVariants = map[string]RegexpOperatorVariant{
|
||||
"!~": RegexpOperatorVariantNegative,
|
||||
"=~": RegexpOperatorVariantPositive,
|
||||
}
|
||||
|
||||
func NewRegexpOperator(
|
||||
src core.SourceMap,
|
||||
left core.Expression,
|
||||
right core.Expression,
|
||||
operator string,
|
||||
operatorStr string,
|
||||
) (*RegexpOperator, error) {
|
||||
op, exists := regexpOperators[operator]
|
||||
variant, exists := regexpVariants[operatorStr]
|
||||
|
||||
if !exists {
|
||||
return nil, core.Error(core.ErrInvalidArgument, "operator")
|
||||
@ -43,12 +44,12 @@ func NewRegexpOperator(
|
||||
left,
|
||||
right,
|
||||
},
|
||||
op,
|
||||
variant,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (operator *RegexpOperator) Type() RegexpOperatorType {
|
||||
return operator.opType
|
||||
func (operator *RegexpOperator) Type() RegexpOperatorVariant {
|
||||
return operator.variant
|
||||
}
|
||||
|
||||
func (operator *RegexpOperator) Exec(ctx context.Context, scope *core.Scope) (core.Value, error) {
|
||||
@ -77,7 +78,7 @@ func (operator *RegexpOperator) Eval(_ context.Context, left, right core.Value)
|
||||
return values.None, err
|
||||
}
|
||||
|
||||
if operator.opType == RegexpOperatorTypePositive {
|
||||
if operator.variant == RegexpOperatorVariantPositive {
|
||||
return values.NewBoolean(r.MatchString(leftStr)), nil
|
||||
}
|
||||
|
||||
|
@ -8,35 +8,37 @@ import (
|
||||
)
|
||||
|
||||
type (
|
||||
UnaryOperatorType string
|
||||
UnaryOperator struct {
|
||||
UnaryOperatorVariant string
|
||||
|
||||
UnaryOperator struct {
|
||||
*baseOperator
|
||||
fn OperatorFunc
|
||||
}
|
||||
)
|
||||
|
||||
const (
|
||||
UnaryOperatorTypeNoop UnaryOperatorType = ""
|
||||
UnaryOperatorTypeNot UnaryOperatorType = "!"
|
||||
UnaryOperatorTypeNot2 UnaryOperatorType = "NOT"
|
||||
UnaryOperatorTypeNegative UnaryOperatorType = "-"
|
||||
UnaryOperatorTypePositive UnaryOperatorType = "+"
|
||||
UnaryOperatorVariantNoop UnaryOperatorVariant = ""
|
||||
UnaryOperatorVariantNot UnaryOperatorVariant = "!"
|
||||
UnaryOperatorVariantNot2 UnaryOperatorVariant = "NOT"
|
||||
UnaryOperatorVariantNegative UnaryOperatorVariant = "-"
|
||||
UnaryOperatorVariantPositive UnaryOperatorVariant = "+"
|
||||
)
|
||||
|
||||
var unaryOperators = map[UnaryOperatorType]OperatorFunc{
|
||||
UnaryOperatorTypeNoop: ToBoolean,
|
||||
UnaryOperatorTypeNot: Not,
|
||||
UnaryOperatorTypeNot2: Not,
|
||||
UnaryOperatorTypeNegative: Negative,
|
||||
UnaryOperatorTypePositive: Positive,
|
||||
var unaryOperatorVariants = map[UnaryOperatorVariant]OperatorFunc{
|
||||
UnaryOperatorVariantNoop: ToBoolean,
|
||||
UnaryOperatorVariantNot: Not,
|
||||
UnaryOperatorVariantNot2: Not,
|
||||
UnaryOperatorVariantNegative: Negative,
|
||||
UnaryOperatorVariantPositive: Positive,
|
||||
}
|
||||
|
||||
func NewUnaryOperator(
|
||||
src core.SourceMap,
|
||||
exp core.Expression,
|
||||
operator UnaryOperatorType,
|
||||
variantStr string,
|
||||
) (*UnaryOperator, error) {
|
||||
fn, exists := unaryOperators[operator]
|
||||
variant := UnaryOperatorVariant(variantStr)
|
||||
fn, exists := unaryOperatorVariants[variant]
|
||||
|
||||
if !exists {
|
||||
return nil, core.Error(core.ErrInvalidArgument, "operator")
|
||||
|
Loading…
Reference in New Issue
Block a user