浏览代码

refactor(*): rename plural packages

Signed-off-by: ngjaying <ngjaying@gmail.com>
ngjaying 3 年之前
父节点
当前提交
225dbb81e5
共有 77 个文件被更改,包括 257 次插入257 次删除
  1. 3 3
      internal/processor/rule.go
  2. 8 8
      internal/server/rpc.go
  3. 1 1
      internal/topo/checkpoints/barrier_handler.go
  4. 1 1
      internal/topo/checkpoints/coordinator.go
  5. 1 1
      internal/topo/checkpoints/defs.go
  6. 1 1
      internal/topo/checkpoints/responder.go
  7. 1 1
      internal/topo/collectors/func.go
  8. 1 1
      internal/topo/contexts/default.go
  9. 4 4
      internal/topo/contexts/default_test.go
  10. 1 1
      internal/topo/contexts/func_context.go
  11. 1 1
      internal/topo/nodes/dynamic_channel_buffer.go
  12. 1 1
      internal/topo/nodes/join_align_node.go
  13. 7 7
      internal/topo/nodes/node.go
  14. 1 1
      internal/topo/nodes/operations.go
  15. 1 1
      internal/topo/nodes/prometheus.go
  16. 4 4
      internal/topo/nodes/sink_cache.go
  17. 7 7
      internal/topo/nodes/sink_node.go
  18. 5 5
      internal/topo/nodes/sink_node_test.go
  19. 5 5
      internal/topo/nodes/source_node.go
  20. 4 4
      internal/topo/nodes/source_node_test.go
  21. 3 3
      internal/topo/nodes/source_pool.go
  22. 9 9
      internal/topo/nodes/source_pool_test.go
  23. 7 7
      internal/topo/nodes/sources_for_test_with_edgex.go
  24. 3 3
      internal/topo/nodes/sources_for_test_without_edgex.go
  25. 5 5
      internal/topo/nodes/sources_with_edgex.go
  26. 1 1
      internal/topo/nodes/sources_without_edgex.go
  27. 1 1
      internal/topo/nodes/stats_manager.go
  28. 1 1
      internal/topo/nodes/watermark.go
  29. 1 1
      internal/topo/nodes/window_op.go
  30. 1 1
      internal/topo/nodes/window_op_test.go
  31. 1 1
      internal/topo/operators/aggregate_operator.go
  32. 4 4
      internal/topo/operators/aggregate_test.go
  33. 1 1
      internal/topo/operators/field_processor.go
  34. 1 1
      internal/topo/operators/filter_operator.go
  35. 4 4
      internal/topo/operators/filter_test.go
  36. 1 1
      internal/topo/operators/having_operator.go
  37. 5 5
      internal/topo/operators/having_test.go
  38. 3 3
      internal/topo/operators/join_multi_test.go
  39. 1 1
      internal/topo/operators/join_operator.go
  40. 8 8
      internal/topo/operators/join_test.go
  41. 3 3
      internal/topo/operators/math_func_test.go
  42. 6 6
      internal/topo/operators/misc_func_test.go
  43. 1 1
      internal/topo/operators/order_operator.go
  44. 3 3
      internal/topo/operators/order_test.go
  45. 1 1
      internal/topo/operators/preprocessor.go
  46. 7 7
      internal/topo/operators/preprocessor_test.go
  47. 1 1
      internal/topo/operators/project_operator.go
  48. 7 7
      internal/topo/operators/project_test.go
  49. 3 3
      internal/topo/operators/str_func_test.go
  50. 1 1
      internal/topo/operators/table_processor.go
  51. 3 3
      internal/topo/operators/table_processor_test.go
  52. 26 26
      internal/topo/planner/planner.go
  53. 1 1
      internal/topo/sinks/edgex_sink.go
  54. 3 3
      internal/topo/sinks/edgex_sink_test.go
  55. 6 6
      internal/topo/sinks/log_sink.go
  56. 1 1
      internal/topo/sinks/mqtt_sink.go
  57. 1 1
      internal/topo/sinks/nop_sink.go
  58. 1 1
      internal/topo/sinks/rest_sink.go
  59. 4 4
      internal/topo/sinks/rest_sink_test.go
  60. 1 1
      internal/topo/extensions/edgex_source.go
  61. 1 1
      internal/topo/extensions/edgex_source_test.go
  62. 1 1
      internal/topo/extensions/file_source.go
  63. 1 1
      internal/topo/extensions/httppull_source.go
  64. 1 1
      internal/topo/extensions/mqtt_source.go
  65. 1 1
      internal/topo/extensions/mqtt_type.go
  66. 1 1
      internal/topo/extensions/mqtt_type_test.go
  67. 4 4
      internal/topo/states/kv_store.go
  68. 2 2
      internal/topo/states/kv_store_test.go
  69. 1 1
      internal/topo/states/memory_store.go
  70. 1 1
      internal/topo/states/store.go
  71. 28 28
      internal/topo/topo.go
  72. 9 9
      internal/topo/topotest/mock_topo.go
  73. 1 1
      internal/topo/topotest/mocknodes/mock_data.go
  74. 1 1
      internal/topo/topotest/mocknodes/mock_sink.go
  75. 1 1
      internal/topo/topotest/mocknodes/mock_source.go
  76. 2 2
      internal/topo/topotest/rule_test.go
  77. 2 2
      internal/xsql/functionRuntime.go

+ 3 - 3
internal/processor/rule.go

@@ -6,7 +6,7 @@ import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/topo"
 	"github.com/emqx/kuiper/internal/topo"
-	"github.com/emqx/kuiper/internal/topo/nodes"
+	"github.com/emqx/kuiper/internal/topo/node"
 	"github.com/emqx/kuiper/internal/topo/planner"
 	"github.com/emqx/kuiper/internal/topo/planner"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
@@ -180,7 +180,7 @@ func (p *RuleProcessor) getRuleByJson(name, ruleJson string) (*api.Rule, error)
 }
 }
 
 
 func (p *RuleProcessor) ExecQuery(ruleid, sql string) (*topo.Topo, error) {
 func (p *RuleProcessor) ExecQuery(ruleid, sql string) (*topo.Topo, error) {
-	if tp, err := planner.PlanWithSourcesAndSinks(p.getDefaultRule(ruleid, sql), p.rootDbDir, nil, []*nodes.SinkNode{nodes.NewSinkNode("sink_memory_log", "logToMemory", nil)}); err != nil {
+	if tp, err := planner.PlanWithSourcesAndSinks(p.getDefaultRule(ruleid, sql), p.rootDbDir, nil, []*node.SinkNode{node.NewSinkNode("sink_memory_log", "logToMemory", nil)}); err != nil {
 		return nil, err
 		return nil, err
 	} else {
 	} else {
 		go func() {
 		go func() {
@@ -257,7 +257,7 @@ func (p *RuleProcessor) ExecDrop(name string) (string, error) {
 
 
 func cleanCheckpoint(name string) error {
 func cleanCheckpoint(name string) error {
 	dbDir, _ := conf.GetDataLoc()
 	dbDir, _ := conf.GetDataLoc()
-	c := path.Join(dbDir, "checkpoints", name)
+	c := path.Join(dbDir, name)
 	return os.RemoveAll(c)
 	return os.RemoveAll(c)
 }
 }
 
 

+ 8 - 8
internal/server/rpc.go

@@ -6,7 +6,7 @@ import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/plugin"
 	"github.com/emqx/kuiper/internal/plugin"
 	"github.com/emqx/kuiper/internal/service"
 	"github.com/emqx/kuiper/internal/service"
-	"github.com/emqx/kuiper/internal/topo/sinks"
+	"github.com/emqx/kuiper/internal/topo/sink"
 	"strings"
 	"strings"
 	"time"
 	"time"
 )
 )
@@ -51,15 +51,15 @@ func (t *Server) GetQueryResult(qid string, reply *string) error {
 		}
 		}
 	}
 	}
 
 
-	sinks.QR.LastFetch = time.Now()
-	sinks.QR.Mux.Lock()
-	if len(sinks.QR.Results) > 0 {
-		*reply = strings.Join(sinks.QR.Results, "")
-		sinks.QR.Results = make([]string, 10)
+	sink.QR.LastFetch = time.Now()
+	sink.QR.Mux.Lock()
+	if len(sink.QR.Results) > 0 {
+		*reply = strings.Join(sink.QR.Results, "")
+		sink.QR.Results = make([]string, 10)
 	} else {
 	} else {
 		*reply = ""
 		*reply = ""
 	}
 	}
-	sinks.QR.Mux.Unlock()
+	sink.QR.Mux.Unlock()
 	return nil
 	return nil
 }
 }
 
 
@@ -432,7 +432,7 @@ func init() {
 
 
 			n := time.Now()
 			n := time.Now()
 			w := 10 * time.Second
 			w := 10 * time.Second
-			if v := n.Sub(sinks.QR.LastFetch); v >= w {
+			if v := n.Sub(sink.QR.LastFetch); v >= w {
 				logger.Printf("The client seems no longer fetch the query result, stop the query now.")
 				logger.Printf("The client seems no longer fetch the query result, stop the query now.")
 				stopQuery()
 				stopQuery()
 				ticker.Stop()
 				ticker.Stop()

+ 1 - 1
internal/topo/checkpoints/barrier_handler.go

@@ -1,4 +1,4 @@
-package checkpoints
+package checkpoint
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/checkpoints/coordinator.go

@@ -1,4 +1,4 @@
-package checkpoints
+package checkpoint
 
 
 import (
 import (
 	"github.com/benbjohnson/clock"
 	"github.com/benbjohnson/clock"

+ 1 - 1
internal/topo/checkpoints/defs.go

@@ -1,4 +1,4 @@
-package checkpoints
+package checkpoint
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/checkpoints/responder.go

@@ -1,4 +1,4 @@
-package checkpoints
+package checkpoint
 
 
 import "fmt"
 import "fmt"
 
 

+ 1 - 1
internal/topo/collectors/func.go

@@ -1,4 +1,4 @@
-package collectors
+package collector
 
 
 import (
 import (
 	"errors"
 	"errors"

+ 1 - 1
internal/topo/contexts/default.go

@@ -1,4 +1,4 @@
-package contexts
+package context
 
 
 import (
 import (
 	"context"
 	"context"

+ 4 - 4
internal/topo/contexts/default_test.go

@@ -1,8 +1,8 @@
-package contexts
+package context
 
 
 import (
 import (
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/states"
+	"github.com/emqx/kuiper/internal/topo/state"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"log"
 	"log"
 	"os"
 	"os"
@@ -24,7 +24,7 @@ func TestState(t *testing.T) {
 		}
 		}
 	)
 	)
 	//initialization
 	//initialization
-	store, err := states.CreateStore(ruleId, api.AtLeastOnce)
+	store, err := state.CreateStore(ruleId, api.AtLeastOnce)
 	if err != nil {
 	if err != nil {
 		t.Errorf("Get store for rule %s error: %s", ruleId, err)
 		t.Errorf("Get store for rule %s error: %s", ruleId, err)
 		return
 		return
@@ -81,7 +81,7 @@ func cleanStateData() {
 	if err != nil {
 	if err != nil {
 		log.Panic(err)
 		log.Panic(err)
 	}
 	}
-	c := path.Join(dbDir, "checkpoints")
+	c := path.Join(dbDir, state.CheckpointListKey)
 	err = os.RemoveAll(c)
 	err = os.RemoveAll(c)
 	if err != nil {
 	if err != nil {
 		conf.Log.Error(err)
 		conf.Log.Error(err)

+ 1 - 1
internal/topo/contexts/func_context.go

@@ -1,4 +1,4 @@
-package contexts
+package context
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 1 - 1
internal/topo/nodes/dynamic_channel_buffer.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/nodes/join_align_node.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 7 - 7
internal/topo/nodes/node.go

@@ -1,9 +1,9 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/checkpoints"
+	"github.com/emqx/kuiper/internal/topo/checkpoint"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"gopkg.in/yaml.v3"
 	"gopkg.in/yaml.v3"
@@ -18,7 +18,7 @@ type OperatorNode interface {
 	GetInputCount() int
 	GetInputCount() int
 	AddInputCount()
 	AddInputCount()
 	SetQos(api.Qos)
 	SetQos(api.Qos)
-	SetBarrierHandler(checkpoints.BarrierHandler)
+	SetBarrierHandler(checkpoint.BarrierHandler)
 }
 }
 
 
 type DataSourceNode interface {
 type DataSourceNode interface {
@@ -81,7 +81,7 @@ func (o *defaultNode) Broadcast(val interface{}) error {
 	}
 	}
 
 
 	if o.qos >= api.AtLeastOnce {
 	if o.qos >= api.AtLeastOnce {
-		boe := &checkpoints.BufferOrEvent{
+		boe := &checkpoint.BufferOrEvent{
 			Data:    val,
 			Data:    val,
 			Channel: o.name,
 			Channel: o.name,
 		}
 		}
@@ -117,7 +117,7 @@ func (o *defaultNode) GetStreamContext() api.StreamContext {
 type defaultSinkNode struct {
 type defaultSinkNode struct {
 	*defaultNode
 	*defaultNode
 	input          chan interface{}
 	input          chan interface{}
-	barrierHandler checkpoints.BarrierHandler
+	barrierHandler checkpoint.BarrierHandler
 	inputCount     int
 	inputCount     int
 }
 }
 
 
@@ -133,7 +133,7 @@ func (o *defaultSinkNode) AddInputCount() {
 	o.inputCount++
 	o.inputCount++
 }
 }
 
 
-func (o *defaultSinkNode) SetBarrierHandler(bh checkpoints.BarrierHandler) {
+func (o *defaultSinkNode) SetBarrierHandler(bh checkpoint.BarrierHandler) {
 	o.barrierHandler = bh
 	o.barrierHandler = bh
 }
 }
 
 
@@ -142,7 +142,7 @@ func (o *defaultSinkNode) preprocess(data interface{}) (interface{}, bool) {
 	if o.qos >= api.AtLeastOnce {
 	if o.qos >= api.AtLeastOnce {
 		logger := o.ctx.GetLogger()
 		logger := o.ctx.GetLogger()
 		logger.Debugf("%s preprocess receive data %+v", o.name, data)
 		logger.Debugf("%s preprocess receive data %+v", o.name, data)
-		b, ok := data.(*checkpoints.BufferOrEvent)
+		b, ok := data.(*checkpoint.BufferOrEvent)
 		if ok {
 		if ok {
 			logger.Debugf("data is BufferOrEvent, start barrier handler")
 			logger.Debugf("data is BufferOrEvent, start barrier handler")
 			//if it is barrier return true and ignore the further processing
 			//if it is barrier return true and ignore the further processing

+ 1 - 1
internal/topo/nodes/operations.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 1 - 1
internal/topo/nodes/prometheus.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/prometheus/client_golang/prometheus"
 	"github.com/prometheus/client_golang/prometheus"

+ 4 - 4
internal/topo/nodes/sink_cache.go

@@ -1,10 +1,10 @@
-package nodes
+package node
 
 
 import (
 import (
 	"encoding/gob"
 	"encoding/gob"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/checkpoints"
+	"github.com/emqx/kuiper/internal/topo/checkpoint"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/kv"
 	"github.com/emqx/kuiper/pkg/kv"
 	"io"
 	"io"
@@ -234,8 +234,8 @@ func (c *Cache) checkpointbasedRun(ctx api.StreamContext, tch <-chan struct{}) {
 		select {
 		select {
 		case item := <-c.in:
 		case item := <-c.in:
 			// possibility of barrier, ignore if found
 			// possibility of barrier, ignore if found
-			if boe, ok := item.(*checkpoints.BufferOrEvent); ok {
-				if _, ok := boe.Data.(*checkpoints.Barrier); ok {
+			if boe, ok := item.(*checkpoint.BufferOrEvent); ok {
+				if _, ok := boe.Data.(*checkpoint.Barrier); ok {
 					c.Out <- &CacheTuple{
 					c.Out <- &CacheTuple{
 						data: item,
 						data: item,
 					}
 					}

+ 7 - 7
internal/topo/nodes/sink_node.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"bytes"
 	"bytes"
@@ -7,7 +7,7 @@ import (
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/plugin"
 	"github.com/emqx/kuiper/internal/plugin"
 	ct "github.com/emqx/kuiper/internal/template"
 	ct "github.com/emqx/kuiper/internal/template"
-	"github.com/emqx/kuiper/internal/topo/sinks"
+	"github.com/emqx/kuiper/internal/topo/sink"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"sync"
 	"sync"
@@ -398,15 +398,15 @@ func doGetSink(name string, action map[string]interface{}) (api.Sink, error) {
 	)
 	)
 	switch name {
 	switch name {
 	case "log":
 	case "log":
-		s = sinks.NewLogSink()
+		s = sink.NewLogSink()
 	case "logToMemory":
 	case "logToMemory":
-		s = sinks.NewLogSinkToMemory()
+		s = sink.NewLogSinkToMemory()
 	case "mqtt":
 	case "mqtt":
-		s = &sinks.MQTTSink{}
+		s = &sink.MQTTSink{}
 	case "rest":
 	case "rest":
-		s = &sinks.RestSink{}
+		s = &sink.RestSink{}
 	case "nop":
 	case "nop":
-		s = &sinks.NopSink{}
+		s = &sink.NopSink{}
 	default:
 	default:
 		s, err = plugin.GetSink(name)
 		s, err = plugin.GetSink(name)
 		if err != nil {
 		if err != nil {

+ 5 - 5
internal/topo/nodes/sink_node_test.go

@@ -1,10 +1,10 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
-	"github.com/emqx/kuiper/internal/topo/topotest/mocknodes"
+	"github.com/emqx/kuiper/internal/topo/context"
+	"github.com/emqx/kuiper/internal/topo/topotest/mocknode"
 	"reflect"
 	"reflect"
 	"testing"
 	"testing"
 	"time"
 	"time"
@@ -92,10 +92,10 @@ func TestSinkTemplate_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestSinkTemplate_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestSinkTemplate_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 
 
 	for i, tt := range tests {
 	for i, tt := range tests {
-		mockSink := mocknodes.NewMockSink()
+		mockSink := mocknode.NewMockSink()
 		s := NewSinkNodeWithSink("mockSink", mockSink, tt.config)
 		s := NewSinkNodeWithSink("mockSink", mockSink, tt.config)
 		s.Open(ctx, make(chan error))
 		s.Open(ctx, make(chan error))
 		s.input <- tt.data
 		s.input <- tt.data

+ 5 - 5
internal/topo/nodes/source_node.go

@@ -1,9 +1,9 @@
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/plugin"
 	"github.com/emqx/kuiper/internal/plugin"
-	"github.com/emqx/kuiper/internal/topo/extensions"
+	"github.com/emqx/kuiper/internal/topo/source"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
@@ -152,11 +152,11 @@ func doGetSource(t string) (api.Source, error) {
 	)
 	)
 	switch t {
 	switch t {
 	case "mqtt":
 	case "mqtt":
-		s = &extensions.MQTTSource{}
+		s = &source.MQTTSource{}
 	case "httppull":
 	case "httppull":
-		s = &extensions.HTTPPullSource{}
+		s = &source.HTTPPullSource{}
 	case "file":
 	case "file":
-		s = &extensions.FileSource{}
+		s = &source.FileSource{}
 	default:
 	default:
 		s, err = plugin.GetSource(t)
 		s, err = plugin.GetSource(t)
 		if err != nil {
 		if err != nil {

+ 4 - 4
internal/topo/nodes/source_node_test.go

@@ -1,8 +1,8 @@
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"reflect"
 	"reflect"
@@ -33,7 +33,7 @@ func TestGetConf_Apply(t *testing.T) {
 		TYPE:       "test",
 		TYPE:       "test",
 	})
 	})
 	contextLogger := conf.Log.WithField("rule", "test")
 	contextLogger := conf.Log.WithField("rule", "test")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	conf := getSourceConf(ctx, n.sourceType, n.options)
 	conf := getSourceConf(ctx, n.sourceType, n.options)
 	if !reflect.DeepEqual(result, conf) {
 	if !reflect.DeepEqual(result, conf) {
 		t.Errorf("result mismatch:\n\nexp=%s\n\ngot=%s\n\n", result, conf)
 		t.Errorf("result mismatch:\n\nexp=%s\n\ngot=%s\n\n", result, conf)
@@ -56,7 +56,7 @@ func TestGetConfAndConvert_Apply(t *testing.T) {
 		CONF_KEY:   "dedup",
 		CONF_KEY:   "dedup",
 	})
 	})
 	contextLogger := conf.Log.WithField("rule", "test")
 	contextLogger := conf.Log.WithField("rule", "test")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	conf := getSourceConf(ctx, n.sourceType, n.options)
 	conf := getSourceConf(ctx, n.sourceType, n.options)
 	if !reflect.DeepEqual(result, conf) {
 	if !reflect.DeepEqual(result, conf) {
 		t.Errorf("result mismatch:\n\nexp=%s\n\ngot=%s\n\n", result, conf)
 		t.Errorf("result mismatch:\n\nexp=%s\n\ngot=%s\n\n", result, conf)

+ 3 - 3
internal/topo/nodes/source_pool.go

@@ -1,10 +1,10 @@
-package nodes
+package node
 
 
 import (
 import (
 	"context"
 	"context"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	kctx "github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"sync"
 	"sync"
 )
 )
@@ -95,7 +95,7 @@ func (p *sourcePool) addInstance(k string, node *SourceNode, source api.Source,
 	s, ok := p.registry[k]
 	s, ok := p.registry[k]
 	if !ok {
 	if !ok {
 		contextLogger := conf.Log.WithField("source_pool", k)
 		contextLogger := conf.Log.WithField("source_pool", k)
-		ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+		ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
 		// TODO cancel
 		// TODO cancel
 		sctx, cancel := ctx.WithCancel()
 		sctx, cancel := ctx.WithCancel()
 		si, err := start(sctx, node, source, index)
 		si, err := start(sctx, node, source, index)

+ 9 - 9
internal/topo/nodes/source_pool_test.go

@@ -1,9 +1,9 @@
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
-	"github.com/emqx/kuiper/internal/topo/states"
+	"github.com/emqx/kuiper/internal/topo/context"
+	"github.com/emqx/kuiper/internal/topo/state"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"testing"
 	"testing"
@@ -17,8 +17,8 @@ func TestSourcePool(t *testing.T) {
 	})
 	})
 	n.concurrency = 2
 	n.concurrency = 2
 	contextLogger := conf.Log.WithField("rule", "mockRule0")
 	contextLogger := conf.Log.WithField("rule", "mockRule0")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
-	tempStore, _ := states.CreateStore("mockRule0", api.AtMostOnce)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
+	tempStore, _ := state.CreateStore("mockRule0", api.AtMostOnce)
 	n.ctx = ctx.WithMeta("mockRule0", "test", tempStore)
 	n.ctx = ctx.WithMeta("mockRule0", "test", tempStore)
 	n1 := NewSourceNode("test", ast.TypeStream, &ast.Options{
 	n1 := NewSourceNode("test", ast.TypeStream, &ast.Options{
 		DATASOURCE: "demo1",
 		DATASOURCE: "demo1",
@@ -27,16 +27,16 @@ func TestSourcePool(t *testing.T) {
 	})
 	})
 
 
 	contextLogger = conf.Log.WithField("rule", "mockRule1")
 	contextLogger = conf.Log.WithField("rule", "mockRule1")
-	ctx = contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
-	tempStore, _ = states.CreateStore("mockRule1", api.AtMostOnce)
+	ctx = context.WithValue(context.Background(), context.LoggerKey, contextLogger)
+	tempStore, _ = state.CreateStore("mockRule1", api.AtMostOnce)
 	n1.ctx = ctx.WithMeta("mockRule1", "test1", tempStore)
 	n1.ctx = ctx.WithMeta("mockRule1", "test1", tempStore)
 	n2 := NewSourceNode("test2", ast.TypeStream, &ast.Options{
 	n2 := NewSourceNode("test2", ast.TypeStream, &ast.Options{
 		DATASOURCE: "demo1",
 		DATASOURCE: "demo1",
 		TYPE:       "mock",
 		TYPE:       "mock",
 	})
 	})
 	contextLogger = conf.Log.WithField("rule", "mockRule2")
 	contextLogger = conf.Log.WithField("rule", "mockRule2")
-	ctx = contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
-	tempStore, _ = states.CreateStore("mockRule2", api.AtMostOnce)
+	ctx = context.WithValue(context.Background(), context.LoggerKey, contextLogger)
+	tempStore, _ = state.CreateStore("mockRule2", api.AtMostOnce)
 	n2.ctx = ctx.WithMeta("mockRule2", "test2", tempStore)
 	n2.ctx = ctx.WithMeta("mockRule2", "test2", tempStore)
 
 
 	// Test add source instance
 	// Test add source instance

+ 7 - 7
internal/topo/nodes/sources_for_test_with_edgex.go

@@ -1,27 +1,27 @@
 // +build test
 // +build test
 // +build edgex
 // +build edgex
 
 
-package nodes
+package node
 
 
 import (
 import (
-	"github.com/emqx/kuiper/internal/topo/extensions"
-	"github.com/emqx/kuiper/internal/topo/sinks"
-	"github.com/emqx/kuiper/internal/topo/topotest/mocknodes"
+	"github.com/emqx/kuiper/internal/topo/sink"
+	"github.com/emqx/kuiper/internal/topo/source"
+	"github.com/emqx/kuiper/internal/topo/topotest/mocknode"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 )
 )
 
 
 func getSource(t string) (api.Source, error) {
 func getSource(t string) (api.Source, error) {
 	if t == "edgex" {
 	if t == "edgex" {
-		return &extensions.EdgexSource{}, nil
+		return &source.EdgexSource{}, nil
 	} else if t == "mock" {
 	} else if t == "mock" {
-		return &mocknodes.MockSource{}, nil
+		return &mocknode.MockSource{}, nil
 	}
 	}
 	return doGetSource(t)
 	return doGetSource(t)
 }
 }
 
 
 func getSink(name string, action map[string]interface{}) (api.Sink, error) {
 func getSink(name string, action map[string]interface{}) (api.Sink, error) {
 	if name == "edgex" {
 	if name == "edgex" {
-		s := &sinks.EdgexMsgBusSink{}
+		s := &sink.EdgexMsgBusSink{}
 		if err := s.Configure(action); err != nil {
 		if err := s.Configure(action); err != nil {
 			return nil, err
 			return nil, err
 		} else {
 		} else {

+ 3 - 3
internal/topo/nodes/sources_for_test_without_edgex.go

@@ -1,16 +1,16 @@
 // +build !edgex
 // +build !edgex
 // +build test
 // +build test
 
 
-package nodes
+package node
 
 
 import (
 import (
-	"github.com/emqx/kuiper/internal/topo/topotest/mocknodes"
+	"github.com/emqx/kuiper/internal/topo/topotest/mocknode"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 )
 )
 
 
 func getSource(t string) (api.Source, error) {
 func getSource(t string) (api.Source, error) {
 	if t == "mock" {
 	if t == "mock" {
-		return &mocknodes.MockSource{}, nil
+		return &mocknode.MockSource{}, nil
 	}
 	}
 	return doGetSource(t)
 	return doGetSource(t)
 }
 }

+ 5 - 5
internal/topo/nodes/sources_with_edgex.go

@@ -1,24 +1,24 @@
 // +build edgex
 // +build edgex
 // +build !test
 // +build !test
 
 
-package nodes
+package node
 
 
 import (
 import (
-	"github.com/emqx/kuiper/internal/topo/extensions"
-	"github.com/emqx/kuiper/internal/topo/sinks"
+	"github.com/emqx/kuiper/internal/topo/sink"
+	"github.com/emqx/kuiper/internal/topo/source"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 )
 )
 
 
 func getSource(t string) (api.Source, error) {
 func getSource(t string) (api.Source, error) {
 	if t == "edgex" {
 	if t == "edgex" {
-		return &extensions.EdgexSource{}, nil
+		return &source.EdgexSource{}, nil
 	}
 	}
 	return doGetSource(t)
 	return doGetSource(t)
 }
 }
 
 
 func getSink(name string, action map[string]interface{}) (api.Sink, error) {
 func getSink(name string, action map[string]interface{}) (api.Sink, error) {
 	if name == "edgex" {
 	if name == "edgex" {
-		s := &sinks.EdgexMsgBusSink{}
+		s := &sink.EdgexMsgBusSink{}
 		if err := s.Configure(action); err != nil {
 		if err := s.Configure(action); err != nil {
 			return nil, err
 			return nil, err
 		} else {
 		} else {

+ 1 - 1
internal/topo/nodes/sources_without_edgex.go

@@ -1,7 +1,7 @@
 // +build !edgex
 // +build !edgex
 // +build !test
 // +build !test
 
 
-package nodes
+package node
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/nodes/stats_manager.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 1 - 1
internal/topo/nodes/watermark.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"context"
 	"context"

+ 1 - 1
internal/topo/nodes/window_op.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"encoding/gob"
 	"encoding/gob"

+ 1 - 1
internal/topo/nodes/window_op_test.go

@@ -1,4 +1,4 @@
-package nodes
+package node
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 1 - 1
internal/topo/operators/aggregate_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 4 - 4
internal/topo/operators/aggregate_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"reflect"
 	"reflect"
@@ -412,7 +412,7 @@ func TestAggregatePlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -483,7 +483,7 @@ func TestAggregatePlanError(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlanError")
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlanError")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 1 - 1
internal/topo/operators/field_processor.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/base64"
 	"encoding/base64"

+ 1 - 1
internal/topo/operators/filter_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 4 - 4
internal/topo/operators/filter_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"reflect"
 	"reflect"
@@ -381,7 +381,7 @@ func TestFilterPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestFilerPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestFilerPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -520,7 +520,7 @@ func TestFilterPlanError(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlanError")
 	contextLogger := conf.Log.WithField("rule", "TestFilterPlanError")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 1 - 1
internal/topo/operators/having_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 5 - 5
internal/topo/operators/having_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"reflect"
 	"reflect"
 	"strings"
 	"strings"
@@ -318,7 +318,7 @@ func TestHavingPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -489,7 +489,7 @@ func TestHavingPlanAlias_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -567,7 +567,7 @@ func TestHavingPlanError(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestHavingPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 3 - 3
internal/topo/operators/join_multi_test.go

@@ -1,9 +1,9 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"reflect"
 	"reflect"
@@ -499,7 +499,7 @@ func TestMultiJoinPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestMultiJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestMultiJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 1 - 1
internal/topo/operators/join_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 8 - 8
internal/topo/operators/join_test.go

@@ -1,11 +1,11 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
@@ -869,7 +869,7 @@ func TestLeftJoinPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestLeftJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestLeftJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -1431,7 +1431,7 @@ func TestInnerJoinPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestInnerJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestInnerJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -1633,7 +1633,7 @@ func TestRightJoinPlan_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestRightJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestRightJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -1905,7 +1905,7 @@ func TestFullJoinPlan_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestFullJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestFullJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -2078,7 +2078,7 @@ func TestCrossJoinPlan_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestCrossJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestCrossJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -2154,7 +2154,7 @@ func TestCrossJoinPlanError(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestCrossJoinPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestCrossJoinPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 3 - 3
internal/topo/operators/math_func_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"reflect"
 	"reflect"
 	"strings"
 	"strings"
@@ -455,7 +455,7 @@ func TestMathAndConversionFunc_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestMathAndConversionFunc_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestMathAndConversionFunc_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		//fmt.Println("Running test " + strconv.Itoa(i))
 		//fmt.Println("Running test " + strconv.Itoa(i))
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()

+ 6 - 6
internal/topo/operators/misc_func_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"reflect"
 	"reflect"
 	"strings"
 	"strings"
@@ -219,7 +219,7 @@ func TestMiscFunc_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestMiscFunc_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestMiscFunc_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil || stmt == nil {
 		if err != nil || stmt == nil {
@@ -271,7 +271,7 @@ func TestMqttFunc_Apply2(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestMqttFunc_Apply2")
 	contextLogger := conf.Log.WithField("rule", "TestMqttFunc_Apply2")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil || stmt == nil {
 		if err != nil || stmt == nil {
@@ -387,7 +387,7 @@ func TestMetaFunc_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestMetaFunc_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestMetaFunc_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil || stmt == nil {
 		if err != nil || stmt == nil {
@@ -756,7 +756,7 @@ func TestJsonPathFunc_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestJsonFunc_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestJsonFunc_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil || stmt == nil {
 		if err != nil || stmt == nil {

+ 1 - 1
internal/topo/operators/order_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 3 - 3
internal/topo/operators/order_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"reflect"
 	"reflect"
@@ -555,7 +555,7 @@ func TestOrderPlan_Apply(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestOrderPlan_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestOrderPlan_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {

+ 1 - 1
internal/topo/operators/preprocessor.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 7 - 7
internal/topo/operators/preprocessor_test.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/base64"
 	"encoding/base64"
@@ -6,7 +6,7 @@ import (
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
@@ -526,7 +526,7 @@ func TestPreprocessor_Apply(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessor_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessor_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		pp := &Preprocessor{}
 		pp := &Preprocessor{}
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
@@ -660,7 +660,7 @@ func TestPreprocessorTime_Apply(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorTime_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorTime_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		pp := &Preprocessor{}
 		pp := &Preprocessor{}
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
@@ -844,7 +844,7 @@ func TestPreprocessorEventtime_Apply(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorEventtime_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorEventtime_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 
 
 		pp := &Preprocessor{
 		pp := &Preprocessor{
@@ -931,7 +931,7 @@ func TestPreprocessorError(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorError")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorError")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 
 
 		pp := &Preprocessor{}
 		pp := &Preprocessor{}
@@ -1059,7 +1059,7 @@ func TestPreprocessorForBinary(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorForBinary")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessorForBinary")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		pp := &Preprocessor{}
 		pp := &Preprocessor{}
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
 		pp.streamFields = convertFields(tt.stmt.StreamFields)

+ 1 - 1
internal/topo/operators/project_operator.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"

+ 7 - 7
internal/topo/operators/project_test.go

@@ -1,11 +1,11 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"errors"
 	"errors"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
@@ -526,7 +526,7 @@ func TestProjectPlan_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -1131,7 +1131,7 @@ func TestProjectPlan_MultiInput(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_MultiInput")
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_MultiInput")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, _ := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, _ := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 
 
@@ -1339,7 +1339,7 @@ func TestProjectPlan_Funcs(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_Funcs")
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_Funcs")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -2128,7 +2128,7 @@ func TestProjectPlan_AggFuncs(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_AggFuncs")
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlan_AggFuncs")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil {
 		if err != nil {
@@ -2285,7 +2285,7 @@ func TestProjectPlanError(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlanError")
 	contextLogger := conf.Log.WithField("rule", "TestProjectPlanError")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, _ := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, _ := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 
 

+ 3 - 3
internal/topo/operators/str_func_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"reflect"
 	"reflect"
@@ -429,7 +429,7 @@ func TestStrFunc_Apply1(t *testing.T) {
 
 
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestStrFunc_Apply1")
 	contextLogger := conf.Log.WithField("rule", "TestStrFunc_Apply1")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		stmt, err := xsql.NewParser(strings.NewReader(tt.sql)).Parse()
 		if err != nil || stmt == nil {
 		if err != nil || stmt == nil {

+ 1 - 1
internal/topo/operators/table_processor.go

@@ -1,4 +1,4 @@
-package operators
+package operator
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 3 - 3
internal/topo/operators/table_processor_test.go

@@ -1,10 +1,10 @@
-package operators
+package operator
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
 	"reflect"
 	"reflect"
@@ -104,7 +104,7 @@ func TestTableProcessor_Apply(t *testing.T) {
 
 
 	defer conf.CloseLogger()
 	defer conf.CloseLogger()
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessor_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestPreprocessor_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 	for i, tt := range tests {
 	for i, tt := range tests {
 		pp := &TableProcessor{isBatchInput: true, emitterName: "demo"}
 		pp := &TableProcessor{isBatchInput: true, emitterName: "demo"}
 		pp.streamFields = convertFields(tt.stmt.StreamFields)
 		pp.streamFields = convertFields(tt.stmt.StreamFields)

+ 26 - 26
internal/topo/planner/planner.go

@@ -5,8 +5,8 @@ import (
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/topo"
 	"github.com/emqx/kuiper/internal/topo"
-	"github.com/emqx/kuiper/internal/topo/nodes"
-	"github.com/emqx/kuiper/internal/topo/operators"
+	"github.com/emqx/kuiper/internal/topo/node"
+	"github.com/emqx/kuiper/internal/topo/operator"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
@@ -19,7 +19,7 @@ func Plan(rule *api.Rule, storePath string) (*topo.Topo, error) {
 }
 }
 
 
 // For test only
 // For test only
-func PlanWithSourcesAndSinks(rule *api.Rule, storePath string, sources []*nodes.SourceNode, sinks []*nodes.SinkNode) (*topo.Topo, error) {
+func PlanWithSourcesAndSinks(rule *api.Rule, storePath string, sources []*node.SourceNode, sinks []*node.SinkNode) (*topo.Topo, error) {
 	sql := rule.Sql
 	sql := rule.Sql
 
 
 	conf.Log.Infof("Init rule with options %+v", rule.Options)
 	conf.Log.Infof("Init rule with options %+v", rule.Options)
@@ -53,7 +53,7 @@ func PlanWithSourcesAndSinks(rule *api.Rule, storePath string, sources []*nodes.
 	return tp, nil
 	return tp, nil
 }
 }
 
 
-func createTopo(rule *api.Rule, lp LogicalPlan, sources []*nodes.SourceNode, sinks []*nodes.SinkNode, streamsFromStmt []string) (*topo.Topo, error) {
+func createTopo(rule *api.Rule, lp LogicalPlan, sources []*node.SourceNode, sinks []*node.SinkNode, streamsFromStmt []string) (*topo.Topo, error) {
 	// Create topology
 	// Create topology
 	tp, err := topo.NewWithNameAndQos(rule.Id, rule.Options.Qos, rule.Options.CheckpointInterval)
 	tp, err := topo.NewWithNameAndQos(rule.Id, rule.Options.Qos, rule.Options.CheckpointInterval)
 	if err != nil {
 	if err != nil {
@@ -77,7 +77,7 @@ func createTopo(rule *api.Rule, lp LogicalPlan, sources []*nodes.SourceNode, sin
 				if !ok {
 				if !ok {
 					return nil, fmt.Errorf("expect map[string]interface{} type for the action properties, but found %v", action)
 					return nil, fmt.Errorf("expect map[string]interface{} type for the action properties, but found %v", action)
 				}
 				}
-				tp.AddSink(inputs, nodes.NewSinkNode(fmt.Sprintf("%s_%d", name, i), name, props))
+				tp.AddSink(inputs, node.NewSinkNode(fmt.Sprintf("%s_%d", name, i), name, props))
 			}
 			}
 		}
 		}
 	}
 	}
@@ -85,7 +85,7 @@ func createTopo(rule *api.Rule, lp LogicalPlan, sources []*nodes.SourceNode, sin
 	return tp, nil
 	return tp, nil
 }
 }
 
 
-func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []*nodes.SourceNode, streamsFromStmt []string, index int) (api.Emitter, int, error) {
+func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []*node.SourceNode, streamsFromStmt []string, index int) (api.Emitter, int, error) {
 	var inputs []api.Emitter
 	var inputs []api.Emitter
 	newIndex := index
 	newIndex := index
 	for _, c := range lp.Children() {
 	for _, c := range lp.Children() {
@@ -98,20 +98,20 @@ func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []
 	}
 	}
 	newIndex++
 	newIndex++
 	var (
 	var (
-		op  nodes.OperatorNode
+		op  node.OperatorNode
 		err error
 		err error
 	)
 	)
 	switch t := lp.(type) {
 	switch t := lp.(type) {
 	case *DataSourcePlan:
 	case *DataSourcePlan:
 		switch t.streamStmt.StreamType {
 		switch t.streamStmt.StreamType {
 		case ast.TypeStream:
 		case ast.TypeStream:
-			pp, err := operators.NewPreprocessor(t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary)
+			pp, err := operator.NewPreprocessor(t.streamFields, t.allMeta, t.metaFields, t.iet, t.timestampField, t.timestampFormat, t.isBinary)
 			if err != nil {
 			if err != nil {
 				return nil, 0, err
 				return nil, 0, err
 			}
 			}
-			var srcNode *nodes.SourceNode
+			var srcNode *node.SourceNode
 			if len(sources) == 0 {
 			if len(sources) == 0 {
-				node := nodes.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
+				node := node.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
 				srcNode = node
 				srcNode = node
 			} else {
 			} else {
 				srcNode = getMockSource(sources, string(t.name))
 				srcNode = getMockSource(sources, string(t.name))
@@ -123,16 +123,16 @@ func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []
 			op = Transform(pp, fmt.Sprintf("%d_preprocessor_%s", newIndex, t.name), options)
 			op = Transform(pp, fmt.Sprintf("%d_preprocessor_%s", newIndex, t.name), options)
 			inputs = []api.Emitter{srcNode}
 			inputs = []api.Emitter{srcNode}
 		case ast.TypeTable:
 		case ast.TypeTable:
-			pp, err := operators.NewTableProcessor(string(t.name), t.streamFields, t.streamStmt.Options)
+			pp, err := operator.NewTableProcessor(string(t.name), t.streamFields, t.streamStmt.Options)
 			if err != nil {
 			if err != nil {
 				return nil, 0, err
 				return nil, 0, err
 			}
 			}
-			var srcNode *nodes.SourceNode
+			var srcNode *node.SourceNode
 			if len(sources) > 0 {
 			if len(sources) > 0 {
 				srcNode = getMockSource(sources, string(t.name))
 				srcNode = getMockSource(sources, string(t.name))
 			}
 			}
 			if srcNode == nil {
 			if srcNode == nil {
-				srcNode = nodes.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
+				srcNode = node.NewSourceNode(string(t.name), t.streamStmt.StreamType, t.streamStmt.Options)
 			}
 			}
 			tp.AddSrc(srcNode)
 			tp.AddSrc(srcNode)
 			op = Transform(pp, fmt.Sprintf("%d_tableprocessor_%s", newIndex, t.name), options)
 			op = Transform(pp, fmt.Sprintf("%d_tableprocessor_%s", newIndex, t.name), options)
@@ -140,13 +140,13 @@ func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []
 		}
 		}
 	case *WindowPlan:
 	case *WindowPlan:
 		if t.condition != nil {
 		if t.condition != nil {
-			wfilterOp := Transform(&operators.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
+			wfilterOp := Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_windowFilter", newIndex), options)
 			wfilterOp.SetConcurrency(options.Concurrency)
 			wfilterOp.SetConcurrency(options.Concurrency)
 			tp.AddOperator(inputs, wfilterOp)
 			tp.AddOperator(inputs, wfilterOp)
 			inputs = []api.Emitter{wfilterOp}
 			inputs = []api.Emitter{wfilterOp}
 		}
 		}
 
 
-		op, err = nodes.NewWindowOp(fmt.Sprintf("%d_window", newIndex), nodes.WindowConfig{
+		op, err = node.NewWindowOp(fmt.Sprintf("%d_window", newIndex), node.WindowConfig{
 			Type:     t.wtype,
 			Type:     t.wtype,
 			Length:   t.length,
 			Length:   t.length,
 			Interval: t.interval,
 			Interval: t.interval,
@@ -155,30 +155,30 @@ func buildOps(lp LogicalPlan, tp *topo.Topo, options *api.RuleOption, sources []
 			return nil, 0, err
 			return nil, 0, err
 		}
 		}
 	case *JoinAlignPlan:
 	case *JoinAlignPlan:
-		op, err = nodes.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
+		op, err = node.NewJoinAlignNode(fmt.Sprintf("%d_join_aligner", newIndex), t.Emitters, options)
 	case *JoinPlan:
 	case *JoinPlan:
-		op = Transform(&operators.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
+		op = Transform(&operator.JoinOp{Joins: t.joins, From: t.from}, fmt.Sprintf("%d_join", newIndex), options)
 	case *FilterPlan:
 	case *FilterPlan:
-		op = Transform(&operators.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_filter", newIndex), options)
+		op = Transform(&operator.FilterOp{Condition: t.condition}, fmt.Sprintf("%d_filter", newIndex), options)
 	case *AggregatePlan:
 	case *AggregatePlan:
-		op = Transform(&operators.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
+		op = Transform(&operator.AggregateOp{Dimensions: t.dimensions}, fmt.Sprintf("%d_aggregate", newIndex), options)
 	case *HavingPlan:
 	case *HavingPlan:
-		op = Transform(&operators.HavingOp{Condition: t.condition}, fmt.Sprintf("%d_having", newIndex), options)
+		op = Transform(&operator.HavingOp{Condition: t.condition}, fmt.Sprintf("%d_having", newIndex), options)
 	case *OrderPlan:
 	case *OrderPlan:
-		op = Transform(&operators.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
+		op = Transform(&operator.OrderOp{SortFields: t.SortFields}, fmt.Sprintf("%d_order", newIndex), options)
 	case *ProjectPlan:
 	case *ProjectPlan:
-		op = Transform(&operators.ProjectOp{Fields: t.fields, IsAggregate: t.isAggregate, SendMeta: t.sendMeta}, fmt.Sprintf("%d_project", newIndex), options)
+		op = Transform(&operator.ProjectOp{Fields: t.fields, IsAggregate: t.isAggregate, SendMeta: t.sendMeta}, fmt.Sprintf("%d_project", newIndex), options)
 	default:
 	default:
 		return nil, 0, fmt.Errorf("unknown logical plan %v", t)
 		return nil, 0, fmt.Errorf("unknown logical plan %v", t)
 	}
 	}
-	if uop, ok := op.(*nodes.UnaryOperator); ok {
+	if uop, ok := op.(*node.UnaryOperator); ok {
 		uop.SetConcurrency(options.Concurrency)
 		uop.SetConcurrency(options.Concurrency)
 	}
 	}
 	tp.AddOperator(inputs, op)
 	tp.AddOperator(inputs, op)
 	return op, newIndex, nil
 	return op, newIndex, nil
 }
 }
 
 
-func getMockSource(sources []*nodes.SourceNode, name string) *nodes.SourceNode {
+func getMockSource(sources []*node.SourceNode, name string) *node.SourceNode {
 	for _, source := range sources {
 	for _, source := range sources {
 		if name == source.GetName() {
 		if name == source.GetName() {
 			return source
 			return source
@@ -311,8 +311,8 @@ func createLogicalPlan(stmt *ast.SelectStatement, opt *api.RuleOption, store kv.
 	return optimize(p)
 	return optimize(p)
 }
 }
 
 
-func Transform(op nodes.UnOperation, name string, options *api.RuleOption) *nodes.UnaryOperator {
-	operator := nodes.New(name, xsql.FuncRegisters, options)
+func Transform(op node.UnOperation, name string, options *api.RuleOption) *node.UnaryOperator {
+	operator := node.New(name, xsql.FuncRegisters, options)
 	operator.SetOperation(op)
 	operator.SetOperation(op)
 	return operator
 	return operator
 }
 }

+ 1 - 1
internal/topo/sinks/edgex_sink.go

@@ -1,6 +1,6 @@
 // +build edgex
 // +build edgex
 
 
-package sinks
+package sink
 
 
 import (
 import (
 	"encoding/base64"
 	"encoding/base64"

+ 3 - 3
internal/topo/sinks/edgex_sink_test.go

@@ -1,6 +1,6 @@
 // +build edgex
 // +build edgex
 
 
-package sinks
+package sink
 
 
 import (
 import (
 	"fmt"
 	"fmt"
@@ -8,14 +8,14 @@ import (
 	"github.com/edgexfoundry/go-mod-core-contracts/v2/dtos"
 	"github.com/edgexfoundry/go-mod-core-contracts/v2/dtos"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/testx"
 	"github.com/emqx/kuiper/internal/testx"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"reflect"
 	"reflect"
 	"testing"
 	"testing"
 )
 )
 
 
 var (
 var (
 	contextLogger = conf.Log.WithField("rule", "testEdgexSink")
 	contextLogger = conf.Log.WithField("rule", "testEdgexSink")
-	ctx           = contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx           = context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 )
 )
 
 
 func compareEvent(expected, actual *dtos.Event) bool {
 func compareEvent(expected, actual *dtos.Event) bool {

+ 6 - 6
internal/topo/sinks/log_sink.go

@@ -1,8 +1,8 @@
-package sinks
+package sink
 
 
 import (
 import (
 	"fmt"
 	"fmt"
-	"github.com/emqx/kuiper/internal/topo/collectors"
+	"github.com/emqx/kuiper/internal/topo/collector"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"sync"
 	"sync"
 	"time"
 	"time"
@@ -10,8 +10,8 @@ import (
 
 
 // NewLogSink log action, no properties now
 // NewLogSink log action, no properties now
 // example: {"log":{}}
 // example: {"log":{}}
-func NewLogSink() *collectors.FuncCollector {
-	return collectors.Func(func(ctx api.StreamContext, data interface{}) error {
+func NewLogSink() *collector.FuncCollector {
+	return collector.Func(func(ctx api.StreamContext, data interface{}) error {
 		log := ctx.GetLogger()
 		log := ctx.GetLogger()
 		log.Infof("sink result for rule %s: %s", ctx.GetRuleId(), data)
 		log.Infof("sink result for rule %s: %s", ctx.GetRuleId(), data)
 		return nil
 		return nil
@@ -26,9 +26,9 @@ type QueryResult struct {
 
 
 var QR = &QueryResult{LastFetch: time.Now()}
 var QR = &QueryResult{LastFetch: time.Now()}
 
 
-func NewLogSinkToMemory() *collectors.FuncCollector {
+func NewLogSinkToMemory() *collector.FuncCollector {
 	QR.Results = make([]string, 10)
 	QR.Results = make([]string, 10)
-	return collectors.Func(func(ctx api.StreamContext, data interface{}) error {
+	return collector.Func(func(ctx api.StreamContext, data interface{}) error {
 		QR.Mux.Lock()
 		QR.Mux.Lock()
 		QR.Results = append(QR.Results, fmt.Sprintf("%s", data))
 		QR.Results = append(QR.Results, fmt.Sprintf("%s", data))
 		QR.Mux.Unlock()
 		QR.Mux.Unlock()

+ 1 - 1
internal/topo/sinks/mqtt_sink.go

@@ -1,4 +1,4 @@
-package sinks
+package sink
 
 
 import (
 import (
 	"crypto/tls"
 	"crypto/tls"

+ 1 - 1
internal/topo/sinks/nop_sink.go

@@ -1,4 +1,4 @@
-package sinks
+package sink
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/sinks/rest_sink.go

@@ -1,4 +1,4 @@
-package sinks
+package sink
 
 
 import (
 import (
 	"crypto/tls"
 	"crypto/tls"

+ 4 - 4
internal/topo/sinks/rest_sink_test.go

@@ -1,10 +1,10 @@
-package sinks
+package sink
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"io/ioutil"
 	"io/ioutil"
 	"net/http"
 	"net/http"
 	"net/http/httptest"
 	"net/http/httptest"
@@ -152,7 +152,7 @@ func TestRestSink_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestRestSink_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestRestSink_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 
 
 	var requests []request
 	var requests []request
 	ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
 	ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
@@ -320,7 +320,7 @@ func TestRestSinkTemplate_Apply(t *testing.T) {
 	}
 	}
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	fmt.Printf("The test bucket size is %d.\n\n", len(tests))
 	contextLogger := conf.Log.WithField("rule", "TestRestSink_Apply")
 	contextLogger := conf.Log.WithField("rule", "TestRestSink_Apply")
-	ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+	ctx := context.WithValue(context.Background(), context.LoggerKey, contextLogger)
 
 
 	var requests []request
 	var requests []request
 	ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
 	ts := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {

+ 1 - 1
internal/topo/extensions/edgex_source.go

@@ -1,6 +1,6 @@
 // +build edgex
 // +build edgex
 
 
-package extensions
+package source
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"

+ 1 - 1
internal/topo/extensions/edgex_source_test.go

@@ -1,6 +1,6 @@
 // +build edgex
 // +build edgex
 
 
-package extensions
+package source
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"

+ 1 - 1
internal/topo/extensions/file_source.go

@@ -1,4 +1,4 @@
-package extensions
+package source
 
 
 import (
 import (
 	"errors"
 	"errors"

+ 1 - 1
internal/topo/extensions/httppull_source.go

@@ -1,4 +1,4 @@
-package extensions
+package source
 
 
 import (
 import (
 	"crypto/md5"
 	"crypto/md5"

+ 1 - 1
internal/topo/extensions/mqtt_source.go

@@ -1,4 +1,4 @@
-package extensions
+package source
 
 
 import (
 import (
 	"crypto/tls"
 	"crypto/tls"

+ 1 - 1
internal/topo/extensions/mqtt_type.go

@@ -1,4 +1,4 @@
-package extensions
+package source
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 1 - 1
internal/topo/extensions/mqtt_type_test.go

@@ -1,4 +1,4 @@
-package extensions
+package source
 
 
 import (
 import (
 	"encoding/json"
 	"encoding/json"

+ 4 - 4
internal/topo/states/kv_store.go

@@ -1,10 +1,10 @@
-package states
+package state
 
 
 import (
 import (
 	"encoding/gob"
 	"encoding/gob"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/checkpoints"
+	"github.com/emqx/kuiper/internal/topo/checkpoint"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/cast"
 	"github.com/emqx/kuiper/pkg/kv"
 	"github.com/emqx/kuiper/pkg/kv"
 	"path"
 	"path"
@@ -13,7 +13,7 @@ import (
 
 
 func init() {
 func init() {
 	gob.Register(map[string]interface{}{})
 	gob.Register(map[string]interface{}{})
-	gob.Register(checkpoints.BufferOrEvent{})
+	gob.Register(checkpoint.BufferOrEvent{})
 }
 }
 
 
 //The manager for checkpoint storage.
 //The manager for checkpoint storage.
@@ -35,7 +35,7 @@ type KVStore struct {
 //Assume each operator only has one instance
 //Assume each operator only has one instance
 func getKVStore(ruleId string) (*KVStore, error) {
 func getKVStore(ruleId string) (*KVStore, error) {
 	dr, _ := conf.GetDataLoc()
 	dr, _ := conf.GetDataLoc()
-	db := kv.GetDefaultKVStore(path.Join(dr, ruleId, "checkpoints"))
+	db := kv.GetDefaultKVStore(path.Join(dr, ruleId, CheckpointListKey))
 	s := &KVStore{db: db, max: 3, mapStore: &sync.Map{}}
 	s := &KVStore{db: db, max: 3, mapStore: &sync.Map{}}
 	//read data from badger db
 	//read data from badger db
 	if err := s.restore(); err != nil {
 	if err := s.restore(); err != nil {

+ 2 - 2
internal/topo/states/kv_store_test.go

@@ -1,4 +1,4 @@
-package states
+package state
 
 
 import (
 import (
 	"fmt"
 	"fmt"
@@ -251,7 +251,7 @@ func cleanStateData() {
 	if err != nil {
 	if err != nil {
 		log.Panic(err)
 		log.Panic(err)
 	}
 	}
-	c := path.Join(dbDir, "checkpoints")
+	c := path.Join(dbDir, CheckpointListKey)
 	err = os.RemoveAll(c)
 	err = os.RemoveAll(c)
 	if err != nil {
 	if err != nil {
 		conf.Log.Error(err)
 		conf.Log.Error(err)

+ 1 - 1
internal/topo/states/memory_store.go

@@ -1,4 +1,4 @@
-package states
+package state
 
 
 import (
 import (
 	"sync"
 	"sync"

+ 1 - 1
internal/topo/states/store.go

@@ -1,4 +1,4 @@
-package states
+package state
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 28 - 28
internal/topo/topo.go

@@ -4,10 +4,10 @@ import (
 	"context"
 	"context"
 	"fmt"
 	"fmt"
 	"github.com/emqx/kuiper/internal/conf"
 	"github.com/emqx/kuiper/internal/conf"
-	"github.com/emqx/kuiper/internal/topo/checkpoints"
-	"github.com/emqx/kuiper/internal/topo/contexts"
-	"github.com/emqx/kuiper/internal/topo/nodes"
-	"github.com/emqx/kuiper/internal/topo/states"
+	"github.com/emqx/kuiper/internal/topo/checkpoint"
+	kctx "github.com/emqx/kuiper/internal/topo/context"
+	"github.com/emqx/kuiper/internal/topo/node"
+	"github.com/emqx/kuiper/internal/topo/state"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"strconv"
 	"strconv"
 )
 )
@@ -18,17 +18,17 @@ type PrintableTopo struct {
 }
 }
 
 
 type Topo struct {
 type Topo struct {
-	sources            []nodes.DataSourceNode
-	sinks              []*nodes.SinkNode
+	sources            []node.DataSourceNode
+	sinks              []*node.SinkNode
 	ctx                api.StreamContext
 	ctx                api.StreamContext
 	cancel             context.CancelFunc
 	cancel             context.CancelFunc
 	drain              chan error
 	drain              chan error
-	ops                []nodes.OperatorNode
+	ops                []node.OperatorNode
 	name               string
 	name               string
 	qos                api.Qos
 	qos                api.Qos
 	checkpointInterval int
 	checkpointInterval int
 	store              api.Store
 	store              api.Store
-	coordinator        *checkpoints.Coordinator
+	coordinator        *checkpoint.Coordinator
 	topo               *PrintableTopo
 	topo               *PrintableTopo
 }
 }
 
 
@@ -58,13 +58,13 @@ func (s *Topo) Cancel() {
 	s.coordinator = nil
 	s.coordinator = nil
 }
 }
 
 
-func (s *Topo) AddSrc(src nodes.DataSourceNode) *Topo {
+func (s *Topo) AddSrc(src node.DataSourceNode) *Topo {
 	s.sources = append(s.sources, src)
 	s.sources = append(s.sources, src)
 	s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
 	s.topo.Sources = append(s.topo.Sources, fmt.Sprintf("source_%s", src.GetName()))
 	return s
 	return s
 }
 }
 
 
-func (s *Topo) AddSink(inputs []api.Emitter, snk *nodes.SinkNode) *Topo {
+func (s *Topo) AddSink(inputs []api.Emitter, snk *node.SinkNode) *Topo {
 	for _, input := range inputs {
 	for _, input := range inputs {
 		input.AddOutput(snk.GetInput())
 		input.AddOutput(snk.GetInput())
 		snk.AddInputCount()
 		snk.AddInputCount()
@@ -74,7 +74,7 @@ func (s *Topo) AddSink(inputs []api.Emitter, snk *nodes.SinkNode) *Topo {
 	return s
 	return s
 }
 }
 
 
-func (s *Topo) AddOperator(inputs []api.Emitter, operator nodes.OperatorNode) *Topo {
+func (s *Topo) AddOperator(inputs []api.Emitter, operator node.OperatorNode) *Topo {
 	for _, input := range inputs {
 	for _, input := range inputs {
 		input.AddOutput(operator.GetInput())
 		input.AddOutput(operator.GetInput())
 		operator.AddInputCount()
 		operator.AddInputCount()
@@ -86,7 +86,7 @@ func (s *Topo) AddOperator(inputs []api.Emitter, operator nodes.OperatorNode) *T
 
 
 func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
 func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
 	fromType := "op"
 	fromType := "op"
-	if _, ok := from.(nodes.DataSourceNode); ok {
+	if _, ok := from.(node.DataSourceNode); ok {
 		fromType = "source"
 		fromType = "source"
 	}
 	}
 	f := fmt.Sprintf("%s_%s", fromType, from.GetName())
 	f := fmt.Sprintf("%s_%s", fromType, from.GetName())
@@ -103,7 +103,7 @@ func (s *Topo) addEdge(from api.TopNode, to api.TopNode, toType string) {
 func (s *Topo) prepareContext() {
 func (s *Topo) prepareContext() {
 	if s.ctx == nil || s.ctx.Err() != nil {
 	if s.ctx == nil || s.ctx.Err() != nil {
 		contextLogger := conf.Log.WithField("rule", s.name)
 		contextLogger := conf.Log.WithField("rule", s.name)
-		ctx := contexts.WithValue(contexts.Background(), contexts.LoggerKey, contextLogger)
+		ctx := kctx.WithValue(kctx.Background(), kctx.LoggerKey, contextLogger)
 		s.ctx, s.cancel = ctx.WithCancel()
 		s.ctx, s.cancel = ctx.WithCancel()
 	}
 	}
 }
 }
@@ -133,7 +133,7 @@ func (s *Topo) Open() <-chan error {
 	// open stream
 	// open stream
 	go func() {
 	go func() {
 		var err error
 		var err error
-		if s.store, err = states.CreateStore(s.name, s.qos); err != nil {
+		if s.store, err = state.CreateStore(s.name, s.qos); err != nil {
 			fmt.Println(err)
 			fmt.Println(err)
 			s.drain <- err
 			s.drain <- err
 			return
 			return
@@ -165,49 +165,49 @@ func (s *Topo) Open() <-chan error {
 
 
 func (s *Topo) enableCheckpoint() error {
 func (s *Topo) enableCheckpoint() error {
 	if s.qos >= api.AtLeastOnce {
 	if s.qos >= api.AtLeastOnce {
-		var sources []checkpoints.StreamTask
+		var sources []checkpoint.StreamTask
 		for _, r := range s.sources {
 		for _, r := range s.sources {
 			sources = append(sources, r)
 			sources = append(sources, r)
 		}
 		}
-		var ops []checkpoints.NonSourceTask
+		var ops []checkpoint.NonSourceTask
 		for _, r := range s.ops {
 		for _, r := range s.ops {
 			ops = append(ops, r)
 			ops = append(ops, r)
 		}
 		}
-		var sinks []checkpoints.SinkTask
+		var sinks []checkpoint.SinkTask
 		for _, r := range s.sinks {
 		for _, r := range s.sinks {
 			sinks = append(sinks, r)
 			sinks = append(sinks, r)
 		}
 		}
-		c := checkpoints.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
+		c := checkpoint.NewCoordinator(s.name, sources, ops, sinks, s.qos, s.store, s.checkpointInterval, s.ctx)
 		s.coordinator = c
 		s.coordinator = c
 	}
 	}
 	return nil
 	return nil
 }
 }
 
 
-func (s *Topo) GetCoordinator() *checkpoints.Coordinator {
+func (s *Topo) GetCoordinator() *checkpoint.Coordinator {
 	return s.coordinator
 	return s.coordinator
 }
 }
 
 
 func (s *Topo) GetMetrics() (keys []string, values []interface{}) {
 func (s *Topo) GetMetrics() (keys []string, values []interface{}) {
-	for _, node := range s.sources {
-		for ins, metrics := range node.GetMetrics() {
+	for _, sn := range s.sources {
+		for ins, metrics := range sn.GetMetrics() {
 			for i, v := range metrics {
 			for i, v := range metrics {
-				keys = append(keys, "source_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
+				keys = append(keys, "source_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+node.MetricNames[i])
 				values = append(values, v)
 				values = append(values, v)
 			}
 			}
 		}
 		}
 	}
 	}
-	for _, node := range s.ops {
-		for ins, metrics := range node.GetMetrics() {
+	for _, so := range s.ops {
+		for ins, metrics := range so.GetMetrics() {
 			for i, v := range metrics {
 			for i, v := range metrics {
-				keys = append(keys, "op_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
+				keys = append(keys, "op_"+so.GetName()+"_"+strconv.Itoa(ins)+"_"+node.MetricNames[i])
 				values = append(values, v)
 				values = append(values, v)
 			}
 			}
 		}
 		}
 	}
 	}
-	for _, node := range s.sinks {
-		for ins, metrics := range node.GetMetrics() {
+	for _, sn := range s.sinks {
+		for ins, metrics := range sn.GetMetrics() {
 			for i, v := range metrics {
 			for i, v := range metrics {
-				keys = append(keys, "sink_"+node.GetName()+"_"+strconv.Itoa(ins)+"_"+nodes.MetricNames[i])
+				keys = append(keys, "sink_"+sn.GetName()+"_"+strconv.Itoa(ins)+"_"+node.MetricNames[i])
 				values = append(values, v)
 				values = append(values, v)
 			}
 			}
 		}
 		}

+ 9 - 9
internal/topo/topotest/mock_topo.go

@@ -7,10 +7,10 @@ import (
 	"github.com/emqx/kuiper/internal/processor"
 	"github.com/emqx/kuiper/internal/processor"
 	"github.com/emqx/kuiper/internal/testx"
 	"github.com/emqx/kuiper/internal/testx"
 	"github.com/emqx/kuiper/internal/topo"
 	"github.com/emqx/kuiper/internal/topo"
-	"github.com/emqx/kuiper/internal/topo/nodes"
+	"github.com/emqx/kuiper/internal/topo/node"
 	"github.com/emqx/kuiper/internal/topo/planner"
 	"github.com/emqx/kuiper/internal/topo/planner"
 	"github.com/emqx/kuiper/internal/topo/topotest/mockclock"
 	"github.com/emqx/kuiper/internal/topo/topotest/mockclock"
-	"github.com/emqx/kuiper/internal/topo/topotest/mocknodes"
+	"github.com/emqx/kuiper/internal/topo/topotest/mocknode"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/internal/xsql"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/ast"
 	"github.com/emqx/kuiper/pkg/ast"
@@ -140,7 +140,7 @@ func doRuleTestBySinkProps(t *testing.T, tests []RuleTest, j int, opt *api.RuleO
 	}
 	}
 }
 }
 
 
-func compareResult(t *testing.T, mockSink *mocknodes.MockSink, resultFunc func(result [][]byte) interface{}, tt RuleTest, i int, tp *topo.Topo) {
+func compareResult(t *testing.T, mockSink *mocknode.MockSink, resultFunc func(result [][]byte) interface{}, tt RuleTest, i int, tp *topo.Topo) {
 	// Check results
 	// Check results
 	results := mockSink.GetResults()
 	results := mockSink.GetResults()
 	maps := resultFunc(results)
 	maps := resultFunc(results)
@@ -209,11 +209,11 @@ func sendData(t *testing.T, dataLength int, metrics map[string]interface{}, data
 	return nil
 	return nil
 }
 }
 
 
-func createStream(t *testing.T, tt RuleTest, j int, opt *api.RuleOption, sinkProps map[string]interface{}) ([][]*xsql.Tuple, int, *topo.Topo, *mocknodes.MockSink, <-chan error) {
+func createStream(t *testing.T, tt RuleTest, j int, opt *api.RuleOption, sinkProps map[string]interface{}) ([][]*xsql.Tuple, int, *topo.Topo, *mocknode.MockSink, <-chan error) {
 	mockclock.ResetClock(1541152486000)
 	mockclock.ResetClock(1541152486000)
 	// Create stream
 	// Create stream
 	var (
 	var (
-		sources    []*nodes.SourceNode
+		sources    []*node.SourceNode
 		datas      [][]*xsql.Tuple
 		datas      [][]*xsql.Tuple
 		dataLength int
 		dataLength int
 	)
 	)
@@ -227,7 +227,7 @@ func createStream(t *testing.T, tt RuleTest, j int, opt *api.RuleOption, sinkPro
 		} else {
 		} else {
 			streams := xsql.GetStreams(selectStmt)
 			streams := xsql.GetStreams(selectStmt)
 			for _, stream := range streams {
 			for _, stream := range streams {
-				data, ok := mocknodes.TestData[stream]
+				data, ok := mocknode.TestData[stream]
 				if !ok {
 				if !ok {
 					continue
 					continue
 				}
 				}
@@ -236,9 +236,9 @@ func createStream(t *testing.T, tt RuleTest, j int, opt *api.RuleOption, sinkPro
 			}
 			}
 		}
 		}
 	}
 	}
-	mockSink := mocknodes.NewMockSink()
-	sink := nodes.NewSinkNodeWithSink("mockSink", mockSink, sinkProps)
-	tp, err := planner.PlanWithSourcesAndSinks(&api.Rule{Id: fmt.Sprintf("%s_%d", tt.Name, j), Sql: tt.Sql, Options: opt}, DbDir, sources, []*nodes.SinkNode{sink})
+	mockSink := mocknode.NewMockSink()
+	sink := node.NewSinkNodeWithSink("mockSink", mockSink, sinkProps)
+	tp, err := planner.PlanWithSourcesAndSinks(&api.Rule{Id: fmt.Sprintf("%s_%d", tt.Name, j), Sql: tt.Sql, Options: opt}, DbDir, sources, []*node.SinkNode{sink})
 	if err != nil {
 	if err != nil {
 		t.Error(err)
 		t.Error(err)
 		return nil, 0, nil, nil, nil
 		return nil, 0, nil, nil, nil

+ 1 - 1
internal/topo/topotest/mocknodes/mock_data.go

@@ -1,4 +1,4 @@
-package mocknodes
+package mocknode
 
 
 import (
 import (
 	"encoding/base64"
 	"encoding/base64"

+ 1 - 1
internal/topo/topotest/mocknodes/mock_sink.go

@@ -1,4 +1,4 @@
-package mocknodes
+package mocknode
 
 
 import (
 import (
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"

+ 1 - 1
internal/topo/topotest/mocknodes/mock_source.go

@@ -1,4 +1,4 @@
-package mocknodes
+package mocknode
 
 
 import (
 import (
 	"fmt"
 	"fmt"

+ 2 - 2
internal/topo/topotest/rule_test.go

@@ -3,7 +3,7 @@ package topotest
 import (
 import (
 	"encoding/json"
 	"encoding/json"
 	"github.com/emqx/kuiper/internal/topo"
 	"github.com/emqx/kuiper/internal/topo"
-	"github.com/emqx/kuiper/internal/topo/topotest/mocknodes"
+	"github.com/emqx/kuiper/internal/topo/topotest/mocknode"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"testing"
 	"testing"
 )
 )
@@ -844,7 +844,7 @@ func TestSingleSQLForBinary(t *testing.T) {
 			Sql:  `SELECT * FROM binDemo`,
 			Sql:  `SELECT * FROM binDemo`,
 			R: [][]map[string]interface{}{
 			R: [][]map[string]interface{}{
 				{{
 				{{
-					"self": mocknodes.Image,
+					"self": mocknode.Image,
 				}},
 				}},
 			},
 			},
 			W: 50,
 			W: 50,

+ 2 - 2
internal/xsql/functionRuntime.go

@@ -1,7 +1,7 @@
 package xsql
 package xsql
 
 
 import (
 import (
-	"github.com/emqx/kuiper/internal/topo/contexts"
+	"github.com/emqx/kuiper/internal/topo/context"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/api"
 	"github.com/emqx/kuiper/pkg/errorx"
 	"github.com/emqx/kuiper/pkg/errorx"
 	"sync"
 	"sync"
@@ -52,7 +52,7 @@ func (fp *funcRuntime) Get(name string) (api.Function, api.FunctionContext, erro
 		if nf == nil {
 		if nf == nil {
 			return nil, nil, errorx.NotFoundErr
 			return nil, nil, errorx.NotFoundErr
 		}
 		}
-		fctx := contexts.NewDefaultFuncContext(fp.parentCtx, len(fp.regs))
+		fctx := context.NewDefaultFuncContext(fp.parentCtx, len(fp.regs))
 		fp.regs[name] = &funcReg{
 		fp.regs[name] = &funcReg{
 			ins: nf,
 			ins: nf,
 			ctx: fctx,
 			ctx: fctx,