├── .github └── workflows │ ├── release.yml │ └── test.yml ├── .gitignore ├── .goreleaser.yml ├── LICENSE ├── README.md ├── aggregates ├── array.go ├── average.go ├── count.go ├── distinct.go ├── max.go ├── min.go ├── sum.go └── table.go ├── benchmarks └── benchmarks.sh ├── cmd ├── describe.go ├── plugin.go ├── plugin_install.go ├── plugin_repository.go ├── plugin_repository_add.go └── root.go ├── config └── config.go ├── datasources ├── csv │ ├── execution.go │ └── impl.go ├── docs │ ├── aggregate_signatures.go │ ├── aggregates.go │ ├── database.go │ ├── function_signatures.go │ ├── functions.go │ └── helper.go ├── json │ ├── execution.go │ ├── impl.go │ └── workers.go ├── lines │ ├── execution.go │ └── impl.go ├── memory │ └── execution.go ├── parquet │ ├── execution.go │ ├── impl.go │ └── reconstruct.go └── plugins │ ├── available_plugins.go │ ├── available_versions.go │ ├── database.go │ ├── installed_plugins.go │ ├── installed_versions.go │ └── repositories.go ├── execution ├── execution.go ├── expressions.go ├── files │ ├── files.go │ └── stdin.go ├── group_key.go ├── nodes │ ├── custom_trigger_group_by.go │ ├── distinct.go │ ├── event_time_buffer.go │ ├── filter.go │ ├── in_memory_records.go │ ├── limit.go │ ├── lookup_join.go │ ├── map.go │ ├── order_sensitive_transform.go │ ├── outer_join.go │ ├── simple_group_by.go │ ├── stream_join.go │ └── unnest.go ├── record_event_time_buffer.go ├── triggers.go └── triggers_test.go ├── functions └── functions.go ├── go.mod ├── go.sum ├── helpers └── graph │ └── graph.go ├── images ├── logo.png ├── octosql-demo-dataflow.gif ├── octosql-demo.gif ├── octosql-explain.png └── octosql.svg ├── logical ├── distinct.go ├── filter.go ├── function.go ├── group_by.go ├── join.go ├── limit.go ├── logical.go ├── map.go ├── requalifier.go ├── tvf.go ├── union_all.go ├── union_distinct.go ├── utils.go └── with.go ├── logs └── logs.go ├── main.go ├── octosql ├── types.go ├── types_test.go └── values.go ├── optimizer ├── filter_datasource_pushdown.go ├── filter_merge.go ├── optimize.go ├── push_filter_into_lookup_join_branch.go ├── push_filter_into_stream_join_branch.go ├── push_filter_into_stream_join_key.go ├── remove_unused_datasource_fields.go ├── remove_unused_groupby_fields.go └── remove_unused_map_fields.go ├── outputs ├── batch │ └── live_output.go ├── eager │ └── eager.go ├── formats │ ├── csv_format.go │ ├── human_readable_schema.go │ ├── json_format.go │ └── table_format.go └── stream │ ├── internally_consistent_output_stream_wrapper.go │ └── printer.go ├── parser ├── parser.go └── sqlparser │ ├── Makefile │ ├── analyzer.go │ ├── analyzer_test.go │ ├── ast.go │ ├── ast_test.go │ ├── comments.go │ ├── comments_test.go │ ├── dependency │ ├── bytes2 │ │ ├── buffer.go │ │ └── buffer_test.go │ ├── hack │ │ ├── hack.go │ │ └── hack_test.go │ ├── query │ │ └── query.pb.go │ └── sqltypes │ │ ├── event_token.go │ │ ├── event_token_test.go │ │ ├── proto3.go │ │ ├── type.go │ │ ├── type_test.go │ │ ├── value.go │ │ └── value_test.go │ ├── parsed_query.go │ ├── precedence_test.go │ ├── sql.go │ ├── sql.y │ ├── token.go │ ├── token_test.go │ ├── tracked_buffer.go │ └── truncate_query.go ├── physical ├── explain.go ├── expression.go ├── nodes.go ├── physical.go ├── rename_variable.go ├── transform.go └── triggers.go ├── plugin_repository.json ├── plugins ├── executor │ └── executor.go ├── internal │ └── plugins │ │ ├── input.go │ │ ├── plugins.go │ │ ├── plugins.pb.go │ │ ├── plugins.proto │ │ ├── plugins_grpc.pb.go │ │ └── plugins_test.go ├── manager │ ├── extensions.go │ └── manager.go ├── plugins.go └── repository │ └── repository.go ├── table_valued_functions ├── max_diff_watermark.go ├── poll.go ├── range.go └── tumble.go ├── telemetry ├── query.go └── telemetry.go └── tests ├── scenarios ├── datasources │ └── json │ │ ├── error1.err │ │ ├── error1.in │ │ ├── error1.out │ │ ├── error2.err │ │ ├── error2.in │ │ ├── error2.out │ │ ├── fixtures │ │ ├── join1.json │ │ ├── join1_big.json │ │ ├── join2.json │ │ ├── join2_big.json │ │ ├── objects.json │ │ ├── objects_bad_1.json │ │ └── objects_bad_2.json │ │ ├── lookup_join.err │ │ ├── lookup_join.in │ │ ├── lookup_join.out │ │ ├── simple.err │ │ ├── simple.in │ │ ├── simple.out │ │ ├── stream_join.err │ │ ├── stream_join.in │ │ └── stream_join.out ├── functions │ ├── arithmetic.err │ ├── arithmetic.in │ ├── arithmetic.out │ ├── comparators.err │ ├── comparators.in │ ├── comparators.out │ ├── conversions.err │ ├── conversions.in │ ├── conversions.out │ ├── in.err │ ├── in.in │ ├── in.out │ ├── is_null.err │ ├── is_null.in │ ├── is_null.out │ ├── logic.err │ ├── logic.in │ ├── logic.out │ ├── math.err │ ├── math.in │ ├── math.out │ ├── panic.err │ ├── panic.in │ ├── panic.out │ ├── strings.err │ ├── strings.in │ └── strings.out ├── nodes │ ├── inner_join_0.err │ ├── inner_join_0.in │ ├── inner_join_0.out │ ├── left_join_0.err │ ├── left_join_0.in │ ├── left_join_0.out │ ├── order_by_limit_0.err │ ├── order_by_limit_0.in │ ├── order_by_limit_0.out │ ├── order_by_limit_1.err │ ├── order_by_limit_1.in │ ├── order_by_limit_1.out │ ├── outer_join_0.err │ ├── outer_join_0.in │ ├── outer_join_0.out │ ├── outer_join_1.err │ ├── outer_join_1.in │ ├── outer_join_1.out │ ├── right_join_0.err │ ├── right_join_0.in │ └── right_join_0.out ├── objects │ ├── explode.err │ ├── explode.in │ ├── explode.out │ ├── fixtures │ │ └── test.json │ ├── varying_object_fields.err │ ├── varying_object_fields.in │ ├── varying_object_fields.out │ ├── varying_object_fields_describe.err │ ├── varying_object_fields_describe.in │ └── varying_object_fields_describe.out ├── outputs │ ├── csv │ │ ├── types.err │ │ ├── types.in │ │ └── types.out │ └── json │ │ ├── fixtures │ │ ├── simple.json │ │ └── union.json │ │ ├── simple.err │ │ ├── simple.in │ │ ├── simple.out │ │ ├── union.err │ │ ├── union.in │ │ └── union.out ├── simple │ ├── select_star_from_range.err │ ├── select_star_from_range.in │ └── select_star_from_range.out └── stdin │ ├── json.err │ ├── json.in │ ├── json.out │ ├── seqsum.err │ ├── seqsum.in │ └── seqsum.out └── tester └── main.go /.github/workflows/release.yml: -------------------------------------------------------------------------------- 1 | name: goreleaser 2 | 3 | on: 4 | push: 5 | tags: 6 | - '*' 7 | 8 | permissions: 9 | contents: write 10 | 11 | jobs: 12 | goreleaser: 13 | runs-on: ubuntu-latest 14 | steps: 15 | - name: Checkout 16 | uses: actions/checkout@v2 17 | with: 18 | fetch-depth: 0 19 | - name: Set up Go 20 | uses: actions/setup-go@v2 21 | with: 22 | go-version: 1.18 23 | - name: Run GoReleaser 24 | uses: goreleaser/goreleaser-action@v2 25 | with: 26 | distribution: goreleaser 27 | version: v1.8.3 28 | args: release --rm-dist 29 | env: 30 | GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} 31 | HOMEBREW_TAP_GITHUB_TOKEN: ${{ secrets.HOMEBREW_TAP_GITHUB_TOKEN }} 32 | -------------------------------------------------------------------------------- /.github/workflows/test.yml: -------------------------------------------------------------------------------- 1 | name: Tests 2 | 3 | on: 4 | push: 5 | 6 | jobs: 7 | unit-tests: 8 | runs-on: ubuntu-latest 9 | steps: 10 | - uses: actions/checkout@v2 11 | 12 | - name: Set up Go 13 | uses: actions/setup-go@v2 14 | with: 15 | go-version: 1.18 16 | 17 | - name: Build 18 | run: go build -v ./... 19 | 20 | - name: Test 21 | run: go test -v ./... 22 | snapshot-tests: 23 | runs-on: ubuntu-latest 24 | steps: 25 | - uses: actions/checkout@v2 26 | 27 | - name: Set up Go 28 | uses: actions/setup-go@v2 29 | with: 30 | go-version: 1.18 31 | 32 | - name: Install 33 | run: go install -v 34 | 35 | - name: Test 36 | run: go run tests/tester/main.go ci 37 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .idea 2 | -------------------------------------------------------------------------------- /.goreleaser.yml: -------------------------------------------------------------------------------- 1 | project_name: octosql 2 | release: 3 | draft: true 4 | builds: 5 | - env: [CGO_ENABLED=0] 6 | goos: 7 | - linux 8 | - windows 9 | - darwin 10 | goarch: 11 | - amd64 12 | - arm64 13 | tags: 14 | - purego 15 | ldflags: 16 | - "-s -w -X github.com/cube2222/octosql/cmd.VERSION={{.Version}}" 17 | brews: 18 | - name: octosql 19 | tap: 20 | owner: "cube2222" 21 | name: "homebrew-octosql" 22 | branch: main 23 | token: "{{ .Env.HOMEBREW_TAP_GITHUB_TOKEN }}" 24 | homepage: https://github.com/cube2222/octosql 25 | description: "OctoSQL is a query tool that allows you to join, analyse and transform data from multiple databases and file formats using SQL." 26 | license: "MPL-2.0" 27 | dependencies: 28 | - name: graphviz 29 | -------------------------------------------------------------------------------- /aggregates/array.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/google/btree" 7 | 8 | "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/execution/nodes" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | var ArrayOverloads = []physical.AggregateDescriptor{ 15 | { 16 | TypeFn: func(t octosql.Type) (octosql.Type, bool) { 17 | return octosql.Type{TypeID: octosql.TypeIDList, List: struct{ Element *octosql.Type }{Element: &t}}, true 18 | }, 19 | Prototype: NewArrayPrototype(), 20 | }, 21 | } 22 | 23 | // TODO: Elements should be sorted as they come, not sorted by value in a BTree. 24 | type Array struct { 25 | items *btree.BTree 26 | } 27 | 28 | func NewArrayPrototype() func() nodes.Aggregate { 29 | return func() nodes.Aggregate { 30 | return &Array{ 31 | items: btree.New(execution.BTreeDefaultDegree), 32 | } 33 | } 34 | } 35 | 36 | type arrayKey struct { 37 | value octosql.Value 38 | count int 39 | } 40 | 41 | func (key *arrayKey) Less(than btree.Item) bool { 42 | thanTyped, ok := than.(*arrayKey) 43 | if !ok { 44 | panic(fmt.Sprintf("invalid key comparison: %T", than)) 45 | } 46 | 47 | return key.value.Compare(thanTyped.value) == -1 48 | } 49 | 50 | func (c *Array) Add(retraction bool, value octosql.Value) bool { 51 | item := c.items.Get(&arrayKey{value: value}) 52 | var itemTyped *arrayKey 53 | 54 | if item == nil { 55 | itemTyped = &arrayKey{value: value, count: 0} 56 | c.items.ReplaceOrInsert(itemTyped) 57 | } else { 58 | var ok bool 59 | itemTyped, ok = item.(*arrayKey) 60 | if !ok { 61 | panic(fmt.Sprintf("invalid received item: %v", item)) 62 | } 63 | } 64 | if !retraction { 65 | itemTyped.count++ 66 | } else { 67 | itemTyped.count-- 68 | } 69 | if itemTyped.count == 0 { 70 | c.items.Delete(itemTyped) 71 | } 72 | return c.items.Len() == 0 73 | } 74 | 75 | func (c *Array) Trigger() octosql.Value { 76 | out := make([]octosql.Value, 0, c.items.Len()) 77 | c.items.Ascend(func(item btree.Item) bool { 78 | itemTyped, ok := item.(*arrayKey) 79 | if !ok { 80 | panic(fmt.Sprintf("invalid received item: %v", item)) 81 | } 82 | for i := 0; i < itemTyped.count; i++ { 83 | out = append(out, itemTyped.value) 84 | } 85 | return true 86 | }) 87 | 88 | return octosql.NewList(out) 89 | } 90 | -------------------------------------------------------------------------------- /aggregates/average.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "time" 5 | 6 | "github.com/cube2222/octosql/execution/nodes" 7 | "github.com/cube2222/octosql/octosql" 8 | "github.com/cube2222/octosql/physical" 9 | ) 10 | 11 | var AverageOverloads = []physical.AggregateDescriptor{ 12 | { 13 | ArgumentType: octosql.Int, 14 | OutputType: octosql.Int, 15 | Prototype: NewAverageIntPrototype(), 16 | }, 17 | { 18 | ArgumentType: octosql.Float, 19 | OutputType: octosql.Float, 20 | Prototype: NewAverageFloatPrototype(), 21 | }, 22 | { 23 | ArgumentType: octosql.Duration, 24 | OutputType: octosql.Duration, 25 | Prototype: NewAverageDurationPrototype(), 26 | }, 27 | } 28 | 29 | type AverageInt struct { 30 | sum SumInt 31 | count Count 32 | } 33 | 34 | func NewAverageIntPrototype() func() nodes.Aggregate { 35 | return func() nodes.Aggregate { 36 | return &AverageInt{ 37 | sum: SumInt{}, 38 | count: Count{}, 39 | } 40 | } 41 | } 42 | 43 | func (c *AverageInt) Add(retraction bool, value octosql.Value) bool { 44 | c.sum.Add(retraction, value) 45 | return c.count.Add(retraction, value) 46 | } 47 | 48 | func (c *AverageInt) Trigger() octosql.Value { 49 | return octosql.NewInt(c.sum.Trigger().Int / c.count.Trigger().Int) 50 | } 51 | 52 | type AverageFloat struct { 53 | sum SumFloat 54 | count Count 55 | } 56 | 57 | func NewAverageFloatPrototype() func() nodes.Aggregate { 58 | return func() nodes.Aggregate { 59 | return &AverageFloat{ 60 | sum: SumFloat{}, 61 | count: Count{}, 62 | } 63 | } 64 | } 65 | 66 | func (c *AverageFloat) Add(retraction bool, value octosql.Value) bool { 67 | c.sum.Add(retraction, value) 68 | return c.count.Add(retraction, value) 69 | } 70 | 71 | func (c *AverageFloat) Trigger() octosql.Value { 72 | return octosql.NewFloat(c.sum.Trigger().Float / float64(c.count.Trigger().Int)) 73 | } 74 | 75 | type AverageDuration struct { 76 | sum SumDuration 77 | count Count 78 | } 79 | 80 | func NewAverageDurationPrototype() func() nodes.Aggregate { 81 | return func() nodes.Aggregate { 82 | return &AverageDuration{ 83 | sum: SumDuration{}, 84 | count: Count{}, 85 | } 86 | } 87 | } 88 | 89 | func (c *AverageDuration) Add(retraction bool, value octosql.Value) bool { 90 | c.sum.Add(retraction, value) 91 | return c.count.Add(retraction, value) 92 | } 93 | 94 | func (c *AverageDuration) Trigger() octosql.Value { 95 | return octosql.NewDuration(c.sum.Trigger().Duration / time.Duration(c.count.Trigger().Int)) 96 | } 97 | -------------------------------------------------------------------------------- /aggregates/count.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "github.com/cube2222/octosql/execution/nodes" 5 | "github.com/cube2222/octosql/octosql" 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | var CountOverloads = []physical.AggregateDescriptor{ 10 | { 11 | ArgumentType: octosql.Any, 12 | OutputType: octosql.Int, 13 | Prototype: NewCountPrototype(), 14 | }, 15 | } 16 | 17 | type Count struct { 18 | count int64 19 | } 20 | 21 | func NewCountPrototype() func() nodes.Aggregate { 22 | return func() nodes.Aggregate { 23 | return &Count{ 24 | count: 0, 25 | } 26 | } 27 | } 28 | 29 | func (c *Count) Add(retraction bool, value octosql.Value) bool { 30 | if !retraction { 31 | c.count++ 32 | } else { 33 | c.count-- 34 | } 35 | return c.count == 0 36 | } 37 | 38 | func (c *Count) Trigger() octosql.Value { 39 | return octosql.NewInt(c.count) 40 | } 41 | -------------------------------------------------------------------------------- /aggregates/distinct.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "github.com/zyedidia/generic/hashmap" 5 | 6 | "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/execution/nodes" 8 | "github.com/cube2222/octosql/octosql" 9 | "github.com/cube2222/octosql/physical" 10 | ) 11 | 12 | func DistinctAggregateOverloads(overloads []physical.AggregateDescriptor) []physical.AggregateDescriptor { 13 | out := make([]physical.AggregateDescriptor, len(overloads)) 14 | for i := range overloads { 15 | out[i] = physical.AggregateDescriptor{ 16 | ArgumentType: overloads[i].ArgumentType, 17 | OutputType: overloads[i].OutputType, 18 | TypeFn: overloads[i].TypeFn, 19 | Prototype: NewDistinctPrototype(overloads[i].Prototype), 20 | } 21 | } 22 | return out 23 | } 24 | 25 | type Distinct struct { 26 | items *hashmap.Map[octosql.Value, *distinctKey] 27 | wrapped nodes.Aggregate 28 | } 29 | 30 | func NewDistinctPrototype(wrapped func() nodes.Aggregate) func() nodes.Aggregate { 31 | return func() nodes.Aggregate { 32 | return &Distinct{ 33 | items: hashmap.New[octosql.Value, *distinctKey]( 34 | execution.BTreeDefaultDegree, 35 | func(a, b octosql.Value) bool { 36 | return a.Compare(b) == 0 37 | }, func(v octosql.Value) uint64 { 38 | return v.Hash() 39 | }), 40 | wrapped: wrapped(), 41 | } 42 | } 43 | } 44 | 45 | type distinctKey struct { 46 | count int 47 | } 48 | 49 | func (c *Distinct) Add(retraction bool, value octosql.Value) bool { 50 | item, ok := c.items.Get(value) 51 | if !ok { 52 | item = &distinctKey{count: 0} 53 | c.items.Put(value, item) 54 | } 55 | if !retraction { 56 | item.count++ 57 | } else { 58 | item.count-- 59 | } 60 | if item.count == 1 && !retraction { 61 | c.wrapped.Add(false, value) 62 | } else if item.count == 0 { 63 | c.items.Remove(value) 64 | c.wrapped.Add(true, value) 65 | } 66 | return c.items.Size() == 0 67 | } 68 | 69 | func (c *Distinct) Trigger() octosql.Value { 70 | return c.wrapped.Trigger() 71 | } 72 | -------------------------------------------------------------------------------- /aggregates/max.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/google/btree" 7 | 8 | "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/execution/nodes" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | var MaxOverloads = []physical.AggregateDescriptor{ 15 | { 16 | ArgumentType: octosql.Int, 17 | OutputType: octosql.Int, 18 | Prototype: NewMaxPrototype(), 19 | }, 20 | { 21 | ArgumentType: octosql.Float, 22 | OutputType: octosql.Float, 23 | Prototype: NewMaxPrototype(), 24 | }, 25 | { 26 | ArgumentType: octosql.Duration, 27 | OutputType: octosql.Duration, 28 | Prototype: NewMaxPrototype(), 29 | }, 30 | { 31 | ArgumentType: octosql.Time, 32 | OutputType: octosql.Time, 33 | Prototype: NewMaxPrototype(), 34 | }, 35 | } 36 | 37 | type Max struct { 38 | items *btree.BTree 39 | } 40 | 41 | func NewMaxPrototype() func() nodes.Aggregate { 42 | return func() nodes.Aggregate { 43 | return &Max{ 44 | items: btree.New(execution.BTreeDefaultDegree), 45 | } 46 | } 47 | } 48 | 49 | type maxKey struct { 50 | value octosql.Value 51 | count int 52 | } 53 | 54 | func (key *maxKey) Less(than btree.Item) bool { 55 | thanTyped, ok := than.(*maxKey) 56 | if !ok { 57 | panic(fmt.Sprintf("invalid key comparison: %T", than)) 58 | } 59 | 60 | return key.value.Compare(thanTyped.value) == -1 61 | } 62 | 63 | func (c *Max) Add(retraction bool, value octosql.Value) bool { 64 | item := c.items.Get(&maxKey{value: value}) 65 | var itemTyped *maxKey 66 | 67 | if item == nil { 68 | itemTyped = &maxKey{value: value, count: 0} 69 | c.items.ReplaceOrInsert(itemTyped) 70 | } else { 71 | var ok bool 72 | itemTyped, ok = item.(*maxKey) 73 | if !ok { 74 | panic(fmt.Sprintf("invalid received item: %v", item)) 75 | } 76 | } 77 | if !retraction { 78 | itemTyped.count++ 79 | } else { 80 | itemTyped.count-- 81 | } 82 | if itemTyped.count == 0 { 83 | c.items.Delete(itemTyped) 84 | } 85 | return c.items.Len() == 0 86 | } 87 | 88 | func (c *Max) Trigger() octosql.Value { 89 | return c.items.Max().(*maxKey).value 90 | } 91 | -------------------------------------------------------------------------------- /aggregates/min.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/google/btree" 7 | 8 | "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/execution/nodes" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | var MinOverloads = []physical.AggregateDescriptor{ 15 | { 16 | ArgumentType: octosql.Int, 17 | OutputType: octosql.Int, 18 | Prototype: NewMinPrototype(), 19 | }, 20 | { 21 | ArgumentType: octosql.Float, 22 | OutputType: octosql.Float, 23 | Prototype: NewMinPrototype(), 24 | }, 25 | { 26 | ArgumentType: octosql.Duration, 27 | OutputType: octosql.Duration, 28 | Prototype: NewMinPrototype(), 29 | }, 30 | } 31 | 32 | type Min struct { 33 | items *btree.BTree 34 | } 35 | 36 | func NewMinPrototype() func() nodes.Aggregate { 37 | return func() nodes.Aggregate { 38 | return &Min{ 39 | items: btree.New(execution.BTreeDefaultDegree), 40 | } 41 | } 42 | } 43 | 44 | type minKey struct { 45 | value octosql.Value 46 | count int 47 | } 48 | 49 | func (key *minKey) Less(than btree.Item) bool { 50 | thanTyped, ok := than.(*minKey) 51 | if !ok { 52 | panic(fmt.Sprintf("invalid key comparison: %T", than)) 53 | } 54 | 55 | return key.value.Compare(thanTyped.value) == -1 56 | } 57 | 58 | func (c *Min) Add(retraction bool, value octosql.Value) bool { 59 | item := c.items.Get(&minKey{value: value}) 60 | var itemTyped *minKey 61 | 62 | if item == nil { 63 | itemTyped = &minKey{value: value, count: 0} 64 | c.items.ReplaceOrInsert(itemTyped) 65 | } else { 66 | var ok bool 67 | itemTyped, ok = item.(*minKey) 68 | if !ok { 69 | panic(fmt.Sprintf("invalid received item: %v", item)) 70 | } 71 | } 72 | if !retraction { 73 | itemTyped.count++ 74 | } else { 75 | itemTyped.count-- 76 | } 77 | if itemTyped.count == 0 { 78 | c.items.Delete(itemTyped) 79 | } 80 | return c.items.Len() == 0 81 | } 82 | 83 | func (c *Min) Trigger() octosql.Value { 84 | return c.items.Min().(*minKey).value 85 | } 86 | -------------------------------------------------------------------------------- /aggregates/sum.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "time" 5 | 6 | "github.com/cube2222/octosql/execution/nodes" 7 | "github.com/cube2222/octosql/octosql" 8 | "github.com/cube2222/octosql/physical" 9 | ) 10 | 11 | var SumOverloads = []physical.AggregateDescriptor{ 12 | { 13 | ArgumentType: octosql.Int, 14 | OutputType: octosql.Int, 15 | Prototype: NewSumIntPrototype(), 16 | }, 17 | { 18 | ArgumentType: octosql.Float, 19 | OutputType: octosql.Float, 20 | Prototype: NewSumFloatPrototype(), 21 | }, 22 | { 23 | ArgumentType: octosql.Duration, 24 | OutputType: octosql.Duration, 25 | Prototype: NewSumDurationPrototype(), 26 | }, 27 | } 28 | 29 | type SumInt struct { 30 | sum int64 31 | } 32 | 33 | func NewSumIntPrototype() func() nodes.Aggregate { 34 | return func() nodes.Aggregate { 35 | return &SumInt{ 36 | sum: 0, 37 | } 38 | } 39 | } 40 | 41 | func (c *SumInt) Add(retraction bool, value octosql.Value) bool { 42 | if !retraction { 43 | c.sum += value.Int 44 | } else { 45 | c.sum -= value.Int 46 | } 47 | return c.sum == 0 48 | } 49 | 50 | func (c *SumInt) Trigger() octosql.Value { 51 | return octosql.NewInt(c.sum) 52 | } 53 | 54 | type SumFloat struct { 55 | sum float64 56 | } 57 | 58 | func NewSumFloatPrototype() func() nodes.Aggregate { 59 | return func() nodes.Aggregate { 60 | return &SumFloat{ 61 | sum: 0, 62 | } 63 | } 64 | } 65 | 66 | func (c *SumFloat) Add(retraction bool, value octosql.Value) bool { 67 | if !retraction { 68 | c.sum += value.Float 69 | } else { 70 | c.sum -= value.Float 71 | } 72 | return c.sum == 0 73 | } 74 | 75 | func (c *SumFloat) Trigger() octosql.Value { 76 | return octosql.NewFloat(c.sum) 77 | } 78 | 79 | type SumDuration struct { 80 | sum time.Duration 81 | } 82 | 83 | func NewSumDurationPrototype() func() nodes.Aggregate { 84 | return func() nodes.Aggregate { 85 | return &SumDuration{ 86 | sum: 0, 87 | } 88 | } 89 | } 90 | 91 | func (c *SumDuration) Add(retraction bool, value octosql.Value) bool { 92 | if !retraction { 93 | c.sum += value.Duration 94 | } else { 95 | c.sum -= value.Duration 96 | } 97 | return c.sum == 0 98 | } 99 | 100 | func (c *SumDuration) Trigger() octosql.Value { 101 | return octosql.NewDuration(c.sum) 102 | } 103 | -------------------------------------------------------------------------------- /aggregates/table.go: -------------------------------------------------------------------------------- 1 | package aggregates 2 | 3 | import ( 4 | "github.com/cube2222/octosql/physical" 5 | ) 6 | 7 | var Aggregates = map[string]physical.AggregateDetails{ 8 | "array_agg": { 9 | Description: "Creates an array of all items in the group.", 10 | Descriptors: ArrayOverloads, 11 | }, 12 | "array_agg_distinct": { 13 | Description: "Creates an array of distinct items in the group.", 14 | Descriptors: DistinctAggregateOverloads(ArrayOverloads), 15 | }, 16 | "count": { 17 | Description: "Counts all items in the group.", 18 | Descriptors: CountOverloads, 19 | }, 20 | "count_distinct": { 21 | Description: "Counts distinct items in the group.", 22 | Descriptors: DistinctAggregateOverloads(CountOverloads), 23 | }, 24 | "sum": { 25 | Description: "Sums all items in the group.", 26 | Descriptors: SumOverloads, 27 | }, 28 | "sum_distinct": { 29 | Description: "Sums distinct items in the group.", 30 | Descriptors: DistinctAggregateOverloads(SumOverloads), 31 | }, 32 | "avg": { 33 | Description: "Averages all items in the group.", 34 | Descriptors: AverageOverloads, 35 | }, 36 | "avg_distinct": { 37 | Description: "Averages distinct items in the group.", 38 | Descriptors: DistinctAggregateOverloads(AverageOverloads), 39 | }, 40 | "max": { 41 | Description: "Returns maximum item in the group.", 42 | Descriptors: MaxOverloads, 43 | }, 44 | "min": { 45 | Description: "Returns minimum item in the group.", 46 | Descriptors: MinOverloads, 47 | }, 48 | } 49 | -------------------------------------------------------------------------------- /benchmarks/benchmarks.sh: -------------------------------------------------------------------------------- 1 | curl https://s3.amazonaws.com/nyc-tlc/csv_backup/yellow_tripdata_2021-04.csv -o taxi.csv 2 | 3 | echo "CREATE EXTERNAL TABLE taxi 4 | STORED AS CSV 5 | WITH HEADER ROW 6 | LOCATION './taxi.csv'; 7 | 8 | SELECT passenger_count, COUNT(*), AVG(total_amount) FROM taxi GROUP BY passenger_count;" > datafusion_commands.txt 9 | 10 | wc -l taxi.csv && wc -l taxi.csv && wc -l taxi.csv && wc -l taxi.csv # Get the cache warm. 11 | 12 | hyperfine --min-runs 10 -w 2 --export-markdown benchmarks.md \ 13 | 'OCTOSQL_NO_TELEMETRY=1 octosql "SELECT passenger_count, COUNT(*), AVG(total_amount) FROM taxi.csv GROUP BY passenger_count"' \ 14 | "q -d ',' -H \"SELECT passenger_count, COUNT(*), AVG(total_amount) FROM taxi.csv GROUP BY passenger_count\"" \ 15 | "q -d ',' -H -C readwrite \"SELECT passenger_count, COUNT(*), AVG(total_amount) FROM taxi.csv GROUP BY passenger_count\"" \ 16 | 'textql -header -sql "SELECT passenger_count, COUNT(*), AVG(total_amount) FROM taxi GROUP BY passenger_count" taxi.csv' \ 17 | 'datafusion-cli -f datafusion_commands.txt' \ 18 | 'dsq taxi.csv "SELECT passenger_count, COUNT(*), AVG(total_amount) FROM {} GROUP BY passenger_count"' \ 19 | 'dsq --cache taxi.csv "SELECT passenger_count, COUNT(*), AVG(total_amount) FROM {} GROUP BY passenger_count"' \ 20 | 'spyql "SELECT passenger_count, count_agg(*), avg_agg(total_amount) FROM csv GROUP BY passenger_count" < taxi.csv' 21 | 22 | -------------------------------------------------------------------------------- /cmd/describe.go: -------------------------------------------------------------------------------- 1 | package cmd 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | . "github.com/cube2222/octosql/execution" 8 | "github.com/cube2222/octosql/octosql" 9 | "github.com/cube2222/octosql/physical" 10 | ) 11 | 12 | var DescribeNodeSchema = physical.NewSchema( 13 | []physical.SchemaField{ 14 | { 15 | Name: "name", 16 | Type: octosql.String, 17 | }, 18 | { 19 | Name: "type", 20 | Type: octosql.String, 21 | }, 22 | { 23 | Name: "time_field", 24 | Type: octosql.Boolean, 25 | }, 26 | }, 27 | -1, 28 | ) 29 | 30 | type DescribeNode struct { 31 | Schema physical.Schema 32 | } 33 | 34 | func (d *DescribeNode) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 35 | for i, field := range d.Schema.Fields { 36 | if err := produce( 37 | ProduceFromExecutionContext(ctx), 38 | NewRecord([]octosql.Value{ 39 | octosql.NewString(field.Name), 40 | octosql.NewString(field.Type.String()), 41 | octosql.NewBoolean(i == d.Schema.TimeField), 42 | }, false, time.Time{}), 43 | ); err != nil { 44 | return fmt.Errorf("couldn't produce record: %w", err) 45 | } 46 | } 47 | return nil 48 | } 49 | -------------------------------------------------------------------------------- /cmd/plugin.go: -------------------------------------------------------------------------------- 1 | package cmd 2 | 3 | import ( 4 | "github.com/spf13/cobra" 5 | ) 6 | 7 | // pluginCmd represents the plugin command 8 | var pluginCmd = &cobra.Command{ 9 | Use: "plugin", 10 | Short: "", 11 | Long: ``, 12 | } 13 | 14 | func init() { 15 | rootCmd.AddCommand(pluginCmd) 16 | } 17 | -------------------------------------------------------------------------------- /cmd/plugin_install.go: -------------------------------------------------------------------------------- 1 | package cmd 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/Masterminds/semver" 7 | "github.com/spf13/cobra" 8 | 9 | "github.com/cube2222/octosql/config" 10 | "github.com/cube2222/octosql/plugins/manager" 11 | "github.com/cube2222/octosql/plugins/repository" 12 | ) 13 | 14 | // pluginInstallCmd represents the plugin install command 15 | var pluginInstallCmd = &cobra.Command{ 16 | Use: "install", 17 | Short: "", 18 | Long: ``, 19 | RunE: func(cmd *cobra.Command, args []string) error { 20 | ctx := cmd.Context() 21 | repositories, err := repository.GetRepositories(ctx) 22 | if err != nil { 23 | return fmt.Errorf("couldn't get repositories: %w", err) 24 | } 25 | pluginManager := &manager.PluginManager{ 26 | Repositories: repositories, 27 | } 28 | 29 | if len(args) > 0 { 30 | for _, arg := range args { 31 | if err := pluginManager.Install(ctx, arg, nil); err != nil { 32 | return err 33 | } 34 | } 35 | return nil 36 | } 37 | 38 | cfg, err := config.Read() 39 | if err != nil { 40 | return fmt.Errorf("couldn't read config: %w", err) 41 | } 42 | 43 | dbLoop: 44 | for i := range cfg.Databases { 45 | installedPlugins, err := pluginManager.ListInstalledPlugins() 46 | if err != nil { 47 | return fmt.Errorf("couldn't list installed plugins: %w", err) 48 | } 49 | 50 | if cfg.Databases[i].Version == nil { 51 | constraint, _ := semver.NewConstraint("*") 52 | cfg.Databases[i].Version = config.NewYamlUnmarshallableVersionConstraint(constraint) 53 | } 54 | for _, plugin := range installedPlugins { 55 | if plugin.Reference != cfg.Databases[i].Type { 56 | continue 57 | } 58 | for _, version := range plugin.Versions { 59 | if cfg.Databases[i].Version.Raw().Check(version.Number) { 60 | continue dbLoop 61 | } 62 | } 63 | } 64 | if err := pluginManager.Install(ctx, cfg.Databases[i].Type.String(), cfg.Databases[i].Version.Raw()); err != nil { 65 | return err 66 | } 67 | } 68 | 69 | return nil 70 | }, 71 | } 72 | 73 | func init() { 74 | pluginCmd.AddCommand(pluginInstallCmd) 75 | } 76 | -------------------------------------------------------------------------------- /cmd/plugin_repository.go: -------------------------------------------------------------------------------- 1 | package cmd 2 | 3 | import ( 4 | "github.com/spf13/cobra" 5 | ) 6 | 7 | // pluginRepositoryCmd represents the plugin command 8 | var pluginRepositoryCmd = &cobra.Command{ 9 | Use: "repository", 10 | Short: "", 11 | Long: ``, 12 | } 13 | 14 | func init() { 15 | pluginCmd.AddCommand(pluginRepositoryCmd) 16 | } 17 | -------------------------------------------------------------------------------- /cmd/plugin_repository_add.go: -------------------------------------------------------------------------------- 1 | package cmd 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/spf13/cobra" 7 | 8 | "github.com/cube2222/octosql/plugins/repository" 9 | ) 10 | 11 | // pluginRepositoryAddCmd represents the plugin install command 12 | var pluginRepositoryAddCmd = &cobra.Command{ 13 | Use: "add", 14 | Short: "", 15 | Long: ``, 16 | RunE: func(cmd *cobra.Command, args []string) error { 17 | ctx := cmd.Context() 18 | for _, arg := range args { 19 | if err := repository.AddRepository(ctx, arg); err != nil { 20 | return fmt.Errorf("couldn't add repository '%s': %s", arg, err) 21 | } 22 | } 23 | 24 | return nil 25 | }, 26 | } 27 | 28 | func init() { 29 | pluginRepositoryCmd.AddCommand(pluginRepositoryAddCmd) 30 | } 31 | -------------------------------------------------------------------------------- /datasources/csv/execution.go: -------------------------------------------------------------------------------- 1 | package csv 2 | 3 | import ( 4 | "encoding/csv" 5 | "fmt" 6 | "io" 7 | "strconv" 8 | "time" 9 | 10 | "github.com/valyala/fastjson/fastfloat" 11 | 12 | . "github.com/cube2222/octosql/execution" 13 | "github.com/cube2222/octosql/execution/files" 14 | "github.com/cube2222/octosql/octosql" 15 | "github.com/cube2222/octosql/physical" 16 | ) 17 | 18 | type DatasourceExecuting struct { 19 | path string 20 | fields []physical.SchemaField 21 | fileFieldNames []string 22 | header bool 23 | separator rune 24 | } 25 | 26 | func (d *DatasourceExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 27 | f, err := files.OpenLocalFile(ctx, d.path) 28 | if err != nil { 29 | return fmt.Errorf("couldn't open local file: %w", err) 30 | } 31 | defer f.Close() 32 | 33 | usedColumns := map[string]bool{} 34 | for i := range d.fields { 35 | usedColumns[d.fields[i].Name] = true 36 | } 37 | 38 | decoder := csv.NewReader(f) 39 | decoder.Comma = d.separator 40 | decoder.ReuseRecord = true 41 | if d.header { 42 | _, err := decoder.Read() 43 | if err != nil { 44 | return fmt.Errorf("couldn't decode csv header row: %w", err) 45 | } 46 | } 47 | 48 | indicesToRead := make([]int, 0) 49 | for i := range d.fileFieldNames { 50 | if usedColumns[d.fileFieldNames[i]] { 51 | indicesToRead = append(indicesToRead, i) 52 | } 53 | } 54 | 55 | for { 56 | row, err := decoder.Read() 57 | if err == io.EOF { 58 | break 59 | } else if err != nil { 60 | return fmt.Errorf("couldn't decode message: %w", err) 61 | } 62 | 63 | values := make([]octosql.Value, len(indicesToRead)) 64 | for i, columnIndex := range indicesToRead { 65 | str := row[columnIndex] 66 | if str == "" { 67 | values[i] = octosql.NewNull() 68 | continue 69 | } 70 | 71 | if octosql.Int.Is(d.fields[i].Type) == octosql.TypeRelationIs { 72 | integer, err := fastfloat.ParseInt64(str) 73 | if err == nil { 74 | values[i] = octosql.NewInt(integer) 75 | continue 76 | } 77 | } 78 | 79 | if octosql.Float.Is(d.fields[i].Type) == octosql.TypeRelationIs { 80 | float, err := fastfloat.Parse(str) 81 | if err == nil { 82 | values[i] = octosql.NewFloat(float) 83 | continue 84 | } 85 | } 86 | 87 | if octosql.Boolean.Is(d.fields[i].Type) == octosql.TypeRelationIs { 88 | b, err := strconv.ParseBool(str) 89 | if err == nil { 90 | values[i] = octosql.NewBoolean(b) 91 | continue 92 | } 93 | } 94 | 95 | if octosql.Time.Is(d.fields[i].Type) == octosql.TypeRelationIs { 96 | t, err := time.Parse(time.RFC3339Nano, str) 97 | if err == nil { 98 | values[i] = octosql.NewTime(t) 99 | continue 100 | } 101 | } 102 | 103 | values[i] = octosql.NewString(str) 104 | } 105 | 106 | if err := produce(ProduceFromExecutionContext(ctx), NewRecord(values, false, time.Time{})); err != nil { 107 | return fmt.Errorf("couldn't produce record: %w", err) 108 | } 109 | } 110 | 111 | return nil 112 | } 113 | -------------------------------------------------------------------------------- /datasources/docs/aggregate_signatures.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | "github.com/cube2222/octosql/aggregates" 9 | . "github.com/cube2222/octosql/execution" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | type aggregateSignaturesPhysical struct { 15 | } 16 | 17 | func (i *aggregateSignaturesPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 18 | return &aggregateSignaturesExecuting{ 19 | fields: schema.Fields, 20 | }, nil 21 | } 22 | 23 | func (i *aggregateSignaturesPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 24 | return newPredicates, []physical.Expression{}, false 25 | } 26 | 27 | type aggregateSignaturesExecuting struct { 28 | fields []physical.SchemaField 29 | } 30 | 31 | func (d *aggregateSignaturesExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 32 | fs := sortedMapNameAndDetails(aggregates.Aggregates) 33 | 34 | output := make([][]octosql.Value, 0) 35 | for _, f := range fs { 36 | if f.Details.Description == "" { 37 | continue 38 | } 39 | for _, descriptor := range f.Details.Descriptors { 40 | row := make([]octosql.Value, len(d.fields)) 41 | for i, field := range d.fields { 42 | switch field.Name { 43 | case "name": 44 | row[i] = octosql.NewString(f.Name) 45 | case "argument_type": 46 | row[i] = octosql.NewString(descriptor.ArgumentType.String()) 47 | case "output_type": 48 | row[i] = octosql.NewString(descriptor.OutputType.String()) 49 | case "simple_signature": 50 | row[i] = octosql.NewBoolean(descriptor.TypeFn == nil) 51 | } 52 | } 53 | output = append(output, row) 54 | } 55 | } 56 | 57 | for i := range output { 58 | if err := produce( 59 | ProduceFromExecutionContext(ctx), 60 | NewRecord(output[i], false, time.Time{}), 61 | ); err != nil { 62 | return fmt.Errorf("couldn't produce record: %w", err) 63 | } 64 | } 65 | 66 | return nil 67 | } 68 | -------------------------------------------------------------------------------- /datasources/docs/aggregates.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | "github.com/cube2222/octosql/aggregates" 9 | . "github.com/cube2222/octosql/execution" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | type aggregatesPhysical struct { 15 | } 16 | 17 | func (i *aggregatesPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 18 | return &aggregatesExecuting{ 19 | fields: schema.Fields, 20 | }, nil 21 | } 22 | 23 | func (i *aggregatesPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 24 | return newPredicates, []physical.Expression{}, false 25 | } 26 | 27 | type aggregatesExecuting struct { 28 | fields []physical.SchemaField 29 | } 30 | 31 | func (d *aggregatesExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 32 | fs := sortedMapNameAndDetails(aggregates.Aggregates) 33 | 34 | output := make([][]octosql.Value, 0) 35 | for _, f := range fs { 36 | if f.Details.Description == "" { 37 | continue 38 | } 39 | row := make([]octosql.Value, len(d.fields)) 40 | for i, field := range d.fields { 41 | switch field.Name { 42 | case "name": 43 | row[i] = octosql.NewString(f.Name) 44 | case "description": 45 | row[i] = octosql.NewString(f.Details.Description) 46 | } 47 | } 48 | output = append(output, row) 49 | } 50 | 51 | for i := range output { 52 | if err := produce( 53 | ProduceFromExecutionContext(ctx), 54 | NewRecord(output[i], false, time.Time{}), 55 | ); err != nil { 56 | return fmt.Errorf("couldn't produce record: %w", err) 57 | } 58 | } 59 | 60 | return nil 61 | } 62 | -------------------------------------------------------------------------------- /datasources/docs/database.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | 7 | _ "github.com/jackc/pgx/stdlib" 8 | 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | ) 12 | 13 | func Creator(ctx context.Context) (physical.Database, error) { 14 | return &Database{}, nil 15 | } 16 | 17 | type Database struct { 18 | } 19 | 20 | func (d *Database) ListTables(ctx context.Context) ([]string, error) { 21 | return []string{ 22 | "functions", 23 | }, nil 24 | } 25 | 26 | func (d *Database) GetTable(ctx context.Context, name string, options map[string]string) (physical.DatasourceImplementation, physical.Schema, error) { 27 | switch name { 28 | case "aggregates": 29 | return &aggregatesPhysical{}, 30 | physical.Schema{ 31 | TimeField: -1, 32 | Fields: []physical.SchemaField{ 33 | { 34 | Name: "name", 35 | Type: octosql.String, 36 | }, 37 | { 38 | Name: "description", 39 | Type: octosql.String, 40 | }, 41 | }, 42 | NoRetractions: true, 43 | }, 44 | nil 45 | case "aggregate_signatures": 46 | return &aggregateSignaturesPhysical{}, 47 | physical.Schema{ 48 | TimeField: -1, 49 | Fields: []physical.SchemaField{ 50 | { 51 | Name: "name", 52 | Type: octosql.String, 53 | }, 54 | { 55 | Name: "argument_type", 56 | Type: octosql.String, 57 | }, 58 | { 59 | Name: "output_type", 60 | Type: octosql.String, 61 | }, 62 | { 63 | Name: "simple_signature", 64 | Type: octosql.Boolean, 65 | }, 66 | }, 67 | NoRetractions: true, 68 | }, 69 | nil 70 | case "functions": 71 | return &functionsPhysical{}, 72 | physical.Schema{ 73 | TimeField: -1, 74 | Fields: []physical.SchemaField{ 75 | { 76 | Name: "name", 77 | Type: octosql.String, 78 | }, 79 | { 80 | Name: "description", 81 | Type: octosql.String, 82 | }, 83 | }, 84 | NoRetractions: true, 85 | }, 86 | nil 87 | case "function_signatures": 88 | return &functionSignaturesPhysical{}, 89 | physical.Schema{ 90 | TimeField: -1, 91 | Fields: []physical.SchemaField{ 92 | { 93 | Name: "name", 94 | Type: octosql.String, 95 | }, 96 | { 97 | Name: "argument_types", 98 | Type: octosql.Type{ 99 | TypeID: octosql.TypeIDList, 100 | List: struct{ Element *octosql.Type }{Element: &octosql.String}, 101 | }, 102 | }, 103 | { 104 | Name: "output_type", 105 | Type: octosql.String, 106 | }, 107 | { 108 | Name: "strict", 109 | Type: octosql.Boolean, 110 | }, 111 | { 112 | Name: "simple_signature", 113 | Type: octosql.Boolean, 114 | }, 115 | }, 116 | NoRetractions: true, 117 | }, 118 | nil 119 | } 120 | return nil, physical.Schema{}, fmt.Errorf("unknown table: %s", name) 121 | } 122 | -------------------------------------------------------------------------------- /datasources/docs/function_signatures.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/functions" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | type functionSignaturesPhysical struct { 15 | } 16 | 17 | func (i *functionSignaturesPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 18 | return &functionSignaturesExecuting{ 19 | fields: schema.Fields, 20 | }, nil 21 | } 22 | 23 | func (i *functionSignaturesPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 24 | return newPredicates, []physical.Expression{}, false 25 | } 26 | 27 | type functionSignaturesExecuting struct { 28 | fields []physical.SchemaField 29 | } 30 | 31 | func (d *functionSignaturesExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 32 | fs := sortedMapNameAndDetails(functions.FunctionMap()) 33 | 34 | output := make([][]octosql.Value, 0) 35 | for _, f := range fs { 36 | if f.Details.Description == "" { 37 | continue 38 | } 39 | for _, descriptor := range f.Details.Descriptors { 40 | parts := make([]octosql.Value, len(descriptor.ArgumentTypes)) 41 | for i := range descriptor.ArgumentTypes { 42 | parts[i] = octosql.NewString(descriptor.ArgumentTypes[i].String()) 43 | } 44 | row := make([]octosql.Value, len(d.fields)) 45 | for i, field := range d.fields { 46 | switch field.Name { 47 | case "name": 48 | row[i] = octosql.NewString(f.Name) 49 | case "argument_types": 50 | row[i] = octosql.NewList(parts) 51 | case "output_type": 52 | row[i] = octosql.NewString(descriptor.OutputType.String()) 53 | case "strict": 54 | row[i] = octosql.NewBoolean(descriptor.Strict) 55 | case "simple_signature": 56 | row[i] = octosql.NewBoolean(descriptor.TypeFn == nil) 57 | } 58 | } 59 | output = append(output, row) 60 | } 61 | } 62 | 63 | for i := range output { 64 | if err := produce( 65 | ProduceFromExecutionContext(ctx), 66 | NewRecord(output[i], false, time.Time{}), 67 | ); err != nil { 68 | return fmt.Errorf("couldn't produce record: %w", err) 69 | } 70 | } 71 | 72 | return nil 73 | } 74 | -------------------------------------------------------------------------------- /datasources/docs/functions.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/functions" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | type functionsPhysical struct { 15 | } 16 | 17 | func (i *functionsPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 18 | return &functionsExecuting{ 19 | fields: schema.Fields, 20 | }, nil 21 | } 22 | 23 | func (i *functionsPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 24 | return newPredicates, []physical.Expression{}, false 25 | } 26 | 27 | type functionsExecuting struct { 28 | fields []physical.SchemaField 29 | } 30 | 31 | func (d *functionsExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 32 | fs := sortedMapNameAndDetails(functions.FunctionMap()) 33 | 34 | output := make([][]octosql.Value, 0) 35 | for _, f := range fs { 36 | if f.Details.Description == "" { 37 | continue 38 | } 39 | row := make([]octosql.Value, len(d.fields)) 40 | for i, field := range d.fields { 41 | switch field.Name { 42 | case "name": 43 | row[i] = octosql.NewString(f.Name) 44 | case "description": 45 | row[i] = octosql.NewString(f.Details.Description) 46 | } 47 | } 48 | output = append(output, row) 49 | } 50 | 51 | for i := range output { 52 | if err := produce( 53 | ProduceFromExecutionContext(ctx), 54 | NewRecord(output[i], false, time.Time{}), 55 | ); err != nil { 56 | return fmt.Errorf("couldn't produce record: %w", err) 57 | } 58 | } 59 | 60 | return nil 61 | } 62 | -------------------------------------------------------------------------------- /datasources/docs/helper.go: -------------------------------------------------------------------------------- 1 | package docs 2 | 3 | import ( 4 | "golang.org/x/exp/slices" 5 | ) 6 | 7 | type NameAndDetails[V any] struct { 8 | Name string 9 | Details V 10 | } 11 | 12 | func sortedMapNameAndDetails[V any](m map[string]V) []NameAndDetails[V] { 13 | pairs := make([]NameAndDetails[V], 0, len(m)) 14 | for k, v := range m { 15 | pairs = append(pairs, NameAndDetails[V]{k, v}) 16 | } 17 | slices.SortFunc(pairs, func(a, b NameAndDetails[V]) bool { 18 | return a.Name < b.Name 19 | }) 20 | return pairs 21 | } 22 | -------------------------------------------------------------------------------- /datasources/json/workers.go: -------------------------------------------------------------------------------- 1 | package json 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "runtime" 7 | "time" 8 | 9 | "github.com/valyala/fastjson" 10 | 11 | . "github.com/cube2222/octosql/execution" 12 | "github.com/cube2222/octosql/octosql" 13 | "github.com/cube2222/octosql/physical" 14 | ) 15 | 16 | // jobIn is a single job for the parser worker pool. 17 | // It contains a list of lines to parse into a list of records. 18 | type jobIn struct { 19 | fields []physical.SchemaField 20 | ctx context.Context 21 | 22 | lines []int // line numbers 23 | data [][]byte // data for each line 24 | 25 | // The channel to send job outputs to. 26 | outChan chan<- []jobOutRecord 27 | } 28 | 29 | // jobOutRecord is the result of parsing a single line. 30 | type jobOutRecord struct { 31 | line int 32 | record Record 33 | err error 34 | } 35 | 36 | // The function below creates the workers and stores the job queue in a global variable. 37 | var parserWorkReceiveChannel = func() chan<- jobIn { 38 | // We should be able to scale to all cores. 39 | workerCount := runtime.GOMAXPROCS(0) 40 | 41 | inChan := make(chan jobIn, 128) 42 | 43 | for i := 0; i < workerCount; i++ { 44 | go func() { 45 | var p fastjson.Parser 46 | 47 | getWorkLoop: 48 | for job := range inChan { 49 | outJobs := make([]jobOutRecord, len(job.lines)) 50 | for i := range outJobs { 51 | out := &outJobs[i] 52 | out.line = job.lines[i] 53 | 54 | v, err := p.ParseBytes(job.data[i]) 55 | if err != nil { 56 | out.err = fmt.Errorf("couldn't parse json: %w", err) 57 | continue 58 | } 59 | 60 | o, err := v.Object() 61 | if err != nil { 62 | out.err = fmt.Errorf("expected JSON object, got '%s'", string(job.data[i])) 63 | continue 64 | } 65 | 66 | values := make([]octosql.Value, len(job.fields)) 67 | for i := range values { 68 | values[i], _ = getOctoSQLValue(job.fields[i].Type, o.Get(job.fields[i].Name)) 69 | } 70 | 71 | out.record = NewRecord(values, false, time.Time{}) 72 | } 73 | select { 74 | case job.outChan <- outJobs: 75 | case <-job.ctx.Done(): 76 | continue getWorkLoop 77 | } 78 | } 79 | }() 80 | } 81 | 82 | return inChan 83 | }() 84 | -------------------------------------------------------------------------------- /datasources/lines/execution.go: -------------------------------------------------------------------------------- 1 | package lines 2 | 3 | import ( 4 | "bufio" 5 | "bytes" 6 | "fmt" 7 | "time" 8 | 9 | . "github.com/cube2222/octosql/execution" 10 | "github.com/cube2222/octosql/execution/files" 11 | "github.com/cube2222/octosql/octosql" 12 | "github.com/cube2222/octosql/physical" 13 | ) 14 | 15 | type DatasourceExecuting struct { 16 | path, separator string 17 | fields []physical.SchemaField 18 | tail bool 19 | } 20 | 21 | func (d *DatasourceExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 22 | f, err := files.OpenLocalFile(ctx, d.path, files.WithTail(d.tail)) 23 | if err != nil { 24 | return fmt.Errorf("couldn't open local file: %w", err) 25 | } 26 | defer f.Close() 27 | 28 | sc := bufio.NewScanner(f) 29 | if d.separator != "\n" { 30 | // Mostly copied from bufio.ScanLines. 31 | sc.Split(func(data []byte, atEOF bool) (advance int, token []byte, err error) { 32 | if atEOF && len(data) == 0 { 33 | return 0, nil, nil 34 | } 35 | if i := bytes.Index(data, []byte(d.separator)); i >= 0 { 36 | // We have a full separator-terminated line. 37 | return i + 1, data[0:i], nil 38 | } 39 | // If we're at EOF, we have a final, non-terminated line. Return it. 40 | if atEOF { 41 | return len(data), data, nil 42 | } 43 | // Request more data. 44 | return 0, nil, nil 45 | }) 46 | } 47 | 48 | line := int64(0) 49 | for sc.Scan() { 50 | values := make([]octosql.Value, len(d.fields)) 51 | for i := range d.fields { 52 | switch d.fields[i].Name { 53 | case "number": 54 | values[i] = octosql.NewInt(line) 55 | case "text": 56 | values[i] = octosql.NewString(sc.Text()) 57 | } 58 | } 59 | 60 | if err := produce(ProduceFromExecutionContext(ctx), NewRecord(values, false, time.Time{})); err != nil { 61 | return fmt.Errorf("couldn't produce record: %w", err) 62 | } 63 | line++ 64 | } 65 | if sc.Err() != nil { 66 | return err 67 | } 68 | return nil 69 | } 70 | -------------------------------------------------------------------------------- /datasources/lines/impl.go: -------------------------------------------------------------------------------- 1 | package lines 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | 7 | "github.com/cube2222/octosql/execution" 8 | "github.com/cube2222/octosql/execution/files" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | ) 12 | 13 | func Creator(ctx context.Context, name string, options map[string]string) (physical.DatasourceImplementation, physical.Schema, error) { 14 | f, err := files.OpenLocalFile(ctx, name, files.WithPreview()) 15 | if err != nil { 16 | return nil, physical.Schema{}, fmt.Errorf("couldn't open local file: %w", err) 17 | } 18 | f.Close() 19 | 20 | separator := "\n" 21 | if sep, ok := options["sep"]; ok { 22 | separator = sep 23 | } 24 | 25 | return &impl{ 26 | path: name, 27 | separator: separator, 28 | tail: options["tail"] == "true", 29 | }, 30 | physical.NewSchema( 31 | []physical.SchemaField{ 32 | { 33 | Name: "number", 34 | Type: octosql.Int, 35 | }, 36 | { 37 | Name: "text", 38 | Type: octosql.String, 39 | }, 40 | }, 41 | -1, 42 | physical.WithNoRetractions(true), 43 | ), 44 | nil 45 | } 46 | 47 | type impl struct { 48 | path, separator string 49 | tail bool 50 | } 51 | 52 | func (i *impl) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (execution.Node, error) { 53 | return &DatasourceExecuting{ 54 | path: i.path, 55 | fields: schema.Fields, 56 | separator: i.separator, 57 | tail: i.tail, 58 | }, nil 59 | } 60 | 61 | func (i *impl) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 62 | return newPredicates, []physical.Expression{}, false 63 | } 64 | -------------------------------------------------------------------------------- /datasources/memory/execution.go: -------------------------------------------------------------------------------- 1 | package memory 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | . "github.com/cube2222/octosql/execution" 8 | ) 9 | 10 | type Datasource struct { 11 | Entries []Entry 12 | } 13 | 14 | type Entry struct { 15 | Record Record 16 | WatermarkEntry bool 17 | Watermark time.Time 18 | } 19 | 20 | func (d *Datasource) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 21 | for _, entry := range d.Entries { 22 | if !entry.WatermarkEntry { 23 | if err := produce(ProduceFromExecutionContext(ctx), entry.Record); err != nil { 24 | return fmt.Errorf("couldn't produce record: %w", err) 25 | } 26 | } else { 27 | if err := metaSend(ProduceFromExecutionContext(ctx), MetadataMessage{ 28 | Type: MetadataMessageTypeWatermark, 29 | Watermark: entry.Watermark, 30 | }); err != nil { 31 | return fmt.Errorf("couldn't send metadata: %w", err) 32 | } 33 | } 34 | 35 | } 36 | return nil 37 | } 38 | -------------------------------------------------------------------------------- /datasources/parquet/execution.go: -------------------------------------------------------------------------------- 1 | package parquet 2 | 3 | import ( 4 | "fmt" 5 | "io" 6 | "os" 7 | "time" 8 | 9 | "github.com/segmentio/parquet-go" 10 | 11 | . "github.com/cube2222/octosql/execution" 12 | "github.com/cube2222/octosql/octosql" 13 | "github.com/cube2222/octosql/physical" 14 | ) 15 | 16 | type DatasourceExecuting struct { 17 | path string 18 | fields []physical.SchemaField 19 | } 20 | 21 | func (d *DatasourceExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 22 | f, err := os.Open(d.path) 23 | if err != nil { 24 | return fmt.Errorf("couldn't open file: %w", err) 25 | } 26 | defer f.Close() 27 | stat, err := f.Stat() 28 | if err != nil { 29 | return fmt.Errorf("couldn't stat file: %w", err) 30 | } 31 | 32 | pf, err := parquet.OpenFile(f, stat.Size(), &parquet.FileConfig{ 33 | SkipPageIndex: true, 34 | SkipBloomFilters: true, 35 | }) 36 | usedFields := make([]string, len(d.fields)) 37 | for i := range usedFields { 38 | usedFields[i] = d.fields[i].Name 39 | } 40 | pf.Schema().MakeColumnReadRowFunc(usedFields) 41 | reconstruct := reconstructFuncOfSchemaFields(pf.Schema(), usedFields) 42 | 43 | var row parquet.Row 44 | pr := parquet.NewReader(pf) 45 | if len(usedFields) > 0 { 46 | for { 47 | row, err := pr.ReadRow(row) 48 | if err != nil { 49 | if err == io.EOF { 50 | break 51 | } 52 | return fmt.Errorf("couldn't read row: %w", err) 53 | } 54 | var value octosql.Value 55 | if _, err := reconstruct(&value, levels{}, row); err != nil { 56 | return fmt.Errorf("couldn't reconstruct value from row: %w", err) 57 | } 58 | if err := produce(ProduceFromExecutionContext(ctx), NewRecord(value.Struct, false, time.Time{})); err != nil { 59 | return fmt.Errorf("couldn't produce value: %w", err) 60 | } 61 | } 62 | } else { 63 | rowCount := int(pr.NumRows()) 64 | for i := 0; i < rowCount; i++ { 65 | if err := produce(ProduceFromExecutionContext(ctx), NewRecord([]octosql.Value{}, false, time.Time{})); err != nil { 66 | return fmt.Errorf("couldn't produce value: %w", err) 67 | } 68 | } 69 | } 70 | 71 | return nil 72 | } 73 | -------------------------------------------------------------------------------- /datasources/plugins/available_plugins.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | "github.com/cube2222/octosql/plugins/repository" 12 | ) 13 | 14 | type availablePluginsPhysical struct { 15 | repositories []repository.Repository 16 | } 17 | 18 | func (i *availablePluginsPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 19 | return &availablePluginsExecuting{ 20 | repositories: i.repositories, 21 | fields: schema.Fields, 22 | }, nil 23 | } 24 | 25 | func (i *availablePluginsPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 26 | return newPredicates, []physical.Expression{}, false 27 | } 28 | 29 | type availablePluginsExecuting struct { 30 | repositories []repository.Repository 31 | fields []physical.SchemaField 32 | } 33 | 34 | func (d *availablePluginsExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 35 | for _, repo := range d.repositories { 36 | for _, plugin := range repo.Plugins { 37 | values := make([]octosql.Value, len(d.fields)) 38 | for i, field := range d.fields { 39 | switch field.Name { 40 | case "name": 41 | values[i] = octosql.NewString(plugin.Name) 42 | case "description": 43 | values[i] = octosql.NewString(plugin.Description) 44 | case "file_extensions": 45 | list := make([]octosql.Value, len(plugin.FileExtensions)) 46 | for j, ext := range plugin.FileExtensions { 47 | list[j] = octosql.NewString(ext) 48 | } 49 | values[i] = octosql.NewList(list) 50 | case "website": 51 | values[i] = octosql.NewString(plugin.Website) 52 | case "contact_email": 53 | values[i] = octosql.NewString(plugin.ContactEmail) 54 | case "license": 55 | values[i] = octosql.NewString(plugin.License) 56 | case "readme_url": 57 | values[i] = octosql.NewString(plugin.ReadmeURL) 58 | case "repo_slug": 59 | values[i] = octosql.NewString(repo.Slug) 60 | } 61 | } 62 | 63 | if err := produce( 64 | ProduceFromExecutionContext(ctx), 65 | NewRecord(values, false, time.Time{}), 66 | ); err != nil { 67 | return fmt.Errorf("couldn't produce record: %w", err) 68 | } 69 | } 70 | } 71 | 72 | return nil 73 | } 74 | -------------------------------------------------------------------------------- /datasources/plugins/installed_plugins.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | "github.com/cube2222/octosql/plugins/manager" 12 | ) 13 | 14 | type installedPluginsPhysical struct { 15 | manager *manager.PluginManager 16 | } 17 | 18 | func (i *installedPluginsPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 19 | return &installedPluginsExecuting{ 20 | manager: i.manager, 21 | fields: schema.Fields, 22 | }, nil 23 | } 24 | 25 | func (i *installedPluginsPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 26 | return newPredicates, []physical.Expression{}, false 27 | } 28 | 29 | type installedPluginsExecuting struct { 30 | manager *manager.PluginManager 31 | fields []physical.SchemaField 32 | } 33 | 34 | func (d *installedPluginsExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 35 | installedPlugins, err := d.manager.ListInstalledPlugins() 36 | if err != nil { 37 | return fmt.Errorf("failed to list installed plugins: %w", err) 38 | } 39 | 40 | for _, plugin := range installedPlugins { 41 | values := make([]octosql.Value, len(d.fields)) 42 | for i, field := range d.fields { 43 | switch field.Name { 44 | case "name": 45 | values[i] = octosql.NewString(plugin.Reference.Name) 46 | case "repo_slug": 47 | values[i] = octosql.NewString(plugin.Reference.Repository) 48 | } 49 | } 50 | 51 | if err := produce( 52 | ProduceFromExecutionContext(ctx), 53 | NewRecord(values, false, time.Time{}), 54 | ); err != nil { 55 | return fmt.Errorf("couldn't produce record: %w", err) 56 | } 57 | } 58 | 59 | return nil 60 | } 61 | -------------------------------------------------------------------------------- /datasources/plugins/installed_versions.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | "github.com/cube2222/octosql/plugins/manager" 12 | ) 13 | 14 | type installedVersionsPhysical struct { 15 | manager *manager.PluginManager 16 | } 17 | 18 | func (i *installedVersionsPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 19 | return &installedVersionsExecuting{ 20 | manager: i.manager, 21 | fields: schema.Fields, 22 | }, nil 23 | } 24 | 25 | func (i *installedVersionsPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 26 | return newPredicates, []physical.Expression{}, false 27 | } 28 | 29 | type installedVersionsExecuting struct { 30 | manager *manager.PluginManager 31 | fields []physical.SchemaField 32 | } 33 | 34 | func (d *installedVersionsExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 35 | installedPlugins, err := d.manager.ListInstalledPlugins() 36 | if err != nil { 37 | return fmt.Errorf("failed to list installed plugins: %w", err) 38 | } 39 | 40 | for _, plugin := range installedPlugins { 41 | for _, version := range plugin.Versions { 42 | values := make([]octosql.Value, len(d.fields)) 43 | for i, field := range d.fields { 44 | switch field.Name { 45 | case "version": 46 | values[i] = octosql.NewString(version.Number.String()) 47 | case "prerelease": 48 | values[i] = octosql.NewBoolean(version.Number.Prerelease() != "") 49 | case "plugin_name": 50 | values[i] = octosql.NewString(plugin.Reference.Name) 51 | case "repo_slug": 52 | values[i] = octosql.NewString(plugin.Reference.Repository) 53 | } 54 | } 55 | 56 | if err := produce( 57 | ProduceFromExecutionContext(ctx), 58 | NewRecord(values, false, time.Time{}), 59 | ); err != nil { 60 | return fmt.Errorf("couldn't produce record: %w", err) 61 | } 62 | } 63 | } 64 | 65 | return nil 66 | } 67 | -------------------------------------------------------------------------------- /datasources/plugins/repositories.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | "github.com/cube2222/octosql/plugins/repository" 12 | ) 13 | 14 | type repositoriesPhysical struct { 15 | repositories []repository.Repository 16 | } 17 | 18 | func (i *repositoriesPhysical) Materialize(ctx context.Context, env physical.Environment, schema physical.Schema, pushedDownPredicates []physical.Expression) (Node, error) { 19 | return &repositoriesExecuting{ 20 | repositories: i.repositories, 21 | fields: schema.Fields, 22 | }, nil 23 | } 24 | 25 | func (i *repositoriesPhysical) PushDownPredicates(newPredicates, pushedDownPredicates []physical.Expression) (rejected, pushedDown []physical.Expression, changed bool) { 26 | return newPredicates, []physical.Expression{}, false 27 | } 28 | 29 | type repositoriesExecuting struct { 30 | repositories []repository.Repository 31 | fields []physical.SchemaField 32 | } 33 | 34 | func (d *repositoriesExecuting) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 35 | for _, repo := range d.repositories { 36 | values := make([]octosql.Value, len(d.fields)) 37 | for i, field := range d.fields { 38 | switch field.Name { 39 | case "slug": 40 | values[i] = octosql.NewString(repo.Slug) 41 | case "name": 42 | values[i] = octosql.NewString(repo.Name) 43 | case "description": 44 | values[i] = octosql.NewString(repo.Description) 45 | } 46 | } 47 | 48 | if err := produce( 49 | ProduceFromExecutionContext(ctx), 50 | NewRecord(values, false, time.Time{}), 51 | ); err != nil { 52 | return fmt.Errorf("couldn't produce record: %w", err) 53 | } 54 | } 55 | 56 | return nil 57 | } 58 | -------------------------------------------------------------------------------- /execution/execution.go: -------------------------------------------------------------------------------- 1 | package execution 2 | 3 | import ( 4 | "context" 5 | "math" 6 | "strings" 7 | "time" 8 | 9 | "github.com/cube2222/octosql/octosql" 10 | ) 11 | 12 | type Node interface { 13 | Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error 14 | } 15 | 16 | type ExecutionContext struct { 17 | context.Context 18 | VariableContext *VariableContext 19 | } 20 | 21 | func (ctx ExecutionContext) WithRecord(record Record) ExecutionContext { 22 | return ExecutionContext{ 23 | Context: ctx.Context, 24 | VariableContext: ctx.VariableContext.WithRecord(record), 25 | } 26 | } 27 | 28 | type VariableContext struct { 29 | Parent *VariableContext 30 | Values []octosql.Value 31 | } 32 | 33 | func (varCtx *VariableContext) WithRecord(record Record) *VariableContext { 34 | return &VariableContext{ 35 | Parent: varCtx, 36 | Values: record.Values, 37 | } 38 | } 39 | 40 | type ProduceFn func(ctx ProduceContext, record Record) error 41 | 42 | type ProduceContext struct { 43 | context.Context 44 | } 45 | 46 | func ProduceFromExecutionContext(ctx ExecutionContext) ProduceContext { 47 | return ProduceContext{ 48 | Context: ctx.Context, 49 | } 50 | } 51 | 52 | func ProduceFnApplyContext(fn ProduceFn, ctx ProduceContext) func(record Record) error { 53 | return func(record Record) error { 54 | return fn(ctx, record) 55 | } 56 | } 57 | 58 | type Record struct { 59 | Values []octosql.Value 60 | Retraction bool 61 | EventTime time.Time 62 | } 63 | 64 | // Functional options? 65 | func NewRecord(values []octosql.Value, retraction bool, eventTime time.Time) Record { 66 | return Record{ 67 | Values: values, 68 | Retraction: retraction, 69 | EventTime: eventTime, 70 | } 71 | } 72 | 73 | func (record Record) String() string { 74 | builder := strings.Builder{} 75 | builder.WriteString("{") 76 | if !record.Retraction { 77 | builder.WriteString("+") 78 | } else { 79 | builder.WriteString("-") 80 | } 81 | builder.WriteString(record.EventTime.Format(time.RFC3339)) 82 | builder.WriteString("| ") 83 | for i := range record.Values { 84 | builder.WriteString(record.Values[i].String()) 85 | if i != len(record.Values)-1 { 86 | builder.WriteString(", ") 87 | } 88 | } 89 | builder.WriteString(" |}") 90 | return builder.String() 91 | } 92 | 93 | type MetaSendFn func(ctx ProduceContext, msg MetadataMessage) error 94 | 95 | type MetadataMessage struct { 96 | Type MetadataMessageType 97 | Watermark time.Time 98 | } 99 | 100 | type MetadataMessageType int 101 | 102 | const ( 103 | MetadataMessageTypeWatermark MetadataMessageType = iota 104 | ) 105 | 106 | var WatermarkMaxValue = time.Unix(0, math.MaxInt64) 107 | -------------------------------------------------------------------------------- /execution/files/files.go: -------------------------------------------------------------------------------- 1 | package files 2 | 3 | import ( 4 | "bufio" 5 | "context" 6 | "fmt" 7 | "io" 8 | "os" 9 | "strings" 10 | "sync" 11 | 12 | "github.com/nxadm/tail" 13 | 14 | "github.com/cube2222/octosql/config" 15 | ) 16 | 17 | type customCloser struct { 18 | io.Reader 19 | close func() error 20 | } 21 | 22 | func (c *customCloser) Close() error { 23 | return c.close() 24 | } 25 | 26 | func Tail(ctx context.Context, path string) (io.ReadCloser, error) { 27 | wg := sync.WaitGroup{} 28 | wg.Add(1) 29 | 30 | t, err := tail.TailFile(path, tail.Config{ 31 | MustExist: true, 32 | Follow: true, 33 | ReOpen: true, 34 | }) 35 | if err != nil { 36 | return nil, fmt.Errorf("couldn't tail file: %w", err) 37 | } 38 | 39 | pr, pw := io.Pipe() 40 | 41 | go func() { 42 | defer wg.Done() 43 | loop: 44 | for { 45 | select { 46 | case line := <-t.Lines: 47 | if line == nil { 48 | t.Stop() 49 | t.Cleanup() 50 | pw.Close() 51 | break loop 52 | } else if line.Err != nil { 53 | pw.CloseWithError(fmt.Errorf("couldn't read line: %w", line.Err)) 54 | t.Cleanup() 55 | break loop 56 | } 57 | pw.Write([]byte(line.Text + "\n")) 58 | case <-ctx.Done(): 59 | t.Stop() 60 | t.Cleanup() 61 | pw.Close() 62 | break loop 63 | } 64 | } 65 | }() 66 | 67 | return &customCloser{ 68 | Reader: pr, 69 | close: func() error { 70 | pr.Close() 71 | t.Kill(nil) 72 | wg.Wait() 73 | return nil 74 | }, 75 | }, nil 76 | } 77 | 78 | type openFileOptions struct { 79 | tail bool 80 | preview bool 81 | } 82 | 83 | type OpenFileOption func(*openFileOptions) 84 | 85 | // WithPreview means we're opening to preview the file, but we'll still need that portion later, when opening it again. 86 | func WithPreview() OpenFileOption { 87 | return func(options *openFileOptions) { 88 | options.preview = true 89 | } 90 | } 91 | 92 | func WithTail(tail bool) OpenFileOption { 93 | return func(options *openFileOptions) { 94 | options.tail = tail 95 | } 96 | } 97 | 98 | func OpenLocalFile(ctx context.Context, path string, opts ...OpenFileOption) (io.ReadCloser, error) { 99 | openFileOpts := &openFileOptions{ 100 | tail: false, 101 | } 102 | for _, opt := range opts { 103 | opt(openFileOpts) 104 | } 105 | 106 | if path == "stdin" || strings.HasPrefix(path, "stdin.") { 107 | f, err := openStdin(openFileOpts.preview) 108 | if err != nil { 109 | return nil, fmt.Errorf("couldn't open stdin: %w", err) 110 | } 111 | return f, nil 112 | } else if !openFileOpts.tail { 113 | f, err := os.Open(path) 114 | if err != nil { 115 | return nil, fmt.Errorf("couldn't open file: %w", err) 116 | } 117 | return &customCloser{ 118 | Reader: bufio.NewReaderSize(f, config.FromContext(ctx).Files.BufferSizeBytes), 119 | close: f.Close, 120 | }, nil 121 | } else { 122 | r, err := Tail(ctx, path) 123 | if err != nil { 124 | return nil, fmt.Errorf("couldn't tail file: %w", err) 125 | } 126 | return r, nil 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /execution/files/stdin.go: -------------------------------------------------------------------------------- 1 | package files 2 | 3 | import ( 4 | "bytes" 5 | "fmt" 6 | "io" 7 | "os" 8 | "sync" 9 | "sync/atomic" 10 | ) 11 | 12 | var previewedBuffer = &bytes.Buffer{} 13 | var previewedBufferMutex sync.Mutex 14 | var alreadyOpenedNoPreview int64 15 | var concurrentReaders int64 16 | 17 | type stdinPreviewingReader struct { 18 | } 19 | 20 | func (r *stdinPreviewingReader) Read(p []byte) (n int, err error) { 21 | n, err = os.Stdin.Read(p) 22 | previewedBufferMutex.Lock() 23 | previewedBuffer.Write(p[:n]) 24 | previewedBufferMutex.Unlock() 25 | return 26 | } 27 | 28 | type concurrentReaderDecrementingCloser struct { 29 | io.Reader 30 | } 31 | 32 | func (r *concurrentReaderDecrementingCloser) Close() error { 33 | atomic.AddInt64(&concurrentReaders, -1) 34 | return nil 35 | } 36 | 37 | func openStdin(preview bool) (io.ReadCloser, error) { 38 | if atomic.AddInt64(&concurrentReaders, 1) > 1 { 39 | return nil, fmt.Errorf("only one simultaneous stdin reader is allowed") 40 | } 41 | 42 | if preview { 43 | previewedPortionCopy := make([]byte, previewedBuffer.Len()) 44 | copy(previewedPortionCopy, previewedBuffer.Bytes()) 45 | return &concurrentReaderDecrementingCloser{io.MultiReader(bytes.NewReader(previewedPortionCopy), &stdinPreviewingReader{})}, nil 46 | } 47 | 48 | if atomic.AddInt64(&alreadyOpenedNoPreview, 1) > 1 { 49 | return nil, fmt.Errorf("stdin already opened") 50 | } 51 | 52 | previewedBufferBytes := previewedBuffer.Bytes() 53 | previewedBuffer = nil 54 | 55 | return &concurrentReaderDecrementingCloser{io.MultiReader(bytes.NewReader(previewedBufferBytes), os.Stdin)}, nil 56 | } 57 | -------------------------------------------------------------------------------- /execution/group_key.go: -------------------------------------------------------------------------------- 1 | package execution 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/google/btree" 7 | 8 | "github.com/cube2222/octosql/octosql" 9 | ) 10 | 11 | type GroupKey []octosql.Value 12 | 13 | type GroupKeyIface interface { 14 | GetGroupKey() GroupKey 15 | } 16 | 17 | func (key GroupKey) GetGroupKey() GroupKey { 18 | return key 19 | } 20 | 21 | func (key GroupKey) Less(than btree.Item) bool { 22 | thanTyped, ok := than.(GroupKeyIface) 23 | if !ok { 24 | panic(fmt.Sprintf("invalid key comparison: %T", than)) 25 | } 26 | 27 | thanKey := thanTyped.GetGroupKey() 28 | 29 | return CompareValueSlices(key, thanKey) 30 | } 31 | 32 | func CompareValueSlices(key, than []octosql.Value) bool { 33 | maxLen := len(key) 34 | if len(than) > maxLen { 35 | maxLen = len(than) 36 | } 37 | 38 | for i := 0; i < maxLen; i++ { 39 | if i == len(key) { 40 | return true 41 | } else if i == len(than) { 42 | return false 43 | } 44 | 45 | if comp := key[i].Compare(than[i]); comp != 0 { 46 | return comp == -1 47 | } 48 | } 49 | 50 | return false 51 | } 52 | -------------------------------------------------------------------------------- /execution/nodes/distinct.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/zyedidia/generic/hashmap" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | ) 11 | 12 | type Distinct struct { 13 | source Node 14 | } 15 | 16 | func NewDistinct(source Node) *Distinct { 17 | return &Distinct{ 18 | source: source, 19 | } 20 | } 21 | 22 | type distinctItem struct { 23 | Count int 24 | } 25 | 26 | func (o *Distinct) Run(execCtx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 27 | recordCounts := hashmap.New[[]octosql.Value, *distinctItem]( 28 | BTreeDefaultDegree, 29 | func(a, b []octosql.Value) bool { 30 | for i := range a { 31 | if a[i].Compare(b[i]) != 0 { 32 | return false 33 | } 34 | } 35 | return true 36 | }, func(k []octosql.Value) uint64 { 37 | return octosql.HashManyValues(k) 38 | }) 39 | o.source.Run( 40 | execCtx, 41 | func(ctx ProduceContext, record Record) error { 42 | item, ok := recordCounts.Get(record.Values) 43 | if !ok { 44 | item = &distinctItem{ 45 | Count: 0, 46 | } 47 | } 48 | if !record.Retraction { 49 | item.Count++ 50 | } else { 51 | item.Count-- 52 | } 53 | if item.Count > 0 { 54 | if !record.Retraction && item.Count == 1 { 55 | // New record. 56 | if err := produce(ctx, record); err != nil { 57 | return fmt.Errorf("couldn't produce new record: %w", err) 58 | } 59 | recordCounts.Put(record.Values, item) 60 | } 61 | } else { 62 | if err := produce(ctx, record); err != nil { 63 | return fmt.Errorf("couldn't retract record record: %w", err) 64 | } 65 | recordCounts.Remove(record.Values) 66 | } 67 | return nil 68 | }, 69 | func(ctx ProduceContext, msg MetadataMessage) error { 70 | return nil 71 | }, 72 | ) 73 | 74 | return nil 75 | } 76 | -------------------------------------------------------------------------------- /execution/nodes/event_time_buffer.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | . "github.com/cube2222/octosql/execution" 7 | ) 8 | 9 | type EventTimeBuffer struct { 10 | source Node 11 | } 12 | 13 | func NewEventTimeBuffer(source Node) *EventTimeBuffer { 14 | return &EventTimeBuffer{source: source} 15 | } 16 | 17 | func (e *EventTimeBuffer) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 18 | records := NewRecordEventTimeBuffer() 19 | 20 | if err := e.source.Run( 21 | ctx, 22 | func(ctx ProduceContext, record Record) error { 23 | if record.EventTime.IsZero() { 24 | // If the event time is zero, don't buffer, there's no point. 25 | // There won't be any record with an event time less than zero. 26 | return produce(ctx, record) 27 | } 28 | records.AddRecord(record) 29 | return nil 30 | }, 31 | func(ctx ProduceContext, msg MetadataMessage) error { 32 | if msg.Type == MetadataMessageTypeWatermark { 33 | if err := records.Emit(msg.Watermark, ProduceFnApplyContext(produce, ctx)); err != nil { 34 | return fmt.Errorf("couldn't emit records up to watermark: %w", err) 35 | } 36 | } 37 | return metaSend(ctx, msg) 38 | }); err != nil { 39 | return err 40 | } 41 | 42 | if err := records.Emit(WatermarkMaxValue, ProduceFnApplyContext(produce, ProduceFromExecutionContext(ctx))); err != nil { 43 | return fmt.Errorf("couldn't emit remaining records: %w", err) 44 | } 45 | return nil 46 | } 47 | -------------------------------------------------------------------------------- /execution/nodes/filter.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | . "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/octosql" 8 | ) 9 | 10 | type Filter struct { 11 | source Node 12 | predicate Expression 13 | } 14 | 15 | func NewFilter(source Node, predicate Expression) *Filter { 16 | return &Filter{ 17 | source: source, 18 | predicate: predicate, 19 | } 20 | } 21 | 22 | func (m *Filter) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 23 | if err := m.source.Run(ctx, func(produceCtx ProduceContext, record Record) error { 24 | ctx := ctx.WithRecord(record) 25 | 26 | ok, err := m.predicate.Evaluate(ctx) 27 | if err != nil { 28 | return fmt.Errorf("couldn't evaluate condition: %w", err) 29 | } 30 | if ok.TypeID == octosql.TypeIDBoolean && ok.Boolean { 31 | if err := produce(produceCtx, record); err != nil { 32 | return fmt.Errorf("couldn't produce: %w", err) 33 | } 34 | } 35 | 36 | return nil 37 | }, metaSend); err != nil { 38 | return fmt.Errorf("couldn't run source: %w", err) 39 | } 40 | return nil 41 | } 42 | -------------------------------------------------------------------------------- /execution/nodes/in_memory_records.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/octosql" 8 | ) 9 | 10 | type InMemoryRecords struct { 11 | records []execution.Record 12 | } 13 | 14 | func NewInMemoryRecords(records []execution.Record) *InMemoryRecords { 15 | return &InMemoryRecords{ 16 | records: records, 17 | } 18 | } 19 | 20 | func (r *InMemoryRecords) Run(ctx execution.ExecutionContext, produce execution.ProduceFn, metaSend execution.MetaSendFn) error { 21 | for i := 0; i < len(r.records); i++ { 22 | recordValues := make([]octosql.Value, len(r.records[i].Values)) 23 | copy(recordValues, r.records[i].Values) 24 | 25 | if err := produce( 26 | execution.ProduceFromExecutionContext(ctx), 27 | execution.NewRecord(recordValues, r.records[i].Retraction, r.records[i].EventTime), 28 | ); err != nil { 29 | return fmt.Errorf("couldn't produce record: %w", err) 30 | } 31 | } 32 | return nil 33 | } 34 | -------------------------------------------------------------------------------- /execution/nodes/limit.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "crypto/rand" 5 | "fmt" 6 | "strings" 7 | 8 | "github.com/oklog/ulid/v2" 9 | 10 | . "github.com/cube2222/octosql/execution" 11 | ) 12 | 13 | type Limit struct { 14 | source Node 15 | limit Expression 16 | } 17 | 18 | func NewLimit(source Node, limit Expression) *Limit { 19 | return &Limit{ 20 | source: source, 21 | limit: limit, 22 | } 23 | } 24 | 25 | func (m *Limit) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 26 | limit, err := m.limit.Evaluate(ctx) 27 | if err != nil { 28 | return fmt.Errorf("couldn't evaluate limit expression: %w", err) 29 | } 30 | 31 | limitNodeID := ulid.MustNew(ulid.Now(), rand.Reader).String() 32 | 33 | i := int64(0) 34 | if err := m.source.Run(ctx, func(produceCtx ProduceContext, record Record) error { 35 | if err := produce(produceCtx, record); err != nil { 36 | return fmt.Errorf("couldn't produce: %w", err) 37 | } 38 | i++ 39 | 40 | if i == limit.Int { 41 | // This error is returned because the limit has been reached, to stop underlying processing. 42 | // It will be caught and silenced by the Limit node that emitted it. 43 | return fmt.Errorf("limit %s reached", limitNodeID) 44 | } 45 | 46 | return nil 47 | }, metaSend); err != nil { 48 | // We can't Unwrap because gRPC doesn't propagate wrapped errors, so we can't Unwrap over the plugin barrier. 49 | if strings.Contains(err.Error(), fmt.Sprintf("limit %s reached", limitNodeID)) { 50 | return nil 51 | } 52 | return fmt.Errorf("couldn't run source: %w", err) 53 | } 54 | return nil 55 | } 56 | -------------------------------------------------------------------------------- /execution/nodes/lookup_join.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | . "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/octosql" 8 | ) 9 | 10 | type LookupJoin struct { 11 | source, joined Node 12 | } 13 | 14 | func NewLookupJoin(source, joined Node) *LookupJoin { 15 | return &LookupJoin{ 16 | source: source, 17 | joined: joined, 18 | } 19 | } 20 | 21 | func (s *LookupJoin) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 22 | // TODO: Add parallelism here. 23 | 24 | // TODO: Maybe this should be fully parallel with no ordering guarantees and maybe a parallelism limit (semaphore). This way it would kinda work when the joined stream has a time field. 25 | if err := s.source.Run(ctx, func(produceCtx ProduceContext, sourceRecord Record) error { 26 | ctx := ctx.WithRecord(sourceRecord) 27 | 28 | if err := s.joined.Run(ctx, func(produceCtx ProduceContext, joinedRecord Record) error { 29 | outputValues := make([]octosql.Value, len(sourceRecord.Values)+len(joinedRecord.Values)) 30 | 31 | copy(outputValues, sourceRecord.Values) 32 | copy(outputValues[len(sourceRecord.Values):], joinedRecord.Values) 33 | 34 | retraction := (sourceRecord.Retraction || joinedRecord.Retraction) && !(sourceRecord.Retraction && joinedRecord.Retraction) 35 | 36 | if err := produce(ProduceFromExecutionContext(ctx), NewRecord(outputValues, retraction, sourceRecord.EventTime)); err != nil { 37 | return fmt.Errorf("couldn't produce: %w", err) 38 | } 39 | 40 | return nil 41 | }, metaSend); err != nil { 42 | return fmt.Errorf("couldn't run joined stream: %w", err) 43 | } 44 | 45 | return nil 46 | }, metaSend); err != nil { 47 | return fmt.Errorf("couldn't run source: %w", err) 48 | } 49 | return nil 50 | } 51 | -------------------------------------------------------------------------------- /execution/nodes/map.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | . "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/octosql" 8 | ) 9 | 10 | type Map struct { 11 | source Node 12 | exprs []Expression 13 | } 14 | 15 | func NewMap(source Node, exprs []Expression) *Map { 16 | return &Map{ 17 | source: source, 18 | exprs: exprs, 19 | } 20 | } 21 | 22 | func (m *Map) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 23 | if err := m.source.Run(ctx, func(produceCtx ProduceContext, record Record) error { 24 | ctx := ctx.WithRecord(record) 25 | 26 | // TODO: Reuse this slice on every produce call? NO, because of stream join for instance. 27 | values := make([]octosql.Value, len(m.exprs)) 28 | for i, expr := range m.exprs { 29 | value, err := expr.Evaluate(ctx) 30 | if err != nil { 31 | return fmt.Errorf("couldn't evaluate %d map expression: %w", i, err) 32 | } 33 | values[i] = value 34 | } 35 | if err := produce(produceCtx, NewRecord(values, record.Retraction, record.EventTime)); err != nil { 36 | return fmt.Errorf("couldn't produce: %w", err) 37 | } 38 | 39 | return nil 40 | }, metaSend); err != nil { 41 | return fmt.Errorf("couldn't run source: %w", err) 42 | } 43 | return nil 44 | } 45 | -------------------------------------------------------------------------------- /execution/nodes/unnest.go: -------------------------------------------------------------------------------- 1 | package nodes 2 | 3 | import ( 4 | "fmt" 5 | 6 | . "github.com/cube2222/octosql/execution" 7 | "github.com/cube2222/octosql/octosql" 8 | ) 9 | 10 | // TODO: Map -> Unnest 11 | 12 | type Unnest struct { 13 | source Node 14 | index int 15 | } 16 | 17 | func NewUnnest(source Node, index int) *Unnest { 18 | return &Unnest{source: source, index: index} 19 | } 20 | 21 | func (u *Unnest) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 22 | return u.source.Run(ctx, func(ctx ProduceContext, record Record) error { 23 | list := record.Values[u.index].List 24 | for i := range list { 25 | values := make([]octosql.Value, len(record.Values)) 26 | copy(values, record.Values[:u.index]) 27 | values[u.index] = list[i] 28 | if u.index < len(record.Values)-1 { 29 | copy(values[u.index+1:], record.Values[u.index+1:]) 30 | } 31 | if err := produce(ctx, NewRecord(values, record.Retraction, record.EventTime)); err != nil { 32 | return fmt.Errorf("couldn't produce unnested record: %w", err) 33 | } 34 | } 35 | return nil 36 | }, metaSend) 37 | } 38 | -------------------------------------------------------------------------------- /execution/record_event_time_buffer.go: -------------------------------------------------------------------------------- 1 | package execution 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | "github.com/google/btree" 8 | ) 9 | 10 | type RecordEventTimeBuffer struct { 11 | tree *btree.BTree 12 | } 13 | 14 | func NewRecordEventTimeBuffer() *RecordEventTimeBuffer { 15 | return &RecordEventTimeBuffer{ 16 | tree: btree.New(BTreeDefaultDegree), 17 | } 18 | } 19 | 20 | type recordEventTimeBufferItem struct { 21 | EventTime time.Time 22 | Records []Record 23 | } 24 | 25 | func (e *recordEventTimeBufferItem) Less(than btree.Item) bool { 26 | thanTyped, ok := than.(*recordEventTimeBufferItem) 27 | if !ok { 28 | panic(fmt.Sprintf("invalid event time key comparison: %T", than)) 29 | } 30 | 31 | return e.EventTime.Before(thanTyped.EventTime) 32 | } 33 | 34 | func (b *RecordEventTimeBuffer) AddRecord(record Record) { 35 | item := b.tree.Get(&recordEventTimeBufferItem{EventTime: record.EventTime}) 36 | var itemTyped *recordEventTimeBufferItem 37 | 38 | if item == nil { 39 | itemTyped = &recordEventTimeBufferItem{EventTime: record.EventTime} 40 | b.tree.ReplaceOrInsert(itemTyped) 41 | } else { 42 | var ok bool 43 | itemTyped, ok = item.(*recordEventTimeBufferItem) 44 | if !ok { 45 | panic(fmt.Sprintf("invalid event time buffer item: %v", item)) 46 | } 47 | } 48 | 49 | itemTyped.Records = append(itemTyped.Records, record) 50 | } 51 | 52 | func (b *RecordEventTimeBuffer) Emit(watermark time.Time, produce func(record Record) error) error { 53 | min := b.tree.Min() 54 | for min != nil && !min.(*recordEventTimeBufferItem).EventTime.After(watermark) { 55 | b.tree.DeleteMin() 56 | for _, record := range min.(*recordEventTimeBufferItem).Records { 57 | if err := produce(record); err != nil { 58 | return err 59 | } 60 | } 61 | min = b.tree.Min() 62 | } 63 | return nil 64 | } 65 | 66 | func (b *RecordEventTimeBuffer) Empty() bool { 67 | return b.tree.Len() == 0 68 | } 69 | -------------------------------------------------------------------------------- /execution/triggers_test.go: -------------------------------------------------------------------------------- 1 | package execution 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | 8 | "github.com/cube2222/octosql/octosql" 9 | ) 10 | 11 | func TestCountingTrigger(t *testing.T) { 12 | trigger := NewCountingTriggerPrototype(2)() 13 | trigger.KeyReceived(GroupKey{octosql.NewInt(2), octosql.NewInt(3)}) 14 | assert.Empty(t, trigger.Poll()) 15 | trigger.KeyReceived(GroupKey{octosql.NewInt(2), octosql.NewInt(3)}) 16 | polled := trigger.Poll() 17 | assert.Len(t, polled, 1) 18 | assert.Equal(t, polled[0], GroupKey{octosql.NewInt(2), octosql.NewInt(3)}) 19 | } 20 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/cube2222/octosql 2 | 3 | go 1.18 4 | 5 | require ( 6 | github.com/Masterminds/semver v1.5.0 7 | github.com/adrg/xdg v0.4.0 8 | github.com/awalterschulze/gographviz v2.0.3+incompatible 9 | github.com/c-bata/go-prompt v0.2.6 10 | github.com/dgraph-io/ristretto v0.0.3 11 | github.com/golang/protobuf v1.5.3 12 | github.com/google/btree v1.1.2 13 | github.com/gosuri/uilive v0.0.4 14 | github.com/jackc/pgx v3.6.2+incompatible 15 | github.com/kr/text v0.2.0 16 | github.com/mholt/archiver v3.1.1+incompatible 17 | github.com/mitchellh/go-homedir v1.1.0 18 | github.com/nxadm/tail v1.4.8 19 | github.com/oklog/ulid/v2 v2.0.2 20 | github.com/olekukonko/tablewriter v0.0.5 21 | github.com/pkg/errors v0.9.1 22 | github.com/pkg/profile v1.6.0 23 | github.com/pmezard/go-difflib v1.0.0 24 | github.com/segmentio/fasthash v1.0.3 25 | github.com/segmentio/parquet-go v0.0.0-20220421002521-93f8e5ed3407 26 | github.com/skratchdot/open-golang v0.0.0-20200116055534-eef842397966 27 | github.com/spf13/cobra v1.4.0 28 | github.com/stretchr/testify v1.7.0 29 | github.com/tidwall/btree v1.3.1 30 | github.com/valyala/fastjson v1.6.3 31 | github.com/zyedidia/generic v1.1.0 32 | golang.org/x/exp v0.0.0-20220414153411-bcd21879b8fd 33 | google.golang.org/grpc v1.55.0 34 | google.golang.org/protobuf v1.30.0 35 | gopkg.in/yaml.v3 v3.0.1 36 | ) 37 | 38 | require ( 39 | github.com/andybalholm/brotli v1.0.3 // indirect 40 | github.com/cespare/xxhash v1.1.0 // indirect 41 | github.com/cockroachdb/apd v1.1.0 // indirect 42 | github.com/davecgh/go-spew v1.1.1 // indirect 43 | github.com/dsnet/compress v0.0.1 // indirect 44 | github.com/frankban/quicktest v1.14.0 // indirect 45 | github.com/fsnotify/fsnotify v1.4.9 // indirect 46 | github.com/gofrs/uuid v4.0.0+incompatible // indirect 47 | github.com/golang/snappy v0.0.4 // indirect 48 | github.com/google/uuid v1.3.0 // indirect 49 | github.com/inconshreveable/mousetrap v1.0.0 // indirect 50 | github.com/jackc/fake v0.0.0-20150926172116-812a484cc733 // indirect 51 | github.com/klauspost/compress v1.15.2 // indirect 52 | github.com/lib/pq v1.9.0 // indirect 53 | github.com/mattn/go-colorable v0.1.12 // indirect 54 | github.com/mattn/go-isatty v0.0.14 // indirect 55 | github.com/mattn/go-runewidth v0.0.13 // indirect 56 | github.com/mattn/go-tty v0.0.3 // indirect 57 | github.com/nwaples/rardecode v1.1.2 // indirect 58 | github.com/pierrec/lz4 v2.6.1+incompatible // indirect 59 | github.com/pierrec/lz4/v4 v4.1.9 // indirect 60 | github.com/pkg/term v1.2.0-beta.2 // indirect 61 | github.com/rivo/uniseg v0.2.0 // indirect 62 | github.com/segmentio/encoding v0.3.5 // indirect 63 | github.com/shopspring/decimal v1.2.0 // indirect 64 | github.com/spf13/pflag v1.0.5 // indirect 65 | github.com/ulikunitz/xz v0.5.10 // indirect 66 | github.com/xi2/xz v0.0.0-20171230120015-48954b6210f8 // indirect 67 | golang.org/x/crypto v0.9.0 // indirect 68 | golang.org/x/net v0.10.0 // indirect 69 | golang.org/x/sys v0.8.0 // indirect 70 | golang.org/x/text v0.9.0 // indirect 71 | google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4 // indirect 72 | gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect 73 | gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect 74 | ) 75 | 76 | replace github.com/segmentio/parquet-go v0.0.0-20220421002521-93f8e5ed3407 => github.com/cube2222/parquet-go v0.0.0-20220512155810-0e06eee50261 77 | -------------------------------------------------------------------------------- /helpers/graph/graph.go: -------------------------------------------------------------------------------- 1 | package graph 2 | 3 | import ( 4 | "fmt" 5 | "log" 6 | "strings" 7 | 8 | "github.com/awalterschulze/gographviz" 9 | ) 10 | 11 | type Field struct { 12 | Name, Value string 13 | } 14 | 15 | type Child struct { 16 | Name string 17 | Node *Node 18 | } 19 | 20 | type Node struct { 21 | Name string 22 | Fields []Field 23 | Children []Child 24 | } 25 | 26 | func NewNode(name string) *Node { 27 | return &Node{ 28 | Name: name, 29 | } 30 | } 31 | 32 | func (n *Node) AddField(name, value string) { 33 | n.Fields = append(n.Fields, Field{ 34 | Name: name, 35 | Value: value, 36 | }) 37 | } 38 | 39 | func (n *Node) AddChild(name string, node *Node) { 40 | n.Children = append(n.Children, Child{ 41 | Name: name, 42 | Node: node, 43 | }) 44 | } 45 | 46 | type Visualizer interface { 47 | Visualize() *Node 48 | } 49 | 50 | func Show(node *Node) *gographviz.Graph { 51 | graph := gographviz.NewGraph() 52 | graph.Directed = true 53 | err := graph.AddAttr("", "rankdir", "LR") 54 | if err != nil { 55 | log.Fatal(err) 56 | } 57 | builder := &graphBuilder{ 58 | graph: graph, 59 | nameCounters: make(map[string]int), 60 | } 61 | 62 | getGraphNode(builder, node) 63 | 64 | return graph 65 | } 66 | 67 | type graphBuilder struct { 68 | graph *gographviz.Graph 69 | nameCounters map[string]int 70 | } 71 | 72 | func (gb *graphBuilder) getID(name string) string { 73 | count := gb.nameCounters[name] 74 | gb.nameCounters[name]++ 75 | return fmt.Sprintf("%s_%d", strings.Replace(name, " ", "_", -1), count) 76 | } 77 | 78 | func getGraphNode(gb *graphBuilder, node *Node) string { 79 | fields := make([]string, len(node.Fields)) 80 | for i, field := range node.Fields { 81 | if field.Value != "" { 82 | fields[i] = fmt.Sprintf("<%s> %s: %s", field.Name, field.Name, field.Value) 83 | } else { 84 | fields[i] = fmt.Sprintf("<%s> %s", field.Name, field.Name) 85 | } 86 | } 87 | childPorts := make([]string, 0) 88 | for _, child := range node.Children { 89 | if child.Name != "" { 90 | childPorts = append(childPorts, fmt.Sprintf("<%s> %s", child.Name, child.Name)) 91 | } 92 | } 93 | 94 | var labelParts []string 95 | labelParts = append(labelParts, fmt.Sprintf(" %s", node.Name)) 96 | 97 | if len(fields) > 0 { 98 | labelParts = append(labelParts, strings.Join(fields, "|")) 99 | } 100 | if len(childPorts) > 0 { 101 | labelParts = append(labelParts, strings.Join(childPorts, "|")) 102 | } 103 | 104 | label := fmt.Sprintf( 105 | "\"{{%s}}\"", 106 | strings.Join(labelParts, "}|{"), 107 | ) 108 | 109 | id := "<" + gb.getID(node.Name) + ">" 110 | err := gb.graph.AddNode("", id, map[string]string{ 111 | "shape": "record", 112 | "label": label, 113 | }) 114 | if err != nil { 115 | log.Fatal(err) 116 | } 117 | 118 | for _, child := range node.Children { 119 | childGraphNode := getGraphNode(gb, child.Node) 120 | if child.Name != "" { 121 | if err := gb.graph.AddPortEdge(id, "<"+child.Name+">", childGraphNode, "", true, map[string]string{}); err != nil { 122 | log.Fatal(err) 123 | } 124 | } else { 125 | if err := gb.graph.AddEdge(id, childGraphNode, true, map[string]string{}); err != nil { 126 | log.Fatal(err) 127 | } 128 | } 129 | } 130 | return id 131 | } 132 | -------------------------------------------------------------------------------- /images/logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/images/logo.png -------------------------------------------------------------------------------- /images/octosql-demo-dataflow.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/images/octosql-demo-dataflow.gif -------------------------------------------------------------------------------- /images/octosql-demo.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/images/octosql-demo.gif -------------------------------------------------------------------------------- /images/octosql-explain.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/images/octosql-explain.png -------------------------------------------------------------------------------- /images/octosql.svg: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 5 | 6 | 7 | 8 | 9 | 10 | 11 | 12 | 13 | 14 | 15 | 16 | image/svg+xml 17 | 18 | 19 | 20 | 21 | 22 | 23 | 24 | 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | -------------------------------------------------------------------------------- /logical/distinct.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | type Distinct struct { 10 | source Node 11 | } 12 | 13 | func NewDistinct(source Node) *Distinct { 14 | return &Distinct{source: source} 15 | } 16 | 17 | func (node *Distinct) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 18 | source, mapping := node.source.Typecheck(ctx, env, logicalEnv) 19 | 20 | return physical.Node{ 21 | Schema: source.Schema, 22 | NodeType: physical.NodeTypeDistinct, 23 | Distinct: &physical.Distinct{ 24 | Source: source, 25 | }, 26 | }, mapping 27 | } 28 | -------------------------------------------------------------------------------- /logical/filter.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/octosql" 7 | "github.com/cube2222/octosql/physical" 8 | ) 9 | 10 | type Filter struct { 11 | predicate Expression 12 | source Node 13 | } 14 | 15 | func NewFilter(predicate Expression, child Node) *Filter { 16 | return &Filter{predicate: predicate, source: child} 17 | } 18 | 19 | func (node *Filter) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 20 | source, mapping := node.source.Typecheck(ctx, env, logicalEnv) 21 | predicate := TypecheckExpression( 22 | ctx, 23 | env.WithRecordSchema(source.Schema), 24 | logicalEnv.WithRecordUniqueVariableNames(mapping), 25 | octosql.TypeSum(octosql.Boolean, octosql.Null), 26 | node.predicate, 27 | ) 28 | 29 | return physical.Node{ 30 | Schema: source.Schema, 31 | NodeType: physical.NodeTypeFilter, 32 | Filter: &physical.Filter{ 33 | Source: source, 34 | Predicate: predicate, 35 | }, 36 | }, mapping 37 | } 38 | -------------------------------------------------------------------------------- /logical/limit.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | 7 | "github.com/cube2222/octosql/octosql" 8 | "github.com/cube2222/octosql/physical" 9 | ) 10 | 11 | type OrderDirection string 12 | 13 | func DirectionsToMultipliers(directions []OrderDirection) []int { 14 | directionMultipliers := make([]int, len(directions)) 15 | for i, direction := range directions { 16 | switch direction { 17 | case "asc": 18 | directionMultipliers[i] = 1 19 | case "desc": 20 | directionMultipliers[i] = -1 21 | default: 22 | panic(fmt.Errorf("invalid order by direction: %s", direction)) 23 | } 24 | } 25 | 26 | return directionMultipliers 27 | } 28 | 29 | type OrderSensitiveTransform struct { 30 | source Node 31 | orderByKeyExprs []Expression 32 | orderByDirections []OrderDirection 33 | limit *Expression 34 | } 35 | 36 | func NewOrderSensitiveTransform(keyExprs []Expression, directions []OrderDirection, limit *Expression, source Node) *OrderSensitiveTransform { 37 | return &OrderSensitiveTransform{ 38 | orderByKeyExprs: keyExprs, 39 | orderByDirections: directions, 40 | limit: limit, 41 | source: source, 42 | } 43 | } 44 | 45 | func (node *OrderSensitiveTransform) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 46 | source, mapping := node.source.Typecheck(ctx, env, logicalEnv) 47 | 48 | orderByKeyExprs := make([]physical.Expression, len(node.orderByKeyExprs)) 49 | for i := range node.orderByKeyExprs { 50 | orderByKeyExprs[i] = node.orderByKeyExprs[i].Typecheck(ctx, env.WithRecordSchema(source.Schema), logicalEnv.WithRecordUniqueVariableNames(mapping)) 51 | } 52 | 53 | orderByDirectionMultipliers := DirectionsToMultipliers(node.orderByDirections) 54 | 55 | var limit *physical.Expression 56 | if node.limit != nil { 57 | expr := TypecheckExpression(ctx, env, logicalEnv, octosql.Int, *node.limit) 58 | limit = &expr 59 | } 60 | 61 | return physical.Node{ 62 | Schema: physical.NewSchema(source.Schema.Fields, source.Schema.TimeField, physical.WithNoRetractions(true)), 63 | NodeType: physical.NodeTypeOrderSensitiveTransform, 64 | OrderSensitiveTransform: &physical.OrderSensitiveTransform{ 65 | Source: source, 66 | OrderByKey: orderByKeyExprs, 67 | OrderByDirectionMultipliers: orderByDirectionMultipliers, 68 | Limit: limit, 69 | }, 70 | }, mapping 71 | } 72 | -------------------------------------------------------------------------------- /logical/requalifier.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "regexp" 7 | "strings" 8 | 9 | "github.com/cube2222/octosql/physical" 10 | ) 11 | 12 | type Requalifier struct { 13 | qualifier string 14 | source Node 15 | } 16 | 17 | func NewRequalifier(qualifier string, child Node) *Requalifier { 18 | return &Requalifier{qualifier: qualifier, source: child} 19 | } 20 | 21 | var qualifiedNameRegexp = regexp.MustCompile(`^[^.]+\..+$`) 22 | 23 | func (node *Requalifier) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 24 | source, mapping := node.source.Typecheck(ctx, env, logicalEnv) 25 | 26 | outMapping := make(map[string]string) 27 | for name, unique := range mapping { 28 | if qualifiedNameRegexp.MatchString(name) { 29 | dotIndex := strings.Index(name, ".") 30 | name = fmt.Sprintf("%s.%s", node.qualifier, name[dotIndex+1:]) 31 | } else { 32 | name = fmt.Sprintf("%s.%s", node.qualifier, name) 33 | } 34 | outMapping[name] = unique 35 | } 36 | 37 | return source, outMapping 38 | } 39 | -------------------------------------------------------------------------------- /logical/union_all.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | type UnionAll struct { 10 | first, second Node 11 | } 12 | 13 | func NewUnionAll(first, second Node) *UnionAll { 14 | return &UnionAll{first: first, second: second} 15 | } 16 | 17 | func (node *UnionAll) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 18 | panic("implement me") 19 | } 20 | -------------------------------------------------------------------------------- /logical/union_distinct.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | type UnionDistinct struct { 10 | first, second Node 11 | } 12 | 13 | func NewUnionDistinct(first, second Node) *UnionDistinct { 14 | return &UnionDistinct{first: first, second: second} 15 | } 16 | 17 | func (node *UnionDistinct) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 18 | panic("implement me") 19 | // return NewDistinct(NewUnionAll(node.first, node.second)).Physical(ctx, physicalCreator) 20 | } 21 | -------------------------------------------------------------------------------- /logical/with.go: -------------------------------------------------------------------------------- 1 | package logical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | type With struct { 10 | cteNames []string 11 | cteNodes []Node 12 | source Node 13 | } 14 | 15 | func NewWith(cteNames []string, cteNodes []Node, source Node) *With { 16 | return &With{ 17 | cteNames: cteNames, 18 | cteNodes: cteNodes, 19 | source: source, 20 | } 21 | } 22 | 23 | func (node *With) Typecheck(ctx context.Context, env physical.Environment, logicalEnv Environment) (physical.Node, map[string]string) { 24 | newCTEs := make(map[string]CommonTableExpression) 25 | for k, v := range logicalEnv.CommonTableExpressions { 26 | newCTEs[k] = v 27 | } 28 | 29 | for i := range node.cteNodes { 30 | cte, mapping := node.cteNodes[i].Typecheck(ctx, env, Environment{ 31 | CommonTableExpressions: newCTEs, 32 | TableValuedFunctions: logicalEnv.TableValuedFunctions, 33 | UniqueVariableNames: logicalEnv.UniqueVariableNames, 34 | UniqueNameGenerator: logicalEnv.UniqueNameGenerator, 35 | }) 36 | newCTEs[node.cteNames[i]] = CommonTableExpression{ 37 | Node: cte, 38 | UniqueVariableMapping: mapping, 39 | } 40 | } 41 | 42 | return node.source.Typecheck(ctx, env, Environment{ 43 | CommonTableExpressions: newCTEs, 44 | TableValuedFunctions: logicalEnv.TableValuedFunctions, 45 | UniqueVariableNames: logicalEnv.UniqueVariableNames, 46 | UniqueNameGenerator: logicalEnv.UniqueNameGenerator, 47 | }) 48 | } 49 | -------------------------------------------------------------------------------- /logs/logs.go: -------------------------------------------------------------------------------- 1 | package logs 2 | 3 | import ( 4 | "log" 5 | "os" 6 | "path/filepath" 7 | 8 | "github.com/cube2222/octosql/config" 9 | ) 10 | 11 | var Output *os.File 12 | 13 | func InitializeFileLogger() { 14 | path := filepath.Join(config.OctosqlCacheDir, "logs.txt") 15 | f, err := os.Create(path) 16 | if err != nil { 17 | log.Fatalf("couldn't create logs file: %s", err) 18 | } 19 | Output = f 20 | log.SetOutput(Output) 21 | } 22 | 23 | func CloseLogger() { 24 | Output.Close() 25 | } 26 | -------------------------------------------------------------------------------- /main.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "os" 7 | "os/signal" 8 | 9 | "github.com/cube2222/octosql/cmd" 10 | ) 11 | 12 | func main() { 13 | ctx, cancel := context.WithCancel(context.Background()) 14 | signals := make(chan os.Signal, 1) 15 | signal.Notify(signals, os.Interrupt) 16 | go func() { 17 | <-signals 18 | cancel() 19 | <-signals 20 | fmt.Println("Force stopped without cleanup.") 21 | os.Exit(1) 22 | }() 23 | 24 | cmd.Execute(ctx) 25 | } 26 | -------------------------------------------------------------------------------- /octosql/types_test.go: -------------------------------------------------------------------------------- 1 | package octosql 2 | 3 | import ( 4 | "fmt" 5 | "reflect" 6 | "testing" 7 | ) 8 | 9 | func TestTypeIntersection(t *testing.T) { 10 | some := func(t Type) *Type { 11 | return &t 12 | } 13 | 14 | tests := []struct { 15 | t1 Type 16 | t2 Type 17 | want *Type 18 | }{ 19 | { 20 | t1: String, 21 | t2: String, 22 | want: some(String), 23 | }, 24 | { 25 | t1: Int, 26 | t2: String, 27 | want: nil, 28 | }, 29 | { 30 | t1: TypeSum(Boolean, Time), 31 | t2: TypeSum(Time, Int), 32 | want: some(Time), 33 | }, 34 | { 35 | t1: TypeSum(TypeSum(Boolean, Time), String), 36 | t2: TypeSum(TypeSum(Time, Int), String), 37 | want: some(TypeSum(String, Time)), 38 | }, 39 | { 40 | t1: TypeSum(Boolean, Time), 41 | t2: TypeSum(String, Int), 42 | want: nil, 43 | }, 44 | } 45 | for i, tt := range tests { 46 | t.Run(fmt.Sprint(i), func(t *testing.T) { 47 | if got := TypeIntersection(tt.t1, tt.t2); !reflect.DeepEqual(got, tt.want) { 48 | t.Errorf("TypeIntersection(%s, %s) = %s, want %s", tt.t1, tt.t2, got, tt.want) 49 | } 50 | }) 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /optimizer/filter_datasource_pushdown.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | "github.com/cube2222/octosql/octosql" 5 | . "github.com/cube2222/octosql/physical" 6 | ) 7 | 8 | func PushDownFilterPredicatesToDatasource(node Node) (Node, bool) { 9 | changed := false 10 | t := Transformers{ 11 | NodeTransformer: func(node Node) Node { 12 | if node.NodeType != NodeTypeFilter { 13 | return node 14 | } 15 | if node.Filter.Source.NodeType != NodeTypeDatasource { 16 | return node 17 | } 18 | 19 | filterPredicates := node.Filter.Predicate.SplitByAnd() 20 | alreadyPushedDown := node.Filter.Source.Datasource.Predicates 21 | 22 | newFilterPredicates, newPushedDownPredicates, curChanged := node.Filter.Source.Datasource.PushDownPredicates(filterPredicates, alreadyPushedDown) 23 | if !curChanged { 24 | return node 25 | } 26 | changed = true 27 | 28 | out := Node{ 29 | Schema: node.Filter.Source.Schema, 30 | NodeType: NodeTypeDatasource, 31 | Datasource: &Datasource{ 32 | Name: node.Filter.Source.Datasource.Name, 33 | Alias: node.Filter.Source.Datasource.Alias, 34 | DatasourceImplementation: node.Filter.Source.Datasource.DatasourceImplementation, 35 | Predicates: newPushedDownPredicates, 36 | VariableMapping: node.Filter.Source.Datasource.VariableMapping, 37 | }, 38 | } 39 | if len(newFilterPredicates) > 0 { 40 | out = Node{ 41 | Schema: node.Schema, 42 | NodeType: NodeTypeFilter, 43 | Filter: &Filter{ 44 | Predicate: Expression{ 45 | Type: octosql.Boolean, 46 | ExpressionType: ExpressionTypeAnd, 47 | And: &And{ 48 | Arguments: newFilterPredicates, 49 | }, 50 | }, 51 | Source: out, 52 | }, 53 | } 54 | } 55 | 56 | return out 57 | }, 58 | } 59 | output := t.TransformNode(node) 60 | 61 | if changed { 62 | return output, true 63 | } else { 64 | return node, false 65 | } 66 | } 67 | -------------------------------------------------------------------------------- /optimizer/filter_merge.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | "github.com/cube2222/octosql/octosql" 5 | . "github.com/cube2222/octosql/physical" 6 | ) 7 | 8 | func MergeFilters(node Node) (Node, bool) { 9 | changed := false 10 | t := Transformers{ 11 | NodeTransformer: func(node Node) Node { 12 | if node.NodeType != NodeTypeFilter { 13 | return node 14 | } 15 | if node.Filter.Source.NodeType != NodeTypeFilter { 16 | return node 17 | } 18 | changed = true 19 | 20 | return Node{ 21 | Schema: node.Filter.Source.Schema, 22 | NodeType: NodeTypeFilter, 23 | Filter: &Filter{ 24 | Predicate: Expression{ 25 | Type: octosql.TypeSum(node.Filter.Predicate.Type, node.Filter.Source.Filter.Predicate.Type), 26 | ExpressionType: ExpressionTypeAnd, 27 | And: &And{ 28 | Arguments: append(node.Filter.Predicate.SplitByAnd(), node.Filter.Source.Filter.Predicate.SplitByAnd()...), 29 | }, 30 | }, 31 | Source: node.Filter.Source.Filter.Source, 32 | }, 33 | } 34 | }, 35 | } 36 | output := t.TransformNode(node) 37 | 38 | if changed { 39 | return output, true 40 | } else { 41 | return node, false 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /optimizer/optimize.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | . "github.com/cube2222/octosql/physical" 5 | ) 6 | 7 | var defaultOptimizationRules = []func(Node) (output Node, changed bool){ 8 | PushDownFilterPredicatesToDatasource, 9 | PushDownFilterPredicatesIntoLookupJoinBranch, 10 | PushDownFilterPredicatesIntoStreamJoinBranch, 11 | PushDownFilterPredicatesIntoStreamJoinKey, 12 | RemoveUnusedMapFields, 13 | RemoveUnusedGroupByNonKeyFields, 14 | RemoveUnusedDatasourceFields, 15 | MergeFilters, 16 | } 17 | 18 | func Optimize(node Node) Node { 19 | // TODO: We could actually get the value of 'changed' by diffing the tree after each round of optimizations, instead of pushing that burden onto the optimization rules. 20 | changed := true 21 | i := 0 22 | for changed { 23 | i++ 24 | changed = false 25 | for _, rule := range defaultOptimizationRules { 26 | output, curChanged := rule(node) 27 | if curChanged { 28 | changed = true 29 | node = output 30 | } 31 | } 32 | } 33 | return node 34 | } 35 | -------------------------------------------------------------------------------- /optimizer/push_filter_into_lookup_join_branch.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | "github.com/cube2222/octosql/octosql" 5 | . "github.com/cube2222/octosql/physical" 6 | ) 7 | 8 | func PushDownFilterPredicatesIntoLookupJoinBranch(node Node) (Node, bool) { 9 | changed := false 10 | t := Transformers{ 11 | NodeTransformer: func(node Node) Node { 12 | if node.NodeType != NodeTypeFilter { 13 | return node 14 | } 15 | if node.Filter.Source.NodeType != NodeTypeLookupJoin { 16 | return node 17 | } 18 | changed = true 19 | 20 | sourceSchema := node.Filter.Source.LookupJoin.Source.Schema 21 | joinedSchema := node.Filter.Source.LookupJoin.Joined.Schema 22 | 23 | filterPredicates := node.Filter.Predicate.SplitByAnd() 24 | var pushedDownSource, pushedDownJoined []Expression 25 | 26 | for i := range filterPredicates { 27 | variablesUsed := filterPredicates[i].VariablesUsed() 28 | if !UsesVariablesFromSchema(joinedSchema, variablesUsed) { 29 | pushedDownSource = append(pushedDownSource, filterPredicates[i]) 30 | } else { 31 | pushedDownJoined = append(pushedDownJoined, filterPredicates[i]) 32 | } 33 | } 34 | 35 | joinSourceSource := node.Filter.Source.LookupJoin.Source 36 | if len(pushedDownSource) > 0 { 37 | joinSourceSource = Node{ 38 | Schema: joinSourceSource.Schema, 39 | NodeType: NodeTypeFilter, 40 | Filter: &Filter{ 41 | Source: joinSourceSource, 42 | Predicate: Expression{ 43 | Type: octosql.Boolean, 44 | ExpressionType: ExpressionTypeAnd, 45 | And: &And{ 46 | Arguments: pushedDownSource, 47 | }, 48 | }, 49 | }, 50 | } 51 | } 52 | joinSourceJoined := node.Filter.Source.LookupJoin.Joined 53 | if len(pushedDownJoined) > 0 { 54 | joinSourceJoined = Node{ 55 | Schema: joinSourceJoined.Schema, 56 | NodeType: NodeTypeFilter, 57 | Filter: &Filter{ 58 | Source: joinSourceJoined, 59 | Predicate: transformVariablesFromSchemaIntoNonLevel0( 60 | sourceSchema, 61 | Expression{ 62 | Type: octosql.Boolean, 63 | ExpressionType: ExpressionTypeAnd, 64 | And: &And{ 65 | Arguments: pushedDownJoined, 66 | }, 67 | }, 68 | ), 69 | }, 70 | } 71 | } 72 | 73 | out := Node{ 74 | Schema: node.Filter.Source.Schema, 75 | NodeType: NodeTypeLookupJoin, 76 | LookupJoin: &LookupJoin{ 77 | Source: joinSourceSource, 78 | Joined: joinSourceJoined, 79 | }, 80 | } 81 | 82 | return out 83 | }, 84 | } 85 | output := t.TransformNode(node) 86 | 87 | if changed { 88 | return output, true 89 | } else { 90 | return node, false 91 | } 92 | } 93 | 94 | func transformVariablesFromSchemaIntoNonLevel0(schema Schema, expr Expression) Expression { 95 | t := Transformers{ 96 | ExpressionTransformer: func(expr Expression) Expression { 97 | if expr.ExpressionType != ExpressionTypeVariable { 98 | return expr 99 | } 100 | for _, field := range schema.Fields { 101 | if field.Name == expr.Variable.Name { 102 | expr.Variable.IsLevel0 = false 103 | return expr 104 | } 105 | } 106 | return expr 107 | }, 108 | } 109 | return t.TransformExpr(expr) 110 | } 111 | -------------------------------------------------------------------------------- /optimizer/push_filter_into_stream_join_branch.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | "github.com/cube2222/octosql/octosql" 5 | . "github.com/cube2222/octosql/physical" 6 | ) 7 | 8 | func PushDownFilterPredicatesIntoStreamJoinBranch(node Node) (Node, bool) { 9 | changed := false 10 | t := Transformers{ 11 | NodeTransformer: func(node Node) Node { 12 | if node.NodeType != NodeTypeFilter { 13 | return node 14 | } 15 | if node.Filter.Source.NodeType != NodeTypeStreamJoin { 16 | return node 17 | } 18 | leftSchema := node.Filter.Source.StreamJoin.Left.Schema 19 | rightSchema := node.Filter.Source.StreamJoin.Right.Schema 20 | 21 | filterPredicates := node.Filter.Predicate.SplitByAnd() 22 | var stayedAbove, pushedDownLeft, pushedDownRight []Expression 23 | 24 | for i := range filterPredicates { 25 | variablesUsed := filterPredicates[i].VariablesUsed() 26 | // Clarification: 27 | // If it doesn't use variables from any join branch, 28 | // then it gets pushed down into both. 29 | usesLeftBranch := UsesVariablesFromSchema(leftSchema, variablesUsed) 30 | usesRightBranch := UsesVariablesFromSchema(rightSchema, variablesUsed) 31 | if !usesLeftBranch { 32 | pushedDownRight = append(pushedDownRight, filterPredicates[i]) 33 | } 34 | if !usesRightBranch { 35 | pushedDownLeft = append(pushedDownLeft, filterPredicates[i]) 36 | } 37 | if usesLeftBranch && usesRightBranch { 38 | stayedAbove = append(stayedAbove, filterPredicates[i]) 39 | } 40 | } 41 | 42 | if len(stayedAbove) == len(filterPredicates) { 43 | return node 44 | } 45 | changed = true 46 | 47 | joinSourceLeft := node.Filter.Source.StreamJoin.Left 48 | if len(pushedDownLeft) > 0 { 49 | joinSourceLeft = Node{ 50 | Schema: joinSourceLeft.Schema, 51 | NodeType: NodeTypeFilter, 52 | Filter: &Filter{ 53 | Source: joinSourceLeft, 54 | Predicate: Expression{ 55 | Type: octosql.Boolean, 56 | ExpressionType: ExpressionTypeAnd, 57 | And: &And{ 58 | Arguments: pushedDownLeft, 59 | }, 60 | }, 61 | }, 62 | } 63 | } 64 | joinSourceRight := node.Filter.Source.StreamJoin.Right 65 | if len(pushedDownRight) > 0 { 66 | joinSourceRight = Node{ 67 | Schema: joinSourceRight.Schema, 68 | NodeType: NodeTypeFilter, 69 | Filter: &Filter{ 70 | Source: joinSourceRight, 71 | Predicate: Expression{ 72 | Type: octosql.Boolean, 73 | ExpressionType: ExpressionTypeAnd, 74 | And: &And{ 75 | Arguments: pushedDownRight, 76 | }, 77 | }, 78 | }, 79 | } 80 | } 81 | 82 | out := Node{ 83 | Schema: node.Filter.Source.Schema, 84 | NodeType: NodeTypeStreamJoin, 85 | StreamJoin: &StreamJoin{ 86 | LeftKey: node.Filter.Source.StreamJoin.LeftKey, 87 | RightKey: node.Filter.Source.StreamJoin.RightKey, 88 | Left: joinSourceLeft, 89 | Right: joinSourceRight, 90 | }, 91 | } 92 | if len(stayedAbove) > 0 { 93 | out = Node{ 94 | Schema: out.Schema, 95 | NodeType: NodeTypeFilter, 96 | Filter: &Filter{ 97 | Predicate: Expression{ 98 | Type: octosql.Boolean, 99 | ExpressionType: ExpressionTypeAnd, 100 | And: &And{ 101 | Arguments: stayedAbove, 102 | }, 103 | }, 104 | Source: out, 105 | }, 106 | } 107 | } 108 | 109 | return out 110 | }, 111 | } 112 | output := t.TransformNode(node) 113 | 114 | if changed { 115 | return output, true 116 | } else { 117 | return node, false 118 | } 119 | } 120 | 121 | func UsesVariablesFromSchema(schema Schema, variables []string) bool { 122 | for _, name := range variables { 123 | for _, field := range schema.Fields { 124 | if VariableNameMatchesField(name, field.Name) { 125 | return true 126 | } 127 | } 128 | } 129 | return false 130 | } 131 | -------------------------------------------------------------------------------- /optimizer/push_filter_into_stream_join_key.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | "github.com/cube2222/octosql/octosql" 5 | . "github.com/cube2222/octosql/physical" 6 | ) 7 | 8 | func PushDownFilterPredicatesIntoStreamJoinKey(node Node) (Node, bool) { 9 | changed := false 10 | t := Transformers{ 11 | NodeTransformer: func(node Node) Node { 12 | if node.NodeType != NodeTypeFilter { 13 | return node 14 | } 15 | if node.Filter.Source.NodeType != NodeTypeStreamJoin { 16 | return node 17 | } 18 | leftSchema := node.Filter.Source.StreamJoin.Left.Schema 19 | rightSchema := node.Filter.Source.StreamJoin.Right.Schema 20 | 21 | filterPredicates := node.Filter.Predicate.SplitByAnd() 22 | var stayedAbove, leftKeyAdd, rightKeyAdd []Expression 23 | 24 | for i := range filterPredicates { 25 | if filterPredicates[i].ExpressionType != ExpressionTypeFunctionCall { 26 | stayedAbove = append(stayedAbove, filterPredicates[i]) 27 | continue 28 | } 29 | if filterPredicates[i].FunctionCall.Name != "=" { 30 | stayedAbove = append(stayedAbove, filterPredicates[i]) 31 | continue 32 | } 33 | firstPart := filterPredicates[i].FunctionCall.Arguments[0] 34 | secondPart := filterPredicates[i].FunctionCall.Arguments[1] 35 | firstPartVariables := firstPart.VariablesUsed() 36 | firstPartUsesLeftVariables := UsesVariablesFromSchema(leftSchema, firstPartVariables) 37 | firstPartUsesRightVariables := UsesVariablesFromSchema(rightSchema, firstPartVariables) 38 | secondPartVariables := secondPart.VariablesUsed() 39 | secondPartUsesLeftVariables := UsesVariablesFromSchema(leftSchema, secondPartVariables) 40 | secondPartUsesRightVariables := UsesVariablesFromSchema(rightSchema, secondPartVariables) 41 | 42 | if firstPartUsesLeftVariables && !firstPartUsesRightVariables && 43 | !secondPartUsesLeftVariables && secondPartUsesRightVariables { 44 | leftKeyAdd = append(leftKeyAdd, firstPart) 45 | rightKeyAdd = append(rightKeyAdd, secondPart) 46 | } else if !firstPartUsesLeftVariables && firstPartUsesRightVariables && 47 | secondPartUsesLeftVariables && !secondPartUsesRightVariables { 48 | rightKeyAdd = append(rightKeyAdd, firstPart) 49 | leftKeyAdd = append(leftKeyAdd, secondPart) 50 | } else { 51 | stayedAbove = append(stayedAbove, filterPredicates[i]) 52 | } 53 | } 54 | 55 | if len(stayedAbove) == len(filterPredicates) { 56 | return node 57 | } 58 | changed = true 59 | 60 | out := Node{ 61 | Schema: node.Filter.Source.Schema, 62 | NodeType: NodeTypeStreamJoin, 63 | StreamJoin: &StreamJoin{ 64 | LeftKey: append(node.Filter.Source.StreamJoin.LeftKey, leftKeyAdd...), 65 | RightKey: append(node.Filter.Source.StreamJoin.RightKey, rightKeyAdd...), 66 | Left: node.Filter.Source.StreamJoin.Left, 67 | Right: node.Filter.Source.StreamJoin.Right, 68 | }, 69 | } 70 | if len(stayedAbove) > 0 { 71 | out = Node{ 72 | Schema: out.Schema, 73 | NodeType: NodeTypeFilter, 74 | Filter: &Filter{ 75 | Predicate: Expression{ 76 | Type: octosql.Boolean, 77 | ExpressionType: ExpressionTypeAnd, 78 | And: &And{ 79 | Arguments: stayedAbove, 80 | }, 81 | }, 82 | Source: out, 83 | }, 84 | } 85 | } 86 | 87 | return out 88 | }, 89 | } 90 | output := t.TransformNode(node) 91 | 92 | if changed { 93 | return output, true 94 | } else { 95 | return node, false 96 | } 97 | } 98 | -------------------------------------------------------------------------------- /optimizer/remove_unused_datasource_fields.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | . "github.com/cube2222/octosql/physical" 5 | ) 6 | 7 | func RemoveUnusedDatasourceFields(node Node) (Node, bool) { 8 | changed := false 9 | 10 | fields := getNonTimeFieldDatasourceFields(node) 11 | for i := range fields { 12 | if !isUsed(fields[i], node) { 13 | node = removeUnusedDatasourceField(fields[i], node) 14 | node = removeFieldFromPassers(fields[i], node) 15 | changed = true 16 | } 17 | } 18 | 19 | return node, changed 20 | } 21 | 22 | func getNonTimeFieldDatasourceFields(node Node) []string { 23 | fields := make([]string, 0) 24 | fieldCollector := Transformers{ 25 | NodeTransformer: func(node Node) Node { 26 | if node.NodeType != NodeTypeDatasource { 27 | return node 28 | } 29 | for i, field := range node.Schema.Fields { 30 | if i == node.Schema.TimeField { 31 | continue 32 | } 33 | fields = append(fields, field.Name) 34 | } 35 | 36 | return node 37 | }, 38 | } 39 | fieldCollector.TransformNode(node) 40 | 41 | return fields 42 | } 43 | 44 | func removeUnusedDatasourceField(field string, node Node) Node { 45 | fieldRemover := Transformers{ 46 | NodeTransformer: func(node Node) Node { 47 | if node.NodeType != NodeTypeDatasource { 48 | return node 49 | } 50 | index := -1 51 | for i := range node.Schema.Fields { 52 | if node.Schema.Fields[i].Name == field { 53 | index = i 54 | } 55 | } 56 | if index == -1 { 57 | return node 58 | } 59 | 60 | node.Schema.Fields = append(node.Schema.Fields[:index], node.Schema.Fields[index+1:]...) 61 | if node.Schema.TimeField > index { 62 | node.Schema.TimeField-- 63 | } 64 | 65 | return node 66 | }, 67 | } 68 | return fieldRemover.TransformNode(node) 69 | } 70 | -------------------------------------------------------------------------------- /optimizer/remove_unused_groupby_fields.go: -------------------------------------------------------------------------------- 1 | package optimizer 2 | 3 | import ( 4 | . "github.com/cube2222/octosql/physical" 5 | ) 6 | 7 | func RemoveUnusedGroupByNonKeyFields(node Node) (Node, bool) { 8 | changed := false 9 | 10 | fields := getNonTimeFieldGroupByFields(node) 11 | for i := range fields { 12 | if !isUsed(fields[i], node) { 13 | node = removeGroupByField(fields[i], node) 14 | node = removeFieldFromPassers(fields[i], node) 15 | changed = true 16 | } 17 | } 18 | 19 | return node, changed 20 | } 21 | 22 | func getNonTimeFieldGroupByFields(node Node) []string { 23 | fields := make([]string, 0) 24 | fieldCollector := Transformers{ 25 | NodeTransformer: func(node Node) Node { 26 | if node.NodeType != NodeTypeGroupBy { 27 | return node 28 | } 29 | for i, field := range node.Schema.Fields { 30 | if i < len(node.GroupBy.Key) { 31 | continue 32 | } 33 | if i == node.Schema.TimeField { 34 | continue 35 | } 36 | fields = append(fields, field.Name) 37 | } 38 | 39 | return node 40 | }, 41 | } 42 | fieldCollector.TransformNode(node) 43 | 44 | return fields 45 | } 46 | 47 | func removeGroupByField(field string, node Node) Node { 48 | fieldRemover := Transformers{ 49 | NodeTransformer: func(node Node) Node { 50 | if node.NodeType != NodeTypeGroupBy { 51 | return node 52 | } 53 | index := -1 54 | for i := range node.Schema.Fields { 55 | if node.Schema.Fields[i].Name == field { 56 | index = i 57 | } 58 | } 59 | if index == -1 { 60 | return node 61 | } 62 | 63 | aggregateIndex := index - len(node.GroupBy.Key) 64 | 65 | node.Schema.Fields = append(node.Schema.Fields[:index], node.Schema.Fields[index+1:]...) 66 | node.GroupBy.AggregateExpressions = append(node.GroupBy.AggregateExpressions[:aggregateIndex], node.GroupBy.AggregateExpressions[aggregateIndex+1:]...) 67 | node.GroupBy.Aggregates = append(node.GroupBy.Aggregates[:aggregateIndex], node.GroupBy.Aggregates[aggregateIndex+1:]...) 68 | if node.Schema.TimeField > index { 69 | node.Schema.TimeField-- 70 | } 71 | 72 | return node 73 | }, 74 | } 75 | return fieldRemover.TransformNode(node) 76 | } 77 | -------------------------------------------------------------------------------- /outputs/eager/eager.go: -------------------------------------------------------------------------------- 1 | package eager 2 | 3 | import ( 4 | "bufio" 5 | "io" 6 | "os" 7 | 8 | . "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/octosql" 10 | "github.com/cube2222/octosql/physical" 11 | ) 12 | 13 | type Format interface { 14 | SetSchema(physical.Schema) 15 | Write([]octosql.Value) error 16 | Close() error 17 | } 18 | 19 | type OutputPrinter struct { 20 | source Node 21 | 22 | schema physical.Schema 23 | format func(io.Writer) Format 24 | } 25 | 26 | func NewOutputPrinter(source Node, schema physical.Schema, format func(io.Writer) Format) *OutputPrinter { 27 | return &OutputPrinter{ 28 | source: source, 29 | schema: schema, 30 | format: format, 31 | } 32 | } 33 | 34 | func (o *OutputPrinter) Run(execCtx ExecutionContext) error { 35 | w := bufio.NewWriterSize(os.Stdout, 4096*1024) 36 | format := o.format(w) 37 | format.SetSchema(o.schema) 38 | 39 | if err := o.source.Run( 40 | execCtx, 41 | func(ctx ProduceContext, record Record) error { 42 | return format.Write(record.Values) 43 | }, 44 | func(ctx ProduceContext, msg MetadataMessage) error { 45 | return nil 46 | }, 47 | ); err != nil { 48 | return err 49 | } 50 | 51 | return w.Flush() 52 | } 53 | -------------------------------------------------------------------------------- /outputs/formats/csv_format.go: -------------------------------------------------------------------------------- 1 | package formats 2 | 3 | import ( 4 | "encoding/csv" 5 | "fmt" 6 | "io" 7 | "strconv" 8 | "strings" 9 | "time" 10 | 11 | "github.com/cube2222/octosql/octosql" 12 | "github.com/cube2222/octosql/physical" 13 | ) 14 | 15 | type CSVFormatter struct { 16 | writer *csv.Writer 17 | fields []physical.SchemaField 18 | } 19 | 20 | func NewCSVFormatter(w io.Writer) *CSVFormatter { 21 | writer := csv.NewWriter(w) 22 | 23 | return &CSVFormatter{ 24 | writer: writer, 25 | } 26 | } 27 | 28 | func (t *CSVFormatter) SetSchema(schema physical.Schema) { 29 | t.fields = WithoutQualifiers(schema.Fields) 30 | 31 | header := make([]string, len(t.fields)) 32 | for i := range t.fields { 33 | header[i] = t.fields[i].Name 34 | } 35 | t.writer.Write(header) 36 | } 37 | 38 | func (t *CSVFormatter) Write(values []octosql.Value) error { 39 | var builder strings.Builder 40 | row := make([]string, len(values)) 41 | for i := range values { 42 | FormatCSVValue(&builder, values[i]) 43 | row[i] = builder.String() 44 | builder.Reset() 45 | } 46 | return t.writer.Write(row) 47 | } 48 | 49 | func FormatCSVValue(builder *strings.Builder, value octosql.Value) { 50 | switch value.TypeID { 51 | case octosql.TypeIDNull: 52 | case octosql.TypeIDInt: 53 | builder.WriteString(strconv.FormatInt(int64(value.Int), 10)) 54 | case octosql.TypeIDFloat: 55 | builder.WriteString(strconv.FormatFloat(value.Float, 'f', -1, 64)) 56 | case octosql.TypeIDBoolean: 57 | builder.WriteString(strconv.FormatBool(value.Boolean)) 58 | case octosql.TypeIDString: 59 | builder.WriteString(value.Str) 60 | case octosql.TypeIDTime: 61 | builder.WriteString(value.Time.Format(time.RFC3339)) 62 | case octosql.TypeIDDuration: 63 | builder.WriteString(fmt.Sprint(value.Duration)) 64 | default: 65 | panic("invalid value type to print in CSV: " + value.TypeID.String()) 66 | } 67 | } 68 | 69 | func (t *CSVFormatter) Close() error { 70 | t.writer.Flush() 71 | return nil 72 | } 73 | -------------------------------------------------------------------------------- /outputs/formats/human_readable_schema.go: -------------------------------------------------------------------------------- 1 | package formats 2 | 3 | import ( 4 | "strings" 5 | 6 | "github.com/cube2222/octosql/physical" 7 | ) 8 | 9 | func WithoutQualifiers(fields []physical.SchemaField) []physical.SchemaField { 10 | shortName := func(name string) string { 11 | if strings.Contains(name, ".") { 12 | name = strings.SplitN(name, ".", 2)[1] 13 | } 14 | return name 15 | } 16 | 17 | nameCount := map[string]int{} 18 | for i := range fields { 19 | nameCount[shortName(fields[i].Name)]++ 20 | } 21 | 22 | outFields := make([]physical.SchemaField, len(fields)) 23 | for i := range fields { 24 | name := fields[i].Name 25 | if short := shortName(fields[i].Name); nameCount[short] == 1 { 26 | name = short 27 | } 28 | outFields[i] = physical.SchemaField{ 29 | Name: name, 30 | Type: fields[i].Type, 31 | } 32 | } 33 | return outFields 34 | } 35 | -------------------------------------------------------------------------------- /outputs/formats/json_format.go: -------------------------------------------------------------------------------- 1 | package formats 2 | 3 | import ( 4 | "fmt" 5 | "io" 6 | "log" 7 | "time" 8 | 9 | "github.com/valyala/fastjson" 10 | 11 | "github.com/cube2222/octosql/octosql" 12 | "github.com/cube2222/octosql/physical" 13 | ) 14 | 15 | type JSONFormatter struct { 16 | buf []byte 17 | arena *fastjson.Arena 18 | w io.Writer 19 | fields []physical.SchemaField 20 | } 21 | 22 | func NewJSONFormatter(w io.Writer) *JSONFormatter { 23 | return &JSONFormatter{ 24 | buf: make([]byte, 0, 1024), 25 | arena: new(fastjson.Arena), 26 | w: w, 27 | } 28 | } 29 | 30 | func (t *JSONFormatter) SetSchema(schema physical.Schema) { 31 | t.fields = WithoutQualifiers(schema.Fields) 32 | } 33 | 34 | func (t *JSONFormatter) Write(values []octosql.Value) error { 35 | obj := t.arena.NewObject() 36 | for i := range t.fields { 37 | obj.Set(t.fields[i].Name, ValueToJson(t.arena, t.fields[i].Type, values[i])) 38 | } 39 | 40 | t.buf = obj.MarshalTo(t.buf) 41 | t.buf = append(t.buf, '\n') 42 | t.w.Write(t.buf) 43 | t.buf = t.buf[:0] 44 | t.arena.Reset() 45 | return nil 46 | } 47 | 48 | func ValueToJson(arena *fastjson.Arena, t octosql.Type, value octosql.Value) *fastjson.Value { 49 | if t.TypeID == octosql.TypeIDUnion { 50 | for i := range t.Union.Alternatives { 51 | if t.Union.Alternatives[i].TypeID == value.TypeID { 52 | return ValueToJson(arena, t.Union.Alternatives[i], value) 53 | } 54 | } 55 | log.Printf("Invalid value of type '%s' for union type '%s'. Using null.", value.TypeID.String(), t.String()) 56 | return arena.NewNull() 57 | } 58 | 59 | switch value.TypeID { 60 | case octosql.TypeIDNull: 61 | return arena.NewNull() 62 | case octosql.TypeIDInt: 63 | return arena.NewNumberInt(int(value.Int)) 64 | case octosql.TypeIDFloat: 65 | return arena.NewNumberFloat64(value.Float) 66 | case octosql.TypeIDBoolean: 67 | if value.Boolean { 68 | return arena.NewTrue() 69 | } else { 70 | return arena.NewFalse() 71 | } 72 | case octosql.TypeIDString: 73 | return arena.NewString(value.Str) 74 | case octosql.TypeIDTime: 75 | return arena.NewString(value.Time.Format(time.RFC3339)) 76 | case octosql.TypeIDDuration: 77 | return arena.NewString(value.Duration.String()) 78 | case octosql.TypeIDList: 79 | arr := arena.NewArray() 80 | for i := range value.List { 81 | arr.SetArrayItem(i, ValueToJson(arena, *t.List.Element, value.List[i])) 82 | } 83 | return arr 84 | case octosql.TypeIDStruct: 85 | arr := arena.NewObject() 86 | for i := range value.Struct { 87 | arr.Set(t.Struct.Fields[i].Name, ValueToJson(arena, t.Struct.Fields[i].Type, value.Struct[i])) 88 | } 89 | return arr 90 | case octosql.TypeIDTuple: 91 | arr := arena.NewArray() 92 | for i := range value.Tuple { 93 | arr.SetArrayItem(i, ValueToJson(arena, t.Tuple.Elements[i], value.Tuple[i])) 94 | } 95 | return arr 96 | default: 97 | panic(fmt.Sprintf("invalid octosql value type to print: %s", value.TypeID.String())) 98 | } 99 | } 100 | 101 | func (t *JSONFormatter) Close() error { 102 | return nil 103 | } 104 | -------------------------------------------------------------------------------- /outputs/formats/table_format.go: -------------------------------------------------------------------------------- 1 | package formats 2 | 3 | import ( 4 | "io" 5 | 6 | "github.com/olekukonko/tablewriter" 7 | 8 | "github.com/cube2222/octosql/octosql" 9 | "github.com/cube2222/octosql/physical" 10 | ) 11 | 12 | type TableFormatter struct { 13 | table *tablewriter.Table 14 | } 15 | 16 | func NewTableFormatter(w io.Writer) *TableFormatter { 17 | table := tablewriter.NewWriter(w) 18 | table.SetColWidth(24) 19 | table.SetRowLine(false) 20 | 21 | return &TableFormatter{ 22 | table: table, 23 | } 24 | } 25 | 26 | func (t *TableFormatter) SetSchema(schema physical.Schema) { 27 | fields := WithoutQualifiers(schema.Fields) 28 | header := make([]string, len(fields)) 29 | for i := range fields { 30 | header[i] = fields[i].Name 31 | } 32 | t.table.SetHeader(header) 33 | t.table.SetAutoFormatHeaders(false) 34 | } 35 | 36 | func (t *TableFormatter) Write(values []octosql.Value) error { 37 | row := make([]string, len(values)) 38 | for i := range values { 39 | row[i] = values[i].String() 40 | } 41 | t.table.Append(row) 42 | return nil 43 | } 44 | 45 | func (t *TableFormatter) Close() error { 46 | t.table.Render() 47 | return nil 48 | } 49 | -------------------------------------------------------------------------------- /outputs/stream/internally_consistent_output_stream_wrapper.go: -------------------------------------------------------------------------------- 1 | package stream 2 | 3 | import ( 4 | "fmt" 5 | "time" 6 | 7 | . "github.com/cube2222/octosql/execution" 8 | ) 9 | 10 | type InternallyConsistentOutputStreamWrapper struct { 11 | Source Node 12 | } 13 | 14 | func (node *InternallyConsistentOutputStreamWrapper) Run(ctx ExecutionContext, produce ProduceFn, metaSend MetaSendFn) error { 15 | var pending []Record 16 | 17 | sendPendingLessOrEqualWatermark := func(ctx ProduceContext, watermark time.Time) error { 18 | crossedOut := make([]bool, len(pending)) 19 | 20 | afterWatermarkCount := 0 21 | for i := range pending { 22 | if pending[i].EventTime.After(watermark) { 23 | afterWatermarkCount++ 24 | } 25 | } 26 | newPending := make([]Record, afterWatermarkCount) 27 | for i := range pending { 28 | if pending[i].EventTime.After(watermark) { 29 | newPending = append(newPending, pending[i]) 30 | crossedOut[i] = true 31 | } 32 | } 33 | 34 | pendingLoop: 35 | for i := range pending { 36 | if crossedOut[i] { 37 | continue 38 | } 39 | if !pending[i].Retraction { 40 | // Look if there is a retraction for this record, this is beautifully quadratic. 41 | // TODO: Optimize. Use a sensible data structure. 42 | findRetractionLoop: 43 | for j := i + 1; j < len(pending); j++ { 44 | if !pending[j].Retraction { 45 | continue 46 | } 47 | for k := range pending[i].Values { 48 | if pending[i].Values[k].Compare(pending[j].Values[k]) != 0 { 49 | continue findRetractionLoop 50 | } 51 | } 52 | crossedOut[i] = true 53 | crossedOut[j] = true 54 | continue pendingLoop 55 | } 56 | } 57 | if err := produce(ctx, pending[i]); err != nil { 58 | return fmt.Errorf("couldn't produce: %w", err) 59 | } 60 | } 61 | 62 | pending = newPending 63 | 64 | return nil 65 | } 66 | 67 | if err := node.Source.Run( 68 | ctx, 69 | func(ctx ProduceContext, record Record) error { 70 | // TODO: Periodic compaction. (by combining records with their retractions) 71 | pending = append(pending, record) 72 | return nil 73 | }, 74 | func(ctx ProduceContext, msg MetadataMessage) error { 75 | if msg.Type == MetadataMessageTypeWatermark { 76 | if err := sendPendingLessOrEqualWatermark(ctx, msg.Watermark); err != nil { 77 | return err 78 | } 79 | } 80 | 81 | if err := metaSend(ctx, msg); err != nil { 82 | return fmt.Errorf("couldn't send metadata: %w", err) 83 | } 84 | 85 | return nil 86 | }, 87 | ); err != nil { 88 | return err 89 | } 90 | 91 | return sendPendingLessOrEqualWatermark(ProduceFromExecutionContext(ctx), WatermarkMaxValue) 92 | } 93 | -------------------------------------------------------------------------------- /outputs/stream/printer.go: -------------------------------------------------------------------------------- 1 | package stream 2 | 3 | import ( 4 | "fmt" 5 | "os" 6 | 7 | "github.com/pkg/errors" 8 | 9 | . "github.com/cube2222/octosql/execution" 10 | "github.com/cube2222/octosql/physical" 11 | ) 12 | 13 | type Format interface { 14 | SetSchema(physical.Schema) 15 | WriteRecord(Record) error 16 | WriteMeta(MetadataMessage) error 17 | Close() error 18 | } 19 | 20 | type OutputPrinter struct { 21 | source Node 22 | format Format 23 | } 24 | 25 | func NewOutputPrinter(source Node, format Format) *OutputPrinter { 26 | return &OutputPrinter{ 27 | source: source, 28 | format: format, 29 | } 30 | } 31 | 32 | func (o *OutputPrinter) Run(execCtx ExecutionContext) error { 33 | if err := o.source.Run(execCtx, func(ctx ProduceContext, record Record) error { 34 | return o.format.WriteRecord(record) 35 | }, func(ctx ProduceContext, msg MetadataMessage) error { 36 | return o.format.WriteMeta(msg) 37 | }); err != nil { 38 | return err 39 | } 40 | 41 | if err := o.format.Close(); err != nil { 42 | return errors.Wrap(err, "couldn't close output formatter") 43 | } 44 | return nil 45 | } 46 | 47 | type NativeFormat struct { 48 | schema physical.Schema 49 | } 50 | 51 | func NewNativeFormat(schema physical.Schema) *NativeFormat { 52 | return &NativeFormat{ 53 | schema: schema, 54 | } 55 | } 56 | 57 | func (n *NativeFormat) WriteRecord(record Record) error { 58 | fmt.Fprintf(os.Stdout, record.String()+"\n") 59 | return nil 60 | } 61 | 62 | func (n *NativeFormat) WriteMeta(message MetadataMessage) error { 63 | fmt.Fprintf(os.Stdout, "{~%s}\n", message.Watermark) 64 | return nil 65 | } 66 | 67 | func (n *NativeFormat) SetSchema(schema physical.Schema) { 68 | n.schema = schema 69 | } 70 | 71 | func (n *NativeFormat) Close() error { 72 | return nil 73 | } 74 | -------------------------------------------------------------------------------- /parser/sqlparser/Makefile: -------------------------------------------------------------------------------- 1 | # Copyright 2017 Google Inc. 2 | # 3 | # Licensed under the Apache License, Version 2.0 (the "License"); 4 | # you may not use this file except in compliance with the License. 5 | # You may obtain a copy of the License at 6 | # 7 | # http://www.apache.org/licenses/LICENSE-2.0 8 | # 9 | # Unless required by applicable law or agreed to in writing, software 10 | # distributed under the License is distributed on an "AS IS" BASIS, 11 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | # See the License for the specific language governing permissions and 13 | # limitations under the License. 14 | 15 | # Copyright 2019 The OctoSQL Authors 16 | # 17 | # Licensed under the MIT license, as in the LICENSE file 18 | 19 | MAKEFLAGS = -s 20 | 21 | sql.go: sql.y 22 | goyacc -o sql.go sql.y 23 | gofmt -w sql.go 24 | 25 | clean: 26 | rm -f y.output sql.go 27 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/bytes2/buffer.go: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2017 Google Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | /* 18 | Copyright 2019 The OctoSQL Authors 19 | 20 | Licensed under the MIT license, as in the LICENSE file 21 | */ 22 | 23 | package bytes2 24 | 25 | // Buffer implements a subset of the write portion of 26 | // bytes.Buffer, but more efficiently. This is meant to 27 | // be used in very high QPS operations, especially for 28 | // WriteByte, and without abstracting it as a Writer. 29 | // Function signatures contain errors for compatibility, 30 | // but they do not return errors. 31 | type Buffer struct { 32 | bytes []byte 33 | } 34 | 35 | // NewBuffer is equivalent to bytes.NewBuffer. 36 | func NewBuffer(b []byte) *Buffer { 37 | return &Buffer{bytes: b} 38 | } 39 | 40 | // Write is equivalent to bytes.Buffer.Write. 41 | func (buf *Buffer) Write(b []byte) (int, error) { 42 | buf.bytes = append(buf.bytes, b...) 43 | return len(b), nil 44 | } 45 | 46 | // WriteString is equivalent to bytes.Buffer.WriteString. 47 | func (buf *Buffer) WriteString(s string) (int, error) { 48 | buf.bytes = append(buf.bytes, s...) 49 | return len(s), nil 50 | } 51 | 52 | // WriteByte is equivalent to bytes.Buffer.WriteByte. 53 | func (buf *Buffer) WriteByte(b byte) error { 54 | buf.bytes = append(buf.bytes, b) 55 | return nil 56 | } 57 | 58 | // Bytes is equivalent to bytes.Buffer.Bytes. 59 | func (buf *Buffer) Bytes() []byte { 60 | return buf.bytes 61 | } 62 | 63 | // Strings is equivalent to bytes.Buffer.Strings. 64 | func (buf *Buffer) String() string { 65 | return string(buf.bytes) 66 | } 67 | 68 | // Len is equivalent to bytes.Buffer.Len. 69 | func (buf *Buffer) Len() int { 70 | return len(buf.bytes) 71 | } 72 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/bytes2/buffer_test.go: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2017 Google Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | /* 18 | Copyright 2019 The OctoSQL Authors 19 | 20 | Licensed under the MIT license, as in the LICENSE file 21 | */ 22 | 23 | package bytes2 24 | 25 | import ( 26 | "testing" 27 | ) 28 | 29 | func TestBuffer(t *testing.T) { 30 | b := NewBuffer(nil) 31 | b.Write([]byte("ab")) 32 | b.WriteString("cd") 33 | b.WriteByte('e') 34 | want := "abcde" 35 | if got := string(b.Bytes()); got != want { 36 | t.Errorf("b.Bytes(): %s, want %s", got, want) 37 | } 38 | if got := b.String(); got != want { 39 | t.Errorf("b.String(): %s, want %s", got, want) 40 | } 41 | if got := b.Len(); got != 5 { 42 | t.Errorf("b.Len(): %d, want 5", got) 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/hack/hack.go: -------------------------------------------------------------------------------- 1 | // Copyright 2012, Google Inc. All rights reserved. 2 | // Use of this source code is governed by a BSD-style 3 | // license that can be found in the LICENSE file. 4 | 5 | /* 6 | Copyright 2019 The OctoSQL Authors 7 | 8 | Licensed under the MIT license, as in the LICENSE file 9 | */ 10 | 11 | // Package hack gives you some efficient functionality at the cost of 12 | // breaking some Go rules. 13 | package hack 14 | 15 | import ( 16 | "reflect" 17 | "unsafe" 18 | ) 19 | 20 | // StringArena lets you consolidate allocations for a group of strings 21 | // that have similar life length 22 | type StringArena struct { 23 | buf []byte 24 | str string 25 | } 26 | 27 | // NewStringArena creates an arena of the specified size. 28 | func NewStringArena(size int) *StringArena { 29 | sa := &StringArena{buf: make([]byte, 0, size)} 30 | pbytes := (*reflect.SliceHeader)(unsafe.Pointer(&sa.buf)) 31 | pstring := (*reflect.StringHeader)(unsafe.Pointer(&sa.str)) 32 | pstring.Data = pbytes.Data 33 | pstring.Len = pbytes.Cap 34 | return sa 35 | } 36 | 37 | // NewString copies a byte slice into the arena and returns it as a string. 38 | // If the arena is full, it returns a traditional go string. 39 | func (sa *StringArena) NewString(b []byte) string { 40 | if len(b) == 0 { 41 | return "" 42 | } 43 | if len(sa.buf)+len(b) > cap(sa.buf) { 44 | return string(b) 45 | } 46 | start := len(sa.buf) 47 | sa.buf = append(sa.buf, b...) 48 | return sa.str[start : start+len(b)] 49 | } 50 | 51 | // SpaceLeft returns the amount of space left in the arena. 52 | func (sa *StringArena) SpaceLeft() int { 53 | return cap(sa.buf) - len(sa.buf) 54 | } 55 | 56 | // String force casts a []byte to a string. 57 | // USE AT YOUR OWN RISK 58 | func String(b []byte) (s string) { 59 | if len(b) == 0 { 60 | return "" 61 | } 62 | pbytes := (*reflect.SliceHeader)(unsafe.Pointer(&b)) 63 | pstring := (*reflect.StringHeader)(unsafe.Pointer(&s)) 64 | pstring.Data = pbytes.Data 65 | pstring.Len = pbytes.Len 66 | return 67 | } 68 | 69 | // StringPointer returns &s[0], which is not allowed in go 70 | func StringPointer(s string) unsafe.Pointer { 71 | pstring := (*reflect.StringHeader)(unsafe.Pointer(&s)) 72 | return unsafe.Pointer(pstring.Data) 73 | } 74 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/hack/hack_test.go: -------------------------------------------------------------------------------- 1 | // Copyright 2012, Google Inc. All rights reserved. 2 | // Use of this source code is governed by a BSD-style 3 | // license that can be found in the LICENSE file. 4 | 5 | /* 6 | Copyright 2019 The OctoSQL Authors 7 | 8 | Licensed under the MIT license, as in the LICENSE file 9 | */ 10 | 11 | package hack 12 | 13 | import "testing" 14 | 15 | func TestStringArena(t *testing.T) { 16 | sarena := NewStringArena(10) 17 | 18 | s0 := sarena.NewString(nil) 19 | checkint(t, len(sarena.buf), 0) 20 | checkint(t, sarena.SpaceLeft(), 10) 21 | checkstring(t, s0, "") 22 | 23 | s1 := sarena.NewString([]byte("01234")) 24 | checkint(t, len(sarena.buf), 5) 25 | checkint(t, sarena.SpaceLeft(), 5) 26 | checkstring(t, s1, "01234") 27 | 28 | s2 := sarena.NewString([]byte("5678")) 29 | checkint(t, len(sarena.buf), 9) 30 | checkint(t, sarena.SpaceLeft(), 1) 31 | checkstring(t, s2, "5678") 32 | 33 | // s3 will be allocated outside of sarena 34 | s3 := sarena.NewString([]byte("ab")) 35 | checkint(t, len(sarena.buf), 9) 36 | checkint(t, sarena.SpaceLeft(), 1) 37 | checkstring(t, s3, "ab") 38 | 39 | // s4 should still fit in sarena 40 | s4 := sarena.NewString([]byte("9")) 41 | checkint(t, len(sarena.buf), 10) 42 | checkint(t, sarena.SpaceLeft(), 0) 43 | checkstring(t, s4, "9") 44 | 45 | sarena.buf[0] = 'A' 46 | checkstring(t, s1, "A1234") 47 | 48 | sarena.buf[5] = 'B' 49 | checkstring(t, s2, "B678") 50 | 51 | sarena.buf[9] = 'C' 52 | // s3 will not change 53 | checkstring(t, s3, "ab") 54 | checkstring(t, s4, "C") 55 | checkstring(t, sarena.str, "A1234B678C") 56 | } 57 | 58 | func checkstring(t *testing.T, actual, expected string) { 59 | if actual != expected { 60 | t.Errorf("received %s, expecting %s", actual, expected) 61 | } 62 | } 63 | 64 | func checkint(t *testing.T, actual, expected int) { 65 | if actual != expected { 66 | t.Errorf("received %d, expecting %d", actual, expected) 67 | } 68 | } 69 | 70 | func TestByteToString(t *testing.T) { 71 | v1 := []byte("1234") 72 | if s := String(v1); s != "1234" { 73 | t.Errorf("String(\"1234\"): %q, want 1234", s) 74 | } 75 | 76 | v1 = []byte("") 77 | if s := String(v1); s != "" { 78 | t.Errorf("String(\"\"): %q, want empty", s) 79 | } 80 | 81 | v1 = nil 82 | if s := String(v1); s != "" { 83 | t.Errorf("String(\"\"): %q, want empty", s) 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/sqltypes/event_token.go: -------------------------------------------------------------------------------- 1 | package sqltypes 2 | 3 | import querypb "github.com/cube2222/octosql/parser/sqlparser/dependency/query" 4 | 5 | // EventTokenMinimum returns an event token that is guaranteed to 6 | // happen before both provided EventToken objects. Note it doesn't 7 | // parse the position, but rather only uses the timestamp. This is 8 | // meant to be used for EventToken objects coming from different 9 | // source shard. 10 | func EventTokenMinimum(ev1, ev2 *querypb.EventToken) *querypb.EventToken { 11 | if ev1 == nil || ev2 == nil { 12 | // One or the other is not set, we can't do anything. 13 | return nil 14 | } 15 | 16 | if ev1.Timestamp < ev2.Timestamp { 17 | return &querypb.EventToken{ 18 | Timestamp: ev1.Timestamp, 19 | } 20 | } 21 | return &querypb.EventToken{ 22 | Timestamp: ev2.Timestamp, 23 | } 24 | } 25 | -------------------------------------------------------------------------------- /parser/sqlparser/dependency/sqltypes/event_token_test.go: -------------------------------------------------------------------------------- 1 | package sqltypes 2 | 3 | import ( 4 | "testing" 5 | 6 | querypb "github.com/cube2222/octosql/parser/sqlparser/dependency/query" 7 | "github.com/golang/protobuf/proto" 8 | ) 9 | 10 | func TestEventTokenMinimum(t *testing.T) { 11 | testcases := []struct { 12 | ev1 *querypb.EventToken 13 | ev2 *querypb.EventToken 14 | expected *querypb.EventToken 15 | }{{ 16 | ev1: nil, 17 | ev2: nil, 18 | expected: nil, 19 | }, { 20 | ev1: &querypb.EventToken{ 21 | Timestamp: 123, 22 | }, 23 | ev2: nil, 24 | expected: nil, 25 | }, { 26 | ev1: nil, 27 | ev2: &querypb.EventToken{ 28 | Timestamp: 123, 29 | }, 30 | expected: nil, 31 | }, { 32 | ev1: &querypb.EventToken{ 33 | Timestamp: 123, 34 | }, 35 | ev2: &querypb.EventToken{ 36 | Timestamp: 456, 37 | }, 38 | expected: &querypb.EventToken{ 39 | Timestamp: 123, 40 | }, 41 | }, { 42 | ev1: &querypb.EventToken{ 43 | Timestamp: 456, 44 | }, 45 | ev2: &querypb.EventToken{ 46 | Timestamp: 123, 47 | }, 48 | expected: &querypb.EventToken{ 49 | Timestamp: 123, 50 | }, 51 | }} 52 | 53 | for _, tcase := range testcases { 54 | got := EventTokenMinimum(tcase.ev1, tcase.ev2) 55 | if tcase.expected == nil && got != nil { 56 | t.Errorf("expected nil result for Minimum(%v, %v) but got: %v", tcase.ev1, tcase.ev2, got) 57 | continue 58 | } 59 | if !proto.Equal(got, tcase.expected) { 60 | t.Errorf("got %v but expected %v for Minimum(%v, %v)", got, tcase.expected, tcase.ev1, tcase.ev2) 61 | } 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /parser/sqlparser/precedence_test.go: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2017 Google Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | /* 18 | Copyright 2019 The OctoSQL Authors 19 | 20 | Licensed under the MIT license, as in the LICENSE file 21 | */ 22 | 23 | package sqlparser 24 | 25 | import ( 26 | "fmt" 27 | "testing" 28 | ) 29 | 30 | func readable(node Expr) string { 31 | switch node := node.(type) { 32 | case *OrExpr: 33 | return fmt.Sprintf("(%s or %s)", readable(node.Left), readable(node.Right)) 34 | case *AndExpr: 35 | return fmt.Sprintf("(%s and %s)", readable(node.Left), readable(node.Right)) 36 | case *BinaryExpr: 37 | return fmt.Sprintf("(%s %s %s)", readable(node.Left), node.Operator, readable(node.Right)) 38 | case *IsExpr: 39 | return fmt.Sprintf("(%s %s)", readable(node.Expr), node.Operator) 40 | default: 41 | return String(node) 42 | } 43 | } 44 | 45 | func TestAndOrPrecedence(t *testing.T) { 46 | validSQL := []struct { 47 | input string 48 | output string 49 | }{{ 50 | input: "select * from a where a=b and c=d or e=f", 51 | output: "((a = b and c = d) or e = f)", 52 | }, { 53 | input: "select * from a where a=b or c=d and e=f", 54 | output: "(a = b or (c = d and e = f))", 55 | }} 56 | for _, tcase := range validSQL { 57 | tree, err := Parse(tcase.input) 58 | if err != nil { 59 | t.Error(err) 60 | continue 61 | } 62 | expr := readable(tree.(*Select).Where.Expr) 63 | if expr != tcase.output { 64 | t.Errorf("Parse: \n%s, want: \n%s", expr, tcase.output) 65 | } 66 | } 67 | } 68 | 69 | func TestPlusStarPrecedence(t *testing.T) { 70 | validSQL := []struct { 71 | input string 72 | output string 73 | }{{ 74 | input: "select 1+2*3 from a", 75 | output: "(1 + (2 * 3))", 76 | }, { 77 | input: "select 1*2+3 from a", 78 | output: "((1 * 2) + 3)", 79 | }} 80 | for _, tcase := range validSQL { 81 | tree, err := Parse(tcase.input) 82 | if err != nil { 83 | t.Error(err) 84 | continue 85 | } 86 | expr := readable(tree.(*Select).SelectExprs[0].(*AliasedExpr).Expr) 87 | if expr != tcase.output { 88 | t.Errorf("Parse: \n%s, want: \n%s", expr, tcase.output) 89 | } 90 | } 91 | } 92 | 93 | func TestIsPrecedence(t *testing.T) { 94 | validSQL := []struct { 95 | input string 96 | output string 97 | }{{ 98 | input: "select * from a where a+b is true", 99 | output: "((a + b) is true)", 100 | }, { 101 | input: "select * from a where a=1 and b=2 is true", 102 | output: "(a = 1 and (b = 2 is true))", 103 | }, { 104 | input: "select * from a where (a=1 and b=2) is true", 105 | output: "((a = 1 and b = 2) is true)", 106 | }} 107 | for _, tcase := range validSQL { 108 | tree, err := Parse(tcase.input) 109 | if err != nil { 110 | t.Error(err) 111 | continue 112 | } 113 | expr := readable(tree.(*Select).Where.Expr) 114 | if expr != tcase.output { 115 | t.Errorf("Parse: \n%s, want: \n%s", expr, tcase.output) 116 | } 117 | } 118 | } 119 | -------------------------------------------------------------------------------- /parser/sqlparser/truncate_query.go: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright 2017 Google Inc. 3 | 4 | Licensed under the Apache License, Version 2.0 (the "License"); 5 | you may not use this file except in compliance with the License. 6 | You may obtain a copy of the License at 7 | 8 | http://www.apache.org/licenses/LICENSE-2.0 9 | 10 | Unless required by applicable law or agreed to in writing, software 11 | distributed under the License is distributed on an "AS IS" BASIS, 12 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | See the License for the specific language governing permissions and 14 | limitations under the License. 15 | */ 16 | 17 | /* 18 | Copyright 2019 The OctoSQL Authors 19 | 20 | Licensed under the MIT license, as in the LICENSE file 21 | */ 22 | 23 | package sqlparser 24 | 25 | import ( 26 | "flag" 27 | ) 28 | 29 | var ( 30 | // TruncateUILen truncate queries in debug UIs to the given length. 0 means unlimited. 31 | TruncateUILen = flag.Int("sql-max-length-ui", 512, "truncate queries in debug UIs to the given length (default 512)") 32 | 33 | // TruncateErrLen truncate queries in error logs to the given length. 0 means unlimited. 34 | TruncateErrLen = flag.Int("sql-max-length-errors", 0, "truncate queries in error logs to the given length (default unlimited)") 35 | ) 36 | 37 | func truncateQuery(query string, max int) string { 38 | sql, comments := SplitMarginComments(query) 39 | 40 | if max == 0 || len(sql) <= max { 41 | return comments.Leading + sql + comments.Trailing 42 | } 43 | 44 | return comments.Leading + sql[:max-12] + " [TRUNCATED]" + comments.Trailing 45 | } 46 | 47 | // TruncateForUI is used when displaying queries on various Vitess status pages 48 | // to keep the pages small enough to load and render properly 49 | func TruncateForUI(query string) string { 50 | return truncateQuery(query, *TruncateUILen) 51 | } 52 | 53 | // TruncateForLog is used when displaying queries as part of error logs 54 | // to avoid overwhelming logging systems with potentially long queries and 55 | // bind value data. 56 | func TruncateForLog(query string) string { 57 | return truncateQuery(query, *TruncateErrLen) 58 | } 59 | -------------------------------------------------------------------------------- /physical/physical.go: -------------------------------------------------------------------------------- 1 | package physical 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "strings" 7 | 8 | "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/execution/nodes" 10 | "github.com/cube2222/octosql/octosql" 11 | ) 12 | 13 | // TODO: There should be a seperate MaterializationContext. 14 | type Environment struct { 15 | Aggregates map[string]AggregateDetails 16 | Datasources *DatasourceRepository 17 | Functions map[string]FunctionDetails 18 | PhysicalConfig map[string]interface{} 19 | VariableContext *VariableContext 20 | } 21 | 22 | func (env Environment) WithRecordSchema(schema Schema) Environment { 23 | newEnv := env 24 | newEnv.VariableContext = newEnv.VariableContext.WithRecordSchema(schema) 25 | return newEnv 26 | } 27 | 28 | type VariableContext struct { 29 | Parent *VariableContext 30 | Fields []SchemaField 31 | } 32 | 33 | func (varCtx *VariableContext) WithRecordSchema(schema Schema) *VariableContext { 34 | return &VariableContext{ 35 | Parent: varCtx, 36 | Fields: schema.Fields, 37 | } 38 | } 39 | 40 | type AggregateDetails struct { 41 | Description string 42 | Descriptors []AggregateDescriptor 43 | } 44 | 45 | type AggregateDescriptor struct { 46 | ArgumentType octosql.Type 47 | OutputType octosql.Type 48 | TypeFn func(octosql.Type) (octosql.Type, bool) 49 | Prototype func() nodes.Aggregate 50 | } 51 | 52 | type DatasourceRepository struct { 53 | // TODO: A może jednak ten bardziej dynamiczny interfejs? Że database. i wtedy sie resolvuje 54 | // Bo inaczej będzie na start strasznie dużo rzeczy ładować niepotrzebnych dla wszystkich 55 | // skonfigurowanych baz danych. 56 | Databases map[string]func() (Database, error) 57 | FileHandlers map[string]func(ctx context.Context, name string, options map[string]string) (DatasourceImplementation, Schema, error) 58 | } 59 | 60 | type Database interface { 61 | ListTables(ctx context.Context) ([]string, error) 62 | GetTable(ctx context.Context, name string, options map[string]string) (DatasourceImplementation, Schema, error) 63 | } 64 | 65 | func (dr *DatasourceRepository) GetDatasource(ctx context.Context, name string, options map[string]string) (DatasourceImplementation, Schema, error) { 66 | if index := strings.Index(name, "."); index != -1 { 67 | dbName := name[:index] 68 | dbConstructor, ok := dr.Databases[dbName] 69 | if ok { 70 | db, err := dbConstructor() 71 | if err != nil { 72 | return nil, Schema{}, fmt.Errorf("couldn't initialize database '%s': %w", dbName, err) 73 | } 74 | 75 | return db.GetTable(ctx, name[index+1:], options) 76 | } 77 | } 78 | if index := strings.LastIndex(name, "."); index != -1 { 79 | extension := name[index+1:] 80 | if handler, ok := dr.FileHandlers[extension]; ok { 81 | return handler(ctx, name, options) 82 | } 83 | } 84 | 85 | return nil, Schema{}, fmt.Errorf("no such table: %s", name) 86 | } 87 | 88 | type DatasourceImplementation interface { 89 | Materialize(ctx context.Context, env Environment, schema Schema, pushedDownPredicates []Expression) (execution.Node, error) 90 | PushDownPredicates(newPredicates, pushedDownPredicates []Expression) (rejected, pushedDown []Expression, changed bool) 91 | } 92 | 93 | type FunctionDetails struct { 94 | Description string 95 | Descriptors []FunctionDescriptor 96 | } 97 | 98 | type FunctionDescriptor struct { 99 | ArgumentTypes []octosql.Type 100 | OutputType octosql.Type 101 | TypeFn func([]octosql.Type) (octosql.Type, bool) `json:"-"` 102 | Strict bool 103 | Function func([]octosql.Value) (octosql.Value, error) `json:"-"` 104 | } 105 | -------------------------------------------------------------------------------- /physical/rename_variable.go: -------------------------------------------------------------------------------- 1 | package physical 2 | 3 | func RenameVariables(oldToNew map[string]string, node Node) Node { 4 | t := Transformers{ 5 | ExpressionTransformer: func(expr Expression) Expression { 6 | // TODO: Think about how this affects TBV argument table mapping. 7 | // It should properly rename descriptors. 8 | if expr.ExpressionType == ExpressionTypeVariable { 9 | if newName, ok := oldToNew[expr.Variable.Name]; ok { 10 | expr.Variable.Name = newName 11 | } 12 | } 13 | return expr 14 | }, 15 | } 16 | return t.TransformNode(node) 17 | } 18 | 19 | func RenameVariablesExpr(oldToNew map[string]string, expr Expression) Expression { 20 | t := Transformers{ 21 | ExpressionTransformer: func(expr Expression) Expression { 22 | // TODO: Think about how this affects TBV argument table mapping. 23 | // It should properly rename descriptors. 24 | if expr.ExpressionType == ExpressionTypeVariable { 25 | if newName, ok := oldToNew[expr.Variable.Name]; ok { 26 | expr.Variable.Name = newName 27 | } 28 | } 29 | return expr 30 | }, 31 | } 32 | return t.TransformExpr(expr) 33 | } 34 | -------------------------------------------------------------------------------- /physical/triggers.go: -------------------------------------------------------------------------------- 1 | package physical 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cube2222/octosql/execution" 7 | ) 8 | 9 | type Trigger struct { 10 | TriggerType TriggerType 11 | // Only one of the below may be non-null. 12 | CountingTrigger *CountingTrigger 13 | EndOfStreamTrigger *EndOfStreamTrigger 14 | WatermarkTrigger *WatermarkTrigger 15 | MultiTrigger *MultiTrigger 16 | } 17 | 18 | type TriggerType int 19 | 20 | const ( 21 | TriggerTypeCounting TriggerType = iota 22 | TriggerTypeEndOfStream 23 | TriggerTypeWatermark 24 | TriggerTypeMulti 25 | ) 26 | 27 | func (t TriggerType) String() string { 28 | switch t { 29 | case TriggerTypeCounting: 30 | return "counting" 31 | case TriggerTypeEndOfStream: 32 | return "end_of_stream" 33 | case TriggerTypeWatermark: 34 | return "watermark" 35 | case TriggerTypeMulti: 36 | return "multi" 37 | } 38 | return "unknown" 39 | } 40 | 41 | type CountingTrigger struct { 42 | TriggerAfter uint 43 | } 44 | 45 | type EndOfStreamTrigger struct { 46 | } 47 | 48 | type WatermarkTrigger struct { 49 | TimeFieldIndex int 50 | } 51 | 52 | type MultiTrigger struct { 53 | Triggers []Trigger 54 | } 55 | 56 | func (t *Trigger) Materialize(ctx context.Context, env Environment) func() execution.Trigger { 57 | switch t.TriggerType { 58 | case TriggerTypeCounting: 59 | return execution.NewCountingTriggerPrototype(t.CountingTrigger.TriggerAfter) 60 | case TriggerTypeEndOfStream: 61 | return execution.NewEndOfStreamTriggerPrototype() 62 | case TriggerTypeWatermark: 63 | return execution.NewWatermarkTriggerPrototype(t.WatermarkTrigger.TimeFieldIndex) 64 | case TriggerTypeMulti: 65 | prototypes := make([]func() execution.Trigger, len(t.MultiTrigger.Triggers)) 66 | for i := range t.MultiTrigger.Triggers { 67 | prototypes[i] = t.MultiTrigger.Triggers[i].Materialize(ctx, env) 68 | } 69 | return execution.NewMultiTriggerPrototype(prototypes) 70 | } 71 | 72 | panic("unexhaustive trigger type match") 73 | } 74 | 75 | // NoRetractions indicates whether the trigger can result in retractions of keys, or not. 76 | // In other words, if a single key can be triggered multiple times. 77 | func (t *Trigger) NoRetractions() bool { 78 | switch t.TriggerType { 79 | case TriggerTypeCounting, TriggerTypeMulti: 80 | return false 81 | case TriggerTypeEndOfStream, TriggerTypeWatermark: 82 | return true 83 | } 84 | 85 | panic("unexhaustive trigger type match") 86 | } 87 | -------------------------------------------------------------------------------- /plugin_repository.json: -------------------------------------------------------------------------------- 1 | { 2 | "name": "Core OctoSQL Plugin Repository", 3 | "description": "The official place to publish and look for open-source OctoSQL plugins.", 4 | "slug": "core", 5 | "plugins": [ 6 | { 7 | "name": "mysql", 8 | "description": "Adds support for querying MySQL databases.", 9 | "website": "https://github.com/cube2222/octosql-plugin-mysql", 10 | "contact_email": "jakub.wit.martin+octosql@gmail.com", 11 | "license": "Mozilla Public License Version 2.0", 12 | "readme_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-mysql/main/README.md", 13 | "manifest_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-mysql/main/octosql_manifest.json" 14 | }, 15 | { 16 | "name": "postgres", 17 | "description": "Adds support for querying PostgreSQL databases.", 18 | "website": "https://github.com/cube2222/octosql-plugin-postgres", 19 | "contact_email": "jakub.wit.martin+octosql@gmail.com", 20 | "license": "Mozilla Public License Version 2.0", 21 | "readme_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-postgres/main/README.md", 22 | "manifest_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-postgres/main/octosql_manifest.json" 23 | }, 24 | { 25 | "name": "random_data", 26 | "description": "Generates random data for testing.", 27 | "website": "https://github.com/cube2222/octosql-plugin-random_data", 28 | "contact_email": "jakub.wit.martin+octosql@gmail.com", 29 | "license": "Mozilla Public License Version 2.0", 30 | "readme_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-random_data/main/README.md", 31 | "manifest_url": "https://raw.githubusercontent.com/cube2222/octosql-plugin-random_data/main/octosql_manifest.json" 32 | } 33 | ] 34 | } 35 | -------------------------------------------------------------------------------- /plugins/internal/plugins/input.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "gopkg.in/yaml.v3" 5 | ) 6 | 7 | type PluginInput struct { 8 | Config yaml.Node 9 | } 10 | -------------------------------------------------------------------------------- /plugins/internal/plugins/plugins.proto: -------------------------------------------------------------------------------- 1 | syntax = "proto3"; 2 | package plugins; 3 | 4 | option go_package = "github.com/cube2222/octosql/plugins/internal/plugins"; 5 | import "google/protobuf/timestamp.proto"; 6 | import "google/protobuf/duration.proto"; 7 | 8 | message TableContext { 9 | string table_name = 1; 10 | map options = 2; 11 | } 12 | 13 | service Datasource { 14 | rpc GetTable (GetTableRequest) returns (GetTableResponse); 15 | rpc PushDownPredicates (PushDownPredicatesRequest) returns (PushDownPredicatesResponse); 16 | rpc Materialize (MaterializeRequest) returns (MaterializeResponse); 17 | rpc Metadata (MetadataRequest) returns (MetadataResponse); 18 | } 19 | 20 | message GetTableRequest { 21 | TableContext table_context = 1; 22 | } 23 | 24 | message GetTableResponse { 25 | Schema schema = 1; 26 | } 27 | 28 | message PushDownPredicatesRequest { 29 | // TODO: Add schema? Maybe TableWithSchemaContext? Or PhysicalTableContext? 30 | 31 | TableContext table_context = 1; 32 | // []Expression JSONs, not performance sensitive. 33 | bytes new_predicates = 2; 34 | bytes pushed_down_predicates = 3; 35 | } 36 | 37 | message PushDownPredicatesResponse { 38 | // []Expression JSONs, not performance sensitive. 39 | bytes rejected = 1; 40 | bytes pushed_down = 2; 41 | bool changed = 3; 42 | } 43 | 44 | message MaterializeRequest { 45 | TableContext table_context = 1; 46 | Schema schema = 2; 47 | bytes pushed_down_predicates = 3; 48 | PhysicalVariableContext variable_context = 4; 49 | } 50 | 51 | message MaterializeResponse { 52 | string socket_path = 1; 53 | } 54 | 55 | message MetadataRequest {} 56 | 57 | message MetadataResponse { 58 | int64 api_level = 1; 59 | } 60 | 61 | service ExecutionDatasource { 62 | rpc Run (RunRequest) returns (stream RunResponseMessage); 63 | } 64 | 65 | message RunRequest { 66 | ExecutionVariableContext variable_context = 4; 67 | } 68 | 69 | message RunResponseMessage { 70 | Record record = 1; 71 | MetadataMessage metadata = 2; 72 | } 73 | 74 | message Record { 75 | repeated Value values = 1; 76 | bool retraction = 2; 77 | google.protobuf.Timestamp event_time = 3; 78 | } 79 | 80 | message MetadataMessage { 81 | int32 message_type = 1; 82 | google.protobuf.Timestamp watermark = 2; 83 | } 84 | 85 | message Value { 86 | int32 type_id = 1; 87 | int64 int = 2; 88 | double float = 3; 89 | bool boolean = 4; 90 | string str = 5; 91 | google.protobuf.Timestamp time = 6; 92 | google.protobuf.Duration duration = 7; 93 | repeated Value list = 8; // TODO: These should have their own messages. 94 | repeated Value struct = 9; 95 | repeated Value tuple = 10; 96 | } 97 | 98 | message Schema { 99 | repeated SchemaField fields = 1; 100 | int32 time_field = 2; 101 | bool no_retractions = 3; 102 | } 103 | 104 | message SchemaField { 105 | string name = 1; 106 | Type type = 2; 107 | } 108 | 109 | message Type { 110 | int32 type_id = 1; 111 | Type list = 2; 112 | repeated StructField struct = 3; 113 | repeated Type tuple = 4; 114 | repeated Type union = 5; 115 | } 116 | 117 | message StructField { 118 | string name = 1; 119 | Type type = 2; 120 | } 121 | 122 | message PhysicalVariableContext { 123 | repeated PhysicalVariableContextFrame frames = 1; 124 | } 125 | 126 | message PhysicalVariableContextFrame { 127 | repeated SchemaField fields = 1; 128 | } 129 | 130 | message ExecutionVariableContext { 131 | repeated ExecutionVariableContextFrame frames = 1; 132 | } 133 | 134 | message ExecutionVariableContextFrame { 135 | repeated Value values = 1; 136 | } 137 | -------------------------------------------------------------------------------- /plugins/internal/plugins/plugins_test.go: -------------------------------------------------------------------------------- 1 | package plugins 2 | 3 | import ( 4 | "testing" 5 | "time" 6 | 7 | "github.com/stretchr/testify/assert" 8 | 9 | "github.com/cube2222/octosql/execution" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | func TestPhysicalVariableContext(t *testing.T) { 15 | c := &physical.VariableContext{ 16 | Fields: []physical.SchemaField{ 17 | { 18 | Name: "test1", 19 | Type: octosql.String, 20 | }, 21 | { 22 | Name: "test2", 23 | Type: octosql.Int, 24 | }, 25 | { 26 | Name: "test3", 27 | Type: octosql.Boolean, 28 | }, 29 | }, 30 | Parent: &physical.VariableContext{ 31 | Fields: []physical.SchemaField{ 32 | { 33 | Name: "test4", 34 | Type: octosql.Time, 35 | }, 36 | { 37 | Name: "test5", 38 | Type: octosql.Duration, 39 | }, 40 | { 41 | Name: "test6", 42 | Type: octosql.Boolean, 43 | }, 44 | }, 45 | Parent: &physical.VariableContext{ 46 | Fields: []physical.SchemaField{ 47 | { 48 | Name: "test7", 49 | Type: octosql.Time, 50 | }, 51 | { 52 | Name: "test8", 53 | Type: octosql.String, 54 | }, 55 | { 56 | Name: "test9", 57 | Type: octosql.Null, 58 | }, 59 | }, 60 | Parent: nil, 61 | }, 62 | }, 63 | } 64 | 65 | outC := NativePhysicalVariableContextToProto(c).ToNativePhysicalVariableContext() 66 | 67 | assert.Equal(t, c, outC) 68 | } 69 | 70 | func TestExecutionVariableContext(t *testing.T) { 71 | c := &execution.VariableContext{ 72 | Values: []octosql.Value{ 73 | octosql.NewString("test1"), 74 | octosql.NewInt(42), 75 | octosql.NewBoolean(true), 76 | }, 77 | Parent: &execution.VariableContext{ 78 | Values: []octosql.Value{ 79 | octosql.NewDuration(time.Second * 3), 80 | octosql.NewBoolean(false), 81 | }, 82 | Parent: &execution.VariableContext{ 83 | Values: []octosql.Value{ 84 | octosql.NewString("test2"), 85 | octosql.NewNull(), 86 | }, 87 | Parent: nil, 88 | }, 89 | }, 90 | } 91 | 92 | outC := NativeExecutionVariableContextToProto(c).ToNativeExecutionVariableContext() 93 | 94 | assert.Equal(t, c, outC) 95 | } 96 | -------------------------------------------------------------------------------- /plugins/manager/extensions.go: -------------------------------------------------------------------------------- 1 | package manager 2 | 3 | import ( 4 | "encoding/json" 5 | "fmt" 6 | "log" 7 | "os" 8 | "path/filepath" 9 | 10 | "github.com/cube2222/octosql/config" 11 | ) 12 | 13 | var octosqlFileExtensionHandlersFile = func() string { 14 | return filepath.Join(config.OctosqlConfigDir, "file_extension_handlers.json") 15 | }() 16 | 17 | func (*PluginManager) GetFileExtensionHandlers() (map[string]string, error) { 18 | return loadFileExtensionHandlers() 19 | } 20 | 21 | func registerFileExtensions(name string, extensions []string) error { 22 | handlers, err := loadFileExtensionHandlers() 23 | if err != nil { 24 | return err 25 | } 26 | for _, ext := range extensions { 27 | if oldName, ok := handlers[ext]; ok && oldName != name { 28 | log.Printf("file extension handler for %s already registered, overwriting", ext) 29 | } 30 | handlers[ext] = name 31 | } 32 | return saveFileExtensionHandlers(handlers) 33 | } 34 | 35 | func loadFileExtensionHandlers() (map[string]string, error) { 36 | data, err := os.ReadFile(octosqlFileExtensionHandlersFile) 37 | if err != nil { 38 | if os.IsNotExist(err) { 39 | return map[string]string{}, nil 40 | } 41 | return nil, fmt.Errorf("couldn't read file extension handlers file: %w", err) 42 | } 43 | var handlers map[string]string 44 | if err := json.Unmarshal(data, &handlers); err != nil { 45 | return nil, fmt.Errorf("couldn't json-decode file extension handlers file: %w", err) 46 | } 47 | return handlers, nil 48 | } 49 | 50 | func saveFileExtensionHandlers(handlers map[string]string) error { 51 | data, err := json.Marshal(handlers) 52 | if err != nil { 53 | return fmt.Errorf("couldn't json-encode file extension handlers: %w", err) 54 | } 55 | if err := os.WriteFile(octosqlFileExtensionHandlersFile, data, 0644); err != nil { 56 | return fmt.Errorf("couldn't write file extension handlers to file: %w", err) 57 | } 58 | return nil 59 | } 60 | -------------------------------------------------------------------------------- /table_valued_functions/range.go: -------------------------------------------------------------------------------- 1 | package table_valued_functions 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "time" 7 | 8 | "github.com/cube2222/octosql/execution" 9 | "github.com/cube2222/octosql/logical" 10 | "github.com/cube2222/octosql/octosql" 11 | "github.com/cube2222/octosql/physical" 12 | ) 13 | 14 | var Range = logical.TableValuedFunctionDescription{ 15 | TypecheckArguments: func(ctx context.Context, env physical.Environment, logicalEnv logical.Environment, args map[string]logical.TableValuedFunctionArgumentValue) map[string]logical.TableValuedFunctionTypecheckedArgument { 16 | outArgs := make(map[string]logical.TableValuedFunctionTypecheckedArgument) 17 | outArgs["start"] = logical.TableValuedFunctionTypecheckedArgument{ 18 | Argument: args["start"].(*logical.TableValuedFunctionArgumentValueExpression).Typecheck(ctx, env, logicalEnv), 19 | } 20 | outArgs["end"] = logical.TableValuedFunctionTypecheckedArgument{ 21 | Argument: args["end"].(*logical.TableValuedFunctionArgumentValueExpression).Typecheck(ctx, env, logicalEnv), 22 | } 23 | return outArgs 24 | }, 25 | Descriptors: []logical.TableValuedFunctionDescriptor{ 26 | { 27 | Arguments: map[string]logical.TableValuedFunctionArgumentMatcher{ 28 | "start": { 29 | Required: true, 30 | TableValuedFunctionArgumentMatcherType: physical.TableValuedFunctionArgumentTypeExpression, 31 | Expression: &logical.TableValuedFunctionArgumentMatcherExpression{ 32 | Type: octosql.Int, 33 | }, 34 | }, 35 | "end": { 36 | Required: true, 37 | TableValuedFunctionArgumentMatcherType: physical.TableValuedFunctionArgumentTypeExpression, 38 | Expression: &logical.TableValuedFunctionArgumentMatcherExpression{ 39 | Type: octosql.Int, 40 | }, 41 | }, 42 | }, 43 | OutputSchema: func( 44 | ctx context.Context, 45 | env physical.Environment, 46 | logicalEnv logical.Environment, 47 | args map[string]logical.TableValuedFunctionTypecheckedArgument, 48 | ) (physical.Schema, map[string]string, error) { 49 | unique := logicalEnv.GetUnique("i") 50 | 51 | return physical.Schema{ 52 | Fields: []physical.SchemaField{ 53 | { 54 | Name: unique, 55 | Type: octosql.Int, 56 | }, 57 | }, 58 | TimeField: -1, 59 | NoRetractions: true, 60 | }, map[string]string{ 61 | "i": unique, 62 | }, nil 63 | }, 64 | Materialize: func( 65 | ctx context.Context, 66 | environment physical.Environment, 67 | args map[string]physical.TableValuedFunctionArgument, 68 | ) (execution.Node, error) { 69 | start, err := args["start"].Expression.Expression.Materialize(ctx, environment) 70 | if err != nil { 71 | return nil, fmt.Errorf("couldn't materialize start: %w", err) 72 | } 73 | end, err := args["end"].Expression.Expression.Materialize(ctx, environment) 74 | if err != nil { 75 | return nil, fmt.Errorf("couldn't materialize end: %w", err) 76 | } 77 | 78 | return &rangeNode{ 79 | start: start, 80 | end: end, 81 | }, err 82 | }, 83 | }, 84 | }, 85 | } 86 | 87 | type rangeNode struct { 88 | start, end execution.Expression 89 | } 90 | 91 | func (r *rangeNode) Run(ctx execution.ExecutionContext, produce execution.ProduceFn, metaSend execution.MetaSendFn) error { 92 | start, err := r.start.Evaluate(ctx) 93 | if err != nil { 94 | return fmt.Errorf("couldn't evaluate start: %w", err) 95 | } 96 | end, err := r.end.Evaluate(ctx) 97 | if err != nil { 98 | return fmt.Errorf("couldn't evaluate end: %w", err) 99 | } 100 | for i := start.Int; i < end.Int; i++ { 101 | if err := produce( 102 | execution.ProduceFromExecutionContext(ctx), 103 | execution.NewRecord([]octosql.Value{octosql.NewInt(i)}, false, time.Time{}), 104 | ); err != nil { 105 | return fmt.Errorf("couldn't produce record: %w", err) 106 | } 107 | } 108 | return nil 109 | } 110 | -------------------------------------------------------------------------------- /telemetry/query.go: -------------------------------------------------------------------------------- 1 | package telemetry 2 | 3 | import ( 4 | "github.com/cube2222/octosql/physical" 5 | "github.com/cube2222/octosql/plugins/manager" 6 | ) 7 | 8 | type QueryTelemetryData struct { 9 | NodeTypesUsed []string `json:"node_types_used"` 10 | ExpressionTypesUsed []string `json:"expression_types_used"` 11 | TriggerTypesUsed []string `json:"trigger_types_used"` 12 | FunctionsUsed []string `json:"functions_used"` 13 | AggregatesUsed []string `json:"aggregates_used"` 14 | PluginsInstalled []string `json:"plugins_installed"` 15 | } 16 | 17 | func GetQueryTelemetryData(node physical.Node, pluginsInstalled []manager.PluginMetadata) QueryTelemetryData { 18 | nodeTypesUsed := make(map[string]bool) 19 | expressionTypesUsed := make(map[string]bool) 20 | triggerTypesUsed := make(map[string]bool) 21 | functionsUsed := make(map[string]bool) 22 | aggregatesUsed := make(map[string]bool) 23 | (&physical.Transformers{ 24 | NodeTransformer: func(node physical.Node) physical.Node { 25 | nodeTypesUsed[node.NodeType.String()] = true 26 | if node.NodeType == physical.NodeTypeGroupBy { 27 | getTriggerTypes(triggerTypesUsed, node.GroupBy.Trigger) 28 | for _, agg := range node.GroupBy.Aggregates { 29 | aggregatesUsed[agg.Name] = true 30 | } 31 | } 32 | return node 33 | }, 34 | ExpressionTransformer: func(expr physical.Expression) physical.Expression { 35 | expressionTypesUsed[expr.ExpressionType.String()] = true 36 | if expr.ExpressionType == physical.ExpressionTypeFunctionCall { 37 | functionsUsed[expr.FunctionCall.Name] = true 38 | } 39 | return expr 40 | }, 41 | }).TransformNode(node) 42 | out := QueryTelemetryData{} 43 | for k := range nodeTypesUsed { 44 | out.NodeTypesUsed = append(out.NodeTypesUsed, k) 45 | } 46 | for k := range expressionTypesUsed { 47 | out.ExpressionTypesUsed = append(out.ExpressionTypesUsed, k) 48 | } 49 | for k := range triggerTypesUsed { 50 | out.TriggerTypesUsed = append(out.TriggerTypesUsed, k) 51 | } 52 | for k := range functionsUsed { 53 | out.FunctionsUsed = append(out.FunctionsUsed, k) 54 | } 55 | for k := range aggregatesUsed { 56 | out.AggregatesUsed = append(out.AggregatesUsed, k) 57 | } 58 | for _, plugin := range pluginsInstalled { 59 | out.PluginsInstalled = append(out.PluginsInstalled, plugin.Reference.String()) 60 | } 61 | return out 62 | } 63 | 64 | func getTriggerTypes(triggerTypesUsed map[string]bool, trigger physical.Trigger) { 65 | triggerTypesUsed[trigger.TriggerType.String()] = true 66 | if trigger.TriggerType == physical.TriggerTypeMulti { 67 | for _, t := range trigger.MultiTrigger.Triggers { 68 | getTriggerTypes(triggerTypesUsed, t) 69 | } 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /telemetry/telemetry.go: -------------------------------------------------------------------------------- 1 | package telemetry 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "crypto/rand" 7 | "encoding/json" 8 | "fmt" 9 | "log" 10 | "net/http" 11 | "os" 12 | "path/filepath" 13 | "runtime" 14 | "time" 15 | 16 | "github.com/cube2222/octosql/config" 17 | "github.com/oklog/ulid/v2" 18 | ) 19 | 20 | var telemetryDir = func() string { 21 | return filepath.Join(config.OctosqlCacheDir, ".octosql/telemetry") 22 | }() 23 | 24 | type event struct { 25 | DeviceID string `json:"device_id"` 26 | Type string `json:"type"` 27 | Version string `json:"version"` 28 | OS string `json:"os"` 29 | Architecture string `json:"architecture"` 30 | NumCPU int `json:"num_cpu"` 31 | Time time.Time `json:"time"` 32 | Data interface{} `json:"data"` 33 | } 34 | 35 | func SendTelemetry(ctx context.Context, version, eventType string, data interface{}) { 36 | if err := sendTelemetry(ctx, version, eventType, data); err != nil { 37 | log.Printf("couldn't send telemetry: %s", err) 38 | } 39 | } 40 | 41 | func sendTelemetry(ctx context.Context, version, eventType string, data interface{}) error { 42 | if os.Getenv("OCTOSQL_NO_TELEMETRY") == "1" { 43 | return nil 44 | } 45 | deviceIDBytes, err := os.ReadFile(filepath.Join(telemetryDir, "device_id")) 46 | if os.IsNotExist(err) { 47 | fmt.Println(`OctoSQL sends anonymous usage statistics to help us guide the development of OctoSQL. 48 | You can view the most recently sent usage events in the ~/.octosql/telemetry/recent directory. 49 | You can turn telemetry off by setting the environment variable OCTOSQL_NO_TELEMETRY to 1. 50 | Please don't though, as it helps us a great deal.`) 51 | os.MkdirAll(telemetryDir, 0755) 52 | os.WriteFile(filepath.Join(telemetryDir, "device_id"), []byte(ulid.MustNew(ulid.Now(), rand.Reader).String()), 0644) 53 | return nil 54 | } 55 | deviceID := string(deviceIDBytes) 56 | 57 | payload := event{ 58 | DeviceID: deviceID, 59 | Type: eventType, 60 | Version: version, 61 | OS: runtime.GOOS, 62 | Architecture: runtime.GOARCH, 63 | NumCPU: runtime.NumCPU(), 64 | Time: time.Now(), 65 | Data: data, 66 | } 67 | body, err := json.Marshal(&payload) 68 | if err != nil { 69 | return err 70 | } 71 | 72 | if err := os.MkdirAll(filepath.Join(telemetryDir, "pending"), 0755); err != nil { 73 | return err 74 | } 75 | 76 | if err := os.WriteFile(filepath.Join(telemetryDir, "pending", ulid.MustNew(ulid.Now(), rand.Reader).String()+".json"), body, 0644); err != nil { 77 | return err 78 | } 79 | 80 | return sendBatch(ctx) 81 | } 82 | 83 | func sendBatch(ctx context.Context) error { 84 | files, err := filepath.Glob(filepath.Join(telemetryDir, "pending/*.json")) 85 | if err != nil { 86 | return err 87 | } 88 | 89 | if len(files) == 0 { 90 | return nil 91 | } 92 | 93 | minimumBatchCount := 10 94 | if _, err := os.Stat(filepath.Join(telemetryDir, "first_message_sent")); os.IsNotExist(err) { 95 | minimumBatchCount = 1 96 | } 97 | 98 | if len(files)%minimumBatchCount != 0 { 99 | return nil 100 | } 101 | 102 | var payload []json.RawMessage 103 | for i := range files { 104 | data, err := os.ReadFile(files[i]) 105 | if err != nil { 106 | return err 107 | } 108 | payload = append(payload, data) 109 | } 110 | data, err := json.Marshal(payload) 111 | if err != nil { 112 | return err 113 | } 114 | 115 | if _, err := http.Post("https://telemetry.octosql.dev/telemetry", "encoding/json", bytes.NewReader(data)); err != nil { 116 | return err 117 | } 118 | 119 | if err := os.RemoveAll(filepath.Join(telemetryDir, "recent")); err != nil { 120 | return err 121 | } 122 | 123 | if err := os.Rename(filepath.Join(telemetryDir, "pending"), filepath.Join(telemetryDir, "recent")); err != nil { 124 | return err 125 | } 126 | 127 | if minimumBatchCount == 1 { 128 | os.WriteFile(filepath.Join(telemetryDir, "first_message_sent"), []byte{}, 0644) 129 | } 130 | 131 | return nil 132 | } 133 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error1.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: couldn't run query: couldn't run source: couldn't run source: couldn't parse line 1123: couldn't parse json: cannot parse JSON: cannot parse number: unexpected char: "a"; unparsed tail: "abcd" 27 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error1.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT COUNT(*), SUM(field2), SUM(len(field3)) FROM fixtures/objects_bad_1.json" 2 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error1.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/datasources/json/error1.out -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error2.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: couldn't run query: couldn't run source: couldn't run source: couldn't parse line 5045: expected JSON object, got '"abcd"' 27 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error2.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT COUNT(*), SUM(field2), SUM(len(field3)) FROM fixtures/objects_bad_2.json" 2 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/error2.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/datasources/json/error2.out -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/lookup_join.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/datasources/json/lookup_join.err -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/lookup_join.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT COUNT(*), SUM(j2.field2), SUM(len(j1.field3)) FROM fixtures/join1.json j1 LOOKUP JOIN fixtures/join2.json j2 ON j1.field1 = j2.field1" -o batch_table 2 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/lookup_join.out: -------------------------------------------------------------------------------- 1 | +-------+------------+-------+ 2 | | count | sum_field2 | sum | 3 | +-------+------------+-------+ 4 | | 18144 | 675864 | 81648 | 5 | +-------+------------+-------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/simple.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/datasources/json/simple.err -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/simple.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT COUNT(*), SUM(field2), SUM(len(field3)) FROM fixtures/objects.json" 2 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/simple.out: -------------------------------------------------------------------------------- 1 | +-------+------------+-------+ 2 | | count | sum_field2 | sum | 3 | +-------+------------+-------+ 4 | | 20500 | 763625 | 92250 | 5 | +-------+------------+-------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/stream_join.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/datasources/json/stream_join.err -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/stream_join.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT COUNT(*), SUM(j2.field2), SUM(len(j1.field3)) FROM fixtures/join1_big.json j1 JOIN fixtures/join2_big.json j2 ON j1.field1 = j2.field1" -o batch_table 2 | -------------------------------------------------------------------------------- /tests/scenarios/datasources/json/stream_join.out: -------------------------------------------------------------------------------- 1 | +--------+---------------+---------+ 2 | | count | sum_field2 | sum | 3 | +--------+---------------+---------+ 4 | | 489888 | 1.8248328e+07 | 2204496 | 5 | +--------+---------------+---------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/arithmetic.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/arithmetic.err -------------------------------------------------------------------------------- /tests/scenarios/functions/arithmetic.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 1 + 2, 2 | 1.0 + 2.0, 3 | INTERVAL 1 SECOND + INTERVAL 2 SECONDS, 4 | time_to_unix(time_from_unix(1655931949) + INTERVAL 2 SECONDS), 5 | time_to_unix(INTERVAL 1 SECONDS + time_from_unix(1655931949)), 6 | 'test1' + 'test2', 7 | 1 - 2, 8 | -2, 9 | 1.0 - 2.0, 10 | -2.0, 11 | INTERVAL 3 SECOND - INTERVAL 1 SECONDS, 12 | -INTERVAL 3 SECOND, 13 | time_to_unix(time_from_unix(1655931949) - INTERVAL 2 SECONDS), 14 | 7 * 3, 15 | 7.0 * 3.0, 16 | INTERVAL 7 SECOND * 3, 17 | 7 * INTERVAL 3 SECOND, 18 | 'test1' * 3, 19 | 3 * 'test1', 20 | 7 / 3, 21 | 7.0 / 3.0, 22 | INTERVAL 7 SECOND / 3, 23 | INTERVAL 7 SECOND / INTERVAL 3 SECOND" 24 | 25 | -------------------------------------------------------------------------------- /tests/scenarios/functions/arithmetic.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+------------+------------+--------------+-------+-------+-------+-------+--------+--------+------------+--------+--------+--------+--------+-------------------+-------------------+--------+--------------------+--------------+--------------------+ 2 | | col_0 | col_1 | col_2 | col_3 | col_4 | col_5 | col_6 | col_7 | col_8 | col_9 | col_10 | col_11 | col_12 | col_13 | col_14 | col_15 | col_16 | col_17 | col_18 | col_19 | col_20 | col_21 | col_22 | 3 | +-------+-------+-------+------------+------------+--------------+-------+-------+-------+-------+--------+--------+------------+--------+--------+--------+--------+-------------------+-------------------+--------+--------------------+--------------+--------------------+ 4 | | 3 | 3 | 3s | 1655931951 | 1655931950 | 'test1test2' | -1 | -2 | -1 | -2 | 2s | -3s | 1655931947 | 21 | 21 | 21s | 21s | 'test1test1test1' | 'test1test1test1' | 2 | 2.3333333333333335 | 2.333333333s | 2.3333333333333335 | 5 | +-------+-------+-------+------------+------------+--------------+-------+-------+-------+-------+--------+--------+------------+--------+--------+--------+--------+-------------------+-------------------+--------+--------------------+--------------+--------------------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/comparators.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/comparators.err -------------------------------------------------------------------------------- /tests/scenarios/functions/comparators.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 0 < 1, 1 < 1, 1 < 0, 2 | 0 <= 1, 1 <= 1, 1 <= 0, 3 | 0 = 1, 1 = 1, 1 = 0, 4 | 0 != 1, 1 != 1, 1 != 0, 5 | 0 >= 1, 1 >= 1, 1 >= 0, 6 | 0 > 1, 1 > 1, 1 > 0, 7 | 'a' < 'b', 'b' < 'b', 'b' < 'a', 8 | 'a' <= 'b', 'b' <= 'b', 'b' <= 'a', 9 | 'a' = 'b', 'b' = 'b', 'b' = 'a', 10 | 'a' != 'b', 'b' != 'b', 'b' != 'a', 11 | 'a' >= 'b', 'b' >= 'b', 'b' >= 'a', 12 | 'a' > 'b', 'b' > 'b', 'b' > 'a', 13 | NULL < NULL, NULL <= NULL, NULL = NULL, NULL >= NULL, NULL > NULL" 14 | -------------------------------------------------------------------------------- /tests/scenarios/functions/comparators.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+ 2 | | col_0 | col_1 | col_2 | col_3 | col_4 | col_5 | col_6 | col_7 | col_8 | col_9 | col_10 | col_11 | col_12 | col_13 | col_14 | col_15 | col_16 | col_17 | col_18 | col_19 | col_20 | col_21 | col_22 | col_23 | col_24 | col_25 | col_26 | col_27 | col_28 | col_29 | col_30 | col_31 | col_32 | col_33 | col_34 | col_35 | col_36 | col_37 | col_38 | col_39 | col_40 | 3 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+ 4 | | true | false | false | true | true | false | false | true | false | true | false | true | false | true | true | false | false | true | true | false | false | true | true | false | false | true | false | true | false | true | false | true | true | false | false | true | | | | | | 5 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/conversions.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/conversions.err -------------------------------------------------------------------------------- /tests/scenarios/functions/conversions.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT int(true), int(false)" 2 | -------------------------------------------------------------------------------- /tests/scenarios/functions/conversions.out: -------------------------------------------------------------------------------- 1 | +-------+-------+ 2 | | col_0 | col_1 | 3 | +-------+-------+ 4 | | 1 | 0 | 5 | +-------+-------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/in.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/in.err -------------------------------------------------------------------------------- /tests/scenarios/functions/in.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 4 IN (1,2,3,4,5), 4 IN (1,2,3,5), 2 | 4 NOT IN (1,2,3,4,5), 4 NOT IN (1,2,3,5)" 3 | -------------------------------------------------------------------------------- /tests/scenarios/functions/in.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+-------+ 2 | | col_0 | col_1 | col_2 | col_3 | 3 | +-------+-------+-------+-------+ 4 | | true | false | false | true | 5 | +-------+-------+-------+-------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/is_null.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/is_null.err -------------------------------------------------------------------------------- /tests/scenarios/functions/is_null.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 0 IS NULL, NULL IS NULL, 0 IS NOT NULL, NULL IS NOT NULL" 2 | -------------------------------------------------------------------------------- /tests/scenarios/functions/is_null.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+-------+ 2 | | col_0 | col_1 | col_2 | col_3 | 3 | +-------+-------+-------+-------+ 4 | | false | true | true | false | 5 | +-------+-------+-------+-------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/logic.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/logic.err -------------------------------------------------------------------------------- /tests/scenarios/functions/logic.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT not(false), not(true), 2 | true AND false, true AND true, false AND false, false AND true, 3 | true OR false, true OR true, false OR false, false OR true, 4 | true OR NULL, false OR NULL, 5 | true AND NULL, false AND NULL" 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/logic.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+ 2 | | col_0 | col_1 | col_2 | col_3 | col_4 | col_5 | col_6 | col_7 | col_8 | col_9 | col_10 | col_11 | col_12 | col_13 | 3 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+ 4 | | true | false | false | true | false | false | true | true | false | true | true | | | false | 5 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------+--------+--------+--------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/math.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/math.err -------------------------------------------------------------------------------- /tests/scenarios/functions/math.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT abs(42), abs(-42), abs(42.0), abs(-42.0), 2 | sqrt(49.0), 3 | ceil(42.0), ceil(42.5), 4 | floor(42.0), floor(42.5), 5 | log2(8.0), 6 | log(9.0), 7 | log10(100.0), 8 | pow(2.0, 3.0)" 9 | -------------------------------------------------------------------------------- /tests/scenarios/functions/math.out: -------------------------------------------------------------------------------- 1 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------------------+--------+--------+ 2 | | col_0 | col_1 | col_2 | col_3 | col_4 | col_5 | col_6 | col_7 | col_8 | col_9 | col_10 | col_11 | col_12 | 3 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------------------+--------+--------+ 4 | | 42 | 42 | 42 | 42 | 7 | 42 | 43 | 42 | 42 | 3 | 2.1972245773362196 | 2 | 8 | 5 | +-------+-------+-------+-------+-------+-------+-------+-------+-------+-------+--------------------+--------+--------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/functions/panic.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: couldn't run query: couldn't run source: couldn't produce record: couldn't evaluate 0 map expression: couldn't evaluate function: panic: 'test' 27 | -------------------------------------------------------------------------------- /tests/scenarios/functions/panic.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT panic('test')" 2 | -------------------------------------------------------------------------------- /tests/scenarios/functions/panic.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/panic.out -------------------------------------------------------------------------------- /tests/scenarios/functions/strings.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/functions/strings.err -------------------------------------------------------------------------------- /tests/scenarios/functions/strings.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 'test' LIKE '%es%' as line_1, 'test' LIKE 'es', 'test' LIKE 'tes%', 'test' LIKE '%est', 'test' LIKE 'es%', 'test' LIKE 't_st', 'test' LIKE 't_t', 2 | 'test' ~ '^te.*' as line_2, 'test' ~ '^tE.*', 'test' ~ 'te.*', 'test' ~ 'es', 'test' ~ '^es$', 3 | 'test' !~ '^te.*' as line_3, 'test' !~ '^tE.*', 'test' !~ 'te.*', 'test' !~ 'es', 'test' !~ '^es$', 4 | 'test' ~* '^te.*' as line_4, 'test' ~* '^tE.*', 'test' ~* 'te.*', 'test' ~* 'es', 'test' ~* '^es$', 5 | 'test' !~* '^te.*' as line_5, 'test' !~* '^tE.*', 'test' !~* 'te.*', 'test' !~* 'es', 'test' !~* '^es$', 6 | upper('test') as line_6, upper('TEST'), 7 | lower('test') as line_7, lower('TEST'), 8 | reverse('test') as line_8, 9 | substr('test', 1) as line_9, substr('test', 1, 2), 10 | replace('test', 'es', 'ESS') as line_10, 11 | position('test', 'es') as line_11, position('test', ''), position('test', 'ex'), position('test', 'testt'), position('test', 'test'), 12 | len('test') as line_12" 13 | -------------------------------------------------------------------------------- /tests/scenarios/functions/strings.out: -------------------------------------------------------------------------------- 1 | +--------+-------+-------+-------+-------+-------+-------+--------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+---------+---------+--------+--------+--------+--------+---------+ 2 | | line_1 | col_1 | col_2 | col_3 | col_4 | col_5 | col_6 | line_2 | col_8 | col_9 | col_10 | col_11 | line_3 | col_13 | col_14 | col_15 | col_16 | line_4 | col_18 | col_19 | col_20 | col_21 | line_5 | col_23 | col_24 | col_25 | col_26 | line_6 | col_28 | line_7 | col_30 | line_8 | line_9 | col_33 | line_10 | line_11 | col_36 | col_37 | col_38 | col_39 | line_12 | 3 | +--------+-------+-------+-------+-------+-------+-------+--------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+---------+---------+--------+--------+--------+--------+---------+ 4 | | true | false | true | true | false | true | false | true | false | true | true | false | false | true | false | false | true | true | true | true | true | false | false | false | false | false | true | 'TEST' | 'TEST' | 'test' | 'test' | 'tset' | 'est' | 'es' | 'tESSt' | 1 | 0 | | | 0 | 4 | 5 | +--------+-------+-------+-------+-------+-------+-------+--------+-------+-------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+--------+---------+---------+--------+--------+--------+--------+---------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/inner_join_0.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/inner_join_0.err -------------------------------------------------------------------------------- /tests/scenarios/nodes/inner_join_0.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) l JOIN range(start=>-5, end=>5) r ON l.i > r.i" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/inner_join_0.out: -------------------------------------------------------------------------------- 1 | +-----+-----+ 2 | | l.i | r.i | 3 | +-----+-----+ 4 | | 1 | -5 | 5 | | 1 | -4 | 6 | | 1 | -3 | 7 | | 1 | -2 | 8 | | 1 | -1 | 9 | | 1 | 0 | 10 | | 2 | -5 | 11 | | 2 | -4 | 12 | | 2 | -3 | 13 | | 2 | -2 | 14 | | 2 | -1 | 15 | | 2 | 0 | 16 | | 2 | 1 | 17 | | 3 | -5 | 18 | | 3 | -4 | 19 | | 3 | -3 | 20 | | 3 | -2 | 21 | | 3 | -1 | 22 | | 3 | 0 | 23 | | 3 | 1 | 24 | | 3 | 2 | 25 | | 4 | -5 | 26 | | 4 | -4 | 27 | | 4 | -3 | 28 | | 4 | -2 | 29 | | 4 | -1 | 30 | | 4 | 0 | 31 | | 4 | 1 | 32 | | 4 | 2 | 33 | | 4 | 3 | 34 | | 5 | -5 | 35 | | 5 | -4 | 36 | | 5 | -3 | 37 | | 5 | -2 | 38 | | 5 | -1 | 39 | | 5 | 0 | 40 | | 5 | 1 | 41 | | 5 | 2 | 42 | | 5 | 3 | 43 | | 5 | 4 | 44 | | 6 | -5 | 45 | | 6 | -4 | 46 | | 6 | -3 | 47 | | 6 | -2 | 48 | | 6 | -1 | 49 | | 6 | 0 | 50 | | 6 | 1 | 51 | | 6 | 2 | 52 | | 6 | 3 | 53 | | 6 | 4 | 54 | | 7 | -5 | 55 | | 7 | -4 | 56 | | 7 | -3 | 57 | | 7 | -2 | 58 | | 7 | -1 | 59 | | 7 | 0 | 60 | | 7 | 1 | 61 | | 7 | 2 | 62 | | 7 | 3 | 63 | | 7 | 4 | 64 | | 8 | -5 | 65 | | 8 | -4 | 66 | | 8 | -3 | 67 | | 8 | -2 | 68 | | 8 | -1 | 69 | | 8 | 0 | 70 | | 8 | 1 | 71 | | 8 | 2 | 72 | | 8 | 3 | 73 | | 8 | 4 | 74 | | 9 | -5 | 75 | | 9 | -4 | 76 | | 9 | -3 | 77 | | 9 | -2 | 78 | | 9 | -1 | 79 | | 9 | 0 | 80 | | 9 | 1 | 81 | | 9 | 2 | 82 | | 9 | 3 | 83 | | 9 | 4 | 84 | +-----+-----+ 85 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/left_join_0.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: typecheck error: outer join predicate must be a conjunction of equalities 27 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/left_join_0.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) l LEFT JOIN range(start=>-5, end=>5) r ON l.i > r.i" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/left_join_0.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/left_join_0.out -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_0.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/order_by_limit_0.err -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_0.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) r ORDER BY i DESC LIMIT 4" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_0.out: -------------------------------------------------------------------------------- 1 | +---+ 2 | | i | 3 | +---+ 4 | | 9 | 5 | | 8 | 6 | | 7 | 7 | | 6 | 8 | +---+ 9 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_1.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/order_by_limit_1.err -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_1.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) r ORDER BY i ASC LIMIT 4" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/order_by_limit_1.out: -------------------------------------------------------------------------------- 1 | +---+ 2 | | i | 3 | +---+ 4 | | 1 | 5 | | 2 | 6 | | 3 | 7 | | 4 | 8 | +---+ 9 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_0.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: typecheck error: outer join predicate must be a conjunction of equalities 27 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_0.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) l OUTER JOIN range(start=>-5, end=>5) r ON l.i > r.i" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_0.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/outer_join_0.out -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_1.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/outer_join_1.err -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_1.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>-10, end=>3) l 2 | OUTER JOIN range(start=>-2, end=>6) r ON l.i = r.i 3 | OUTER JOIN range(start=>-20, end=>20) rr ON l.i + r.i = rr.i" 4 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/outer_join_1.out: -------------------------------------------------------------------------------- 1 | +--------+--------+--------+ 2 | | l.i | r.i | rr.i | 3 | +--------+--------+--------+ 4 | | | | -20 | 5 | | | | -19 | 6 | | | | -18 | 7 | | | | -17 | 8 | | | | -16 | 9 | | | | -15 | 10 | | | | -14 | 11 | | | | -13 | 12 | | | | -12 | 13 | | | | -11 | 14 | | | | -10 | 15 | | | | -9 | 16 | | | | -8 | 17 | | | | -7 | 18 | | | | -6 | 19 | | | | -5 | 20 | | | | -3 | 21 | | | | -1 | 22 | | | | 1 | 23 | | | | 3 | 24 | | | | 5 | 25 | | | | 6 | 26 | | | | 7 | 27 | | | | 8 | 28 | | | | 9 | 29 | | | | 10 | 30 | | | | 11 | 31 | | | | 12 | 32 | | | | 13 | 33 | | | | 14 | 34 | | | | 15 | 35 | | | | 16 | 36 | | | | 17 | 37 | | | | 18 | 38 | | | | 19 | 39 | | | 3 | | 40 | | | 4 | | 41 | | | 5 | | 42 | | -10 | | | 43 | | -9 | | | 44 | | -8 | | | 45 | | -7 | | | 46 | | -6 | | | 47 | | -5 | | | 48 | | -4 | | | 49 | | -3 | | | 50 | | -2 | -2 | -4 | 51 | | -1 | -1 | -2 | 52 | | 0 | 0 | 0 | 53 | | 1 | 1 | 2 | 54 | | 2 | 2 | 4 | 55 | +--------+--------+--------+ 56 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/right_join_0.err: -------------------------------------------------------------------------------- 1 | Usage: 2 | octosql [flags] 3 | octosql [command] 4 | 5 | Examples: 6 | octosql "SELECT * FROM myfile.json" 7 | octosql "SELECT * FROM mydir/myfile.csv" 8 | octosql "SELECT * FROM plugins.plugins" 9 | 10 | Available Commands: 11 | completion Generate the autocompletion script for the specified shell 12 | help Help about any command 13 | plugin 14 | 15 | Flags: 16 | --describe Describe query output schema. 17 | --explain int Describe query output schema. 18 | -h, --help help for octosql 19 | --optimize Whether OctoSQL should optimize the query. (default true) 20 | -o, --output string Output format to use. Available options are live_table, batch_table, csv, json and stream_native. (default "live_table") 21 | --profile string Enable profiling of the given type: cpu, memory, trace. 22 | -v, --version version for octosql 23 | 24 | Use "octosql [command] --help" for more information about a command. 25 | 26 | Error: typecheck error: outer join predicate must be a conjunction of equalities 27 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/right_join_0.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start=>1, end=>10) l RIGHT JOIN range(start=>-5, end=>5) r ON l.i > r.i" 2 | -------------------------------------------------------------------------------- /tests/scenarios/nodes/right_join_0.out: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/nodes/right_join_0.out -------------------------------------------------------------------------------- /tests/scenarios/objects/explode.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/objects/explode.err -------------------------------------------------------------------------------- /tests/scenarios/objects/explode.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT field1, field3->* FROM fixtures/test.json" -o json 2 | -------------------------------------------------------------------------------- /tests/scenarios/objects/explode.out: -------------------------------------------------------------------------------- 1 | {"field1":"value","field4":"eulav","field5":24,"field6":null} 2 | {"field1":"value","field4":null,"field5":"eulav","field6":"value"} 3 | -------------------------------------------------------------------------------- /tests/scenarios/objects/fixtures/test.json: -------------------------------------------------------------------------------- 1 | {"field1": "value", "field2": 42, "field3": {"field4": "eulav", "field5": 24}} 2 | {"field1": "value", "field2": 42, "field3": {"field5": "eulav", "field6": "value"}} 3 | -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/objects/varying_object_fields.err -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM fixtures/test.json" --output stream_native 2 | -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields.out: -------------------------------------------------------------------------------- 1 | {+0001-01-01T00:00:00Z| 'value', 42, { 'eulav', 24, } |} 2 | {+0001-01-01T00:00:00Z| 'value', 42, { , 'eulav', 'value' } |} 3 | -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields_describe.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/objects/varying_object_fields_describe.err -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields_describe.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM fixtures/test.json" --output stream_native --describe 2 | -------------------------------------------------------------------------------- /tests/scenarios/objects/varying_object_fields_describe.out: -------------------------------------------------------------------------------- 1 | {+0001-01-01T00:00:00Z| 'field1', 'String', false |} 2 | {+0001-01-01T00:00:00Z| 'field2', 'Float', false |} 3 | {+0001-01-01T00:00:00Z| 'field3', '{field4: NULL | String; field5: Float | String; field6: NULL | String}', false |} 4 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/csv/types.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/outputs/csv/types.err -------------------------------------------------------------------------------- /tests/scenarios/outputs/csv/types.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT 42, 2 | 42.42, 42.42424242424242, float(42), 3 | true, false, 4 | null, 5 | 'test' as hello, 6 | INTERVAL 5 HOURS + INTERVAL 32 MINUTES + INTERVAL 42 SECONDS" -ocsv 7 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/csv/types.out: -------------------------------------------------------------------------------- 1 | col_0,col_1,col_2,col_3,col_4,col_5,col_6,hello,col_8 2 | 42,42.42,42.42424242424242,42,true,false,,test,5h32m42s 3 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/fixtures/simple.json: -------------------------------------------------------------------------------- 1 | {"field1": "value", "field2": 42, "field3": {"field4": "eulav", "field5": 24}, "field7": [42, 43, {"field8": "value8"}]} 2 | {"field1": "value", "field2": 42, "field3": {"field5": "eulav", "field6": "value"}} 3 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/fixtures/union.json: -------------------------------------------------------------------------------- 1 | {"id": 0, "value": 42} 2 | {"id": 0, "value": "42"} 3 | {"id": 0, "value": [42, "42"]} 4 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/simple.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/outputs/json/simple.err -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/simple.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM fixtures/simple.json" --output json 2 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/simple.out: -------------------------------------------------------------------------------- 1 | {"field1":"value","field2":42,"field3":{"field4":"eulav","field5":24,"field6":null},"field7":[42,43,{"field8":"value8"}]} 2 | {"field1":"value","field2":42,"field3":{"field4":null,"field5":"eulav","field6":"value"},"field7":null} 3 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/union.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/outputs/json/union.err -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/union.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM fixtures/union.json" --output json 2 | -------------------------------------------------------------------------------- /tests/scenarios/outputs/json/union.out: -------------------------------------------------------------------------------- 1 | {"id":0,"value":42} 2 | {"id":0,"value":"42"} 3 | {"id":0,"value":[42,"42"]} 4 | -------------------------------------------------------------------------------- /tests/scenarios/simple/select_star_from_range.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/simple/select_star_from_range.err -------------------------------------------------------------------------------- /tests/scenarios/simple/select_star_from_range.in: -------------------------------------------------------------------------------- 1 | octosql "SELECT * FROM range(start => 1, end => 10) r" --output stream_native 2 | -------------------------------------------------------------------------------- /tests/scenarios/simple/select_star_from_range.out: -------------------------------------------------------------------------------- 1 | {+0001-01-01T00:00:00Z| 1 |} 2 | {+0001-01-01T00:00:00Z| 2 |} 3 | {+0001-01-01T00:00:00Z| 3 |} 4 | {+0001-01-01T00:00:00Z| 4 |} 5 | {+0001-01-01T00:00:00Z| 5 |} 6 | {+0001-01-01T00:00:00Z| 6 |} 7 | {+0001-01-01T00:00:00Z| 7 |} 8 | {+0001-01-01T00:00:00Z| 8 |} 9 | {+0001-01-01T00:00:00Z| 9 |} 10 | -------------------------------------------------------------------------------- /tests/scenarios/stdin/json.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/stdin/json.err -------------------------------------------------------------------------------- /tests/scenarios/stdin/json.in: -------------------------------------------------------------------------------- 1 | echo '{"hello": "world"}' | octosql "SELECT * FROM stdin.json" 2 | -------------------------------------------------------------------------------- /tests/scenarios/stdin/json.out: -------------------------------------------------------------------------------- 1 | +---------+ 2 | | hello | 3 | +---------+ 4 | | 'world' | 5 | +---------+ 6 | -------------------------------------------------------------------------------- /tests/scenarios/stdin/seqsum.err: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/cube2222/octosql/4a66fe03bda9a6e0bbafe46bbef5c7a32645e2e8/tests/scenarios/stdin/seqsum.err -------------------------------------------------------------------------------- /tests/scenarios/stdin/seqsum.in: -------------------------------------------------------------------------------- 1 | seq 100 | octosql "SELECT SUM(int(text)) FROM lines.stdin" -ojson 2 | -------------------------------------------------------------------------------- /tests/scenarios/stdin/seqsum.out: -------------------------------------------------------------------------------- 1 | {"sum":5050} 2 | --------------------------------------------------------------------------------