Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: support incremental event sliding window #3421

Merged
merged 6 commits into from
Dec 6, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
147 changes: 147 additions & 0 deletions internal/topo/node/window_inc_agg_event_op.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,147 @@
// Copyright 2024 EMQ Technologies Co., Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package node

import (
"time"

"github.com/lf-edge/ekuiper/contract/v2/api"

"github.com/lf-edge/ekuiper/v2/internal/xsql"
)

type SlidingWindowIncAggEventOp struct {
*SlidingWindowIncAggOp
EmitList []*IncAggWindow
}

func NewSlidingWindowIncAggEventOp(o *WindowIncAggOperator) *SlidingWindowIncAggEventOp {
op := &SlidingWindowIncAggEventOp{}
op.SlidingWindowIncAggOp = NewSlidingWindowIncAggOp(o)
op.EmitList = make([]*IncAggWindow, 0)
return op
}

func (so *SlidingWindowIncAggEventOp) exec(ctx api.StreamContext, errCh chan<- error) {
fv, _ := xsql.NewFunctionValuersForOp(ctx)
for {
select {
case <-ctx.Done():
return
case input := <-so.input:
data, processed := so.ingest(ctx, input)
if processed {
break

Check warning on line 46 in internal/topo/node/window_inc_agg_event_op.go

View check run for this annotation

Codecov / codecov/patch

internal/topo/node/window_inc_agg_event_op.go#L46

Added line #L46 was not covered by tests
}
switch tuple := data.(type) {
case *xsql.WatermarkTuple:
now := tuple.GetTimestamp()
so.emitList(ctx, errCh, now)
so.CurrWindowList = gcIncAggWindow(so.CurrWindowList, so.Length, now)
case *xsql.Tuple:
if so.Delay > 0 {
so.appendDelayIncAggWindowInEvent(ctx, errCh, fv, tuple)
continue
}
so.appendIncAggWindowInEvent(ctx, errCh, fv, tuple)
}
}
}
}

func (so *SlidingWindowIncAggEventOp) emitList(ctx api.StreamContext, errCh chan<- error, triggerTS time.Time) {
if len(so.EmitList) > 0 {
triggerIndex := -1
for index, window := range so.EmitList {
if window.EventTime.Add(so.Delay).Compare(triggerTS) <= 0 {
triggerIndex = index
so.emit(ctx, errCh, window, triggerTS)
} else {
break
}
}
// emit nothing
if triggerIndex == -1 {
return
}

Check warning on line 78 in internal/topo/node/window_inc_agg_event_op.go

View check run for this annotation

Codecov / codecov/patch

internal/topo/node/window_inc_agg_event_op.go#L77-L78

Added lines #L77 - L78 were not covered by tests
// emit all windows
if triggerIndex >= len(so.EmitList)-1 {
so.EmitList = make([]*IncAggWindow, 0)
return
}
// emit part of windows
so.EmitList = so.EmitList[triggerIndex+1:]
}
}

func (so *SlidingWindowIncAggEventOp) appendIncAggWindowInEvent(ctx api.StreamContext, errCh chan<- error, fv *xsql.FunctionValuer, row *xsql.Tuple) {
now := row.GetTimestamp()
name := calDimension(fv, so.Dimensions, row)
if so.isMatchCondition(ctx, fv, row) {
so.CurrWindowList = append(so.CurrWindowList, newIncAggWindow(ctx, now))
}
for _, incWindow := range so.CurrWindowList {
if incWindow.StartTime.Compare(now) <= 0 && incWindow.StartTime.Add(so.Length).After(now) {
incAggCal(ctx, name, row, incWindow, so.aggFields)
}
}
if so.isMatchCondition(ctx, fv, row) {
emitWindow := so.CurrWindowList[0].Clone(ctx)
emitWindow.StartTime = row.GetTimestamp()
so.EmitList = append(so.EmitList, emitWindow)
}
return
}

func (so *SlidingWindowIncAggEventOp) appendDelayIncAggWindowInEvent(ctx api.StreamContext, errCh chan<- error, fv *xsql.FunctionValuer, row *xsql.Tuple) {
now := row.GetTimestamp()
name := calDimension(fv, so.Dimensions, row)
so.CurrWindowList = append(so.CurrWindowList, newIncAggWindow(ctx, row.GetTimestamp()))
for _, incWindow := range so.CurrWindowList {
if incWindow.StartTime.Compare(now) <= 0 && incWindow.StartTime.Add(so.Length).After(now) {
incAggCal(ctx, name, row, incWindow, so.aggFields)
}
}
for _, incWindow := range so.EmitList {
if incWindow.EventTime.Compare(now) <= 0 && incWindow.EventTime.Add(so.Delay).After(now) {
incAggCal(ctx, name, row, incWindow, so.aggFields)
}
}
if so.isMatchCondition(ctx, fv, row) {
emitWindow := so.CurrWindowList[0].Clone(ctx)
emitWindow.EventTime = row.GetTimestamp()
so.EmitList = append(so.EmitList, emitWindow)
}
}

func (o *WindowIncAggOperator) ingest(ctx api.StreamContext, item any) (any, bool) {
ctx.GetLogger().Debugf("receive %v", item)
item, processed := o.preprocess(ctx, item)
if processed {
return item, processed
}

Check warning on line 134 in internal/topo/node/window_inc_agg_event_op.go

View check run for this annotation

Codecov / codecov/patch

internal/topo/node/window_inc_agg_event_op.go#L133-L134

Added lines #L133 - L134 were not covered by tests
switch d := item.(type) {
case error:
if o.sendError {
o.Broadcast(d)
}
return nil, true
case xsql.EOFTuple:
o.Broadcast(d)
return nil, true

Check warning on line 143 in internal/topo/node/window_inc_agg_event_op.go

View check run for this annotation

Codecov / codecov/patch

internal/topo/node/window_inc_agg_event_op.go#L136-L143

Added lines #L136 - L143 were not covered by tests
}
// watermark tuple should return
return item, false
}
158 changes: 158 additions & 0 deletions internal/topo/node/window_inc_agg_event_op_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,158 @@
// Copyright 2024 EMQ Technologies Co., Ltd.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

package node_test

import (
"strings"
"testing"
"time"

"github.com/stretchr/testify/require"

"github.com/lf-edge/ekuiper/v2/internal/conf"
"github.com/lf-edge/ekuiper/v2/internal/pkg/def"
"github.com/lf-edge/ekuiper/v2/internal/pkg/store"
"github.com/lf-edge/ekuiper/v2/internal/topo/node"
"github.com/lf-edge/ekuiper/v2/internal/topo/planner"
"github.com/lf-edge/ekuiper/v2/internal/xsql"
mockContext "github.com/lf-edge/ekuiper/v2/pkg/mock/context"
)

func TestIncEventSlidingWindow(t *testing.T) {
conf.IsTesting = true
o := &def.RuleOption{
BufferLength: 10,
IsEventTime: true,
}
kv, err := store.GetKV("stream")
require.NoError(t, err)
require.NoError(t, prepareStream())
sql := "select count(*) from stream group by slidingWindow(ss,10)"
stmt, err := xsql.NewParser(strings.NewReader(sql)).Parse()
require.NoError(t, err)
p, err := planner.CreateLogicalPlan(stmt, &def.RuleOption{
PlanOptimizeStrategy: &def.PlanOptimizeStrategy{
EnableIncrementalWindow: true,
},
Qos: 0,
}, kv)
require.NoError(t, err)
require.NotNil(t, p)
incPlan := extractIncWindowPlan(p)
require.NotNil(t, incPlan)
op, err := node.NewWindowIncAggOp("1", &node.WindowConfig{
Type: incPlan.WType,
Length: time.Second,
}, incPlan.Dimensions, incPlan.IncAggFuncs, o)
require.NoError(t, err)
require.NotNil(t, op)
input, _ := op.GetInput()
output := make(chan any, 10)
op.AddOutput(output, "output")
errCh := make(chan error, 10)
ctx, cancel := mockContext.NewMockContext("1", "2").WithCancel()
op.Exec(ctx, errCh)
time.Sleep(10 * time.Millisecond)
tuple1Ts := time.Now()
tuple2Ts := tuple1Ts.Add(500 * time.Millisecond)
waterMark1Ts := tuple1Ts.Add(1200 * time.Millisecond)
input <- &xsql.Tuple{Message: map[string]any{"a": int64(1)}, Timestamp: tuple1Ts}
input <- &xsql.Tuple{Message: map[string]any{"a": int64(2)}, Timestamp: tuple2Ts}
input <- &xsql.WatermarkTuple{Timestamp: waterMark1Ts}
got1 := <-output
wt, ok := got1.(*xsql.WindowTuples)
require.True(t, ok)
require.NotNil(t, wt)
d := wt.ToMaps()
require.Equal(t, []map[string]any{
{
"a": int64(1),
"inc_agg_col_1": int64(1),
},
}, d)
got2 := <-output
wt, ok = got2.(*xsql.WindowTuples)
require.True(t, ok)
require.NotNil(t, wt)
d = wt.ToMaps()
require.Equal(t, []map[string]any{
{
"a": int64(2),
"inc_agg_col_1": int64(2),
},
}, d)
cancel()
time.Sleep(10 * time.Millisecond)
op.Close()
}

func TestIncEventDelaySlidingWindow(t *testing.T) {
conf.IsTesting = true
o := &def.RuleOption{
BufferLength: 10,
IsEventTime: true,
}
kv, err := store.GetKV("stream")
require.NoError(t, err)
require.NoError(t, prepareStream())
sql := "select count(*) from stream group by slidingWindow(ss,1,1)"
stmt, err := xsql.NewParser(strings.NewReader(sql)).Parse()
require.NoError(t, err)
p, err := planner.CreateLogicalPlan(stmt, &def.RuleOption{
PlanOptimizeStrategy: &def.PlanOptimizeStrategy{
EnableIncrementalWindow: true,
},
Qos: 0,
}, kv)
require.NoError(t, err)
require.NotNil(t, p)
incPlan := extractIncWindowPlan(p)
require.NotNil(t, incPlan)
op, err := node.NewWindowIncAggOp("1", &node.WindowConfig{
Type: incPlan.WType,
Length: time.Second,
Delay: time.Second,
}, incPlan.Dimensions, incPlan.IncAggFuncs, o)
require.NoError(t, err)
require.NotNil(t, op)
input, _ := op.GetInput()
output := make(chan any, 10)
op.AddOutput(output, "output")
errCh := make(chan error, 10)
ctx, cancel := mockContext.NewMockContext("1", "2").WithCancel()
op.Exec(ctx, errCh)
time.Sleep(10 * time.Millisecond)

tuple1Ts := time.Now()
tuple2Ts := tuple1Ts.Add(500 * time.Millisecond)
waterMark1Ts := tuple1Ts.Add(1100 * time.Millisecond)
input <- &xsql.Tuple{Message: map[string]any{"a": int64(1)}, Timestamp: tuple1Ts}
input <- &xsql.Tuple{Message: map[string]any{"a": int64(2)}, Timestamp: tuple2Ts}
input <- &xsql.WatermarkTuple{Timestamp: waterMark1Ts}
got1 := <-output
wt, ok := got1.(*xsql.WindowTuples)
require.True(t, ok)
require.NotNil(t, wt)
d := wt.ToMaps()
require.Equal(t, []map[string]any{
{
"a": int64(2),
"inc_agg_col_1": int64(2),
},
}, d)
cancel()
time.Sleep(10 * time.Millisecond)
op.Close()
}
Loading
Loading