executor

package
v1.5.3-rc4 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jan 15, 2026 License: Apache-2.0 Imports: 78 Imported by: 0

Documentation

Overview

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. nolint

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. nolint

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. nolint

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License. nolint

Copyright 2022 Huawei Cloud Computing Technologies Co., Ltd.

Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. You may obtain a copy of the License at

http://www.apache.org/licenses/LICENSE-2.0

Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the specific language governing permissions and limitations under the License.

Index

Constants

View Source
const (
	TrimPercent     = 0.1  // Percentage of extreme values dropped at each tail when computing trimmed mean
	Q1Percentile    = 0.25 // 1st quartile used to build the IQR
	Q3Percentile    = 0.75 // 3rd quartile used to build the IQR
	WeightMedian    = 0.5  // Weight of median absolute deviation in the final synthetic score
	WeightTrimMean  = 0.3  // Weight of trimmed-mean absolute deviation in the final synthetic score
	WeightIQRCenter = 0.2  // Weight of IQR-center absolute deviation in the final synthetic score

	SlopeThreshold    = 0.1 // Minimum slope (change per unit time) required to consider an uptrend or downtrend significant
	AbsoluteThreshold = 5.0 // Minimum absolute difference between actual and predicted values that triggers a level-based alert
	TrendScore        = 1.1 // Boost factor applied to the final score when a significant trend is detected
)
View Source
const (
	DefaultClusterNum = 100
	MaxClusterNum     = 100000
)
View Source
const (
	OGSketchInsert     = "ogsketch_insert"
	OGSketchMerge      = "ogsketch_merge"
	OGSketchPercentile = "ogsketch_percentile"
	PercentileApprox   = "percentile_approx"
	PercentileOGSketch = "percentile_ogsketch"
)
View Source
const (
	Algo castorArgIdx
	Conf
	AlgoType
)
View Source
const (
	ML_INFER_UDF_NAME = "model_infer"
	UDF_DETECT        = "_udf_detect"
)
View Source
const (
	NOT_READY = "not_ready"
	RUNNING   = "running"
	READY     = "ready"
)
View Source
const (
	Kind     string = "kind"
	Uid      string = "uid"
	IncomeOp string = "income"
	OutGoOp  string = "outgo"
)
View Source
const (
	LenOfValuers  = 2 // MathValuer + MapValuer
	ZeroTimeStamp = 0
)
View Source
const (
	SlowFuncIdx   int = 0
	FastFuncIdx   int = 1
	FuncPathCount int = 2
)
View Source
const (
	IncAggChunkCacheSize int64 = 100 * 1024 * 1024
	IncAggChunkCacheTTL        = 10 * time.Minute
)
View Source
const (
	IncHashAggChunkCacheSize int64 = 100 * 1024 * 1024
	IncHashAggChunkCacheTTL        = 10 * time.Minute
	MaxRowsInChunk           int   = 32768
)
View Source
const (
	ID          = "id"
	Name        = "name"
	Level       = "level"
	RuleID      = "rule_id"
	EntityID    = "entity_id"
	Type        = "type"
	Annotations = "annotations"
)
View Source
const (
	// anomaly
	ANOMALY    = "anomaly"    // event type
	Timestamps = "timestamps" // required field

	// alarm
	ALARM = "alarm" // event type

	// event
	EVENT     = "event"       // event type
	CreatedTS = "create_time" // required field

	// alarm & event
	StartTS = "start_time" // optional field for EVENT, required field for ALARM
	EndTS   = "end_time"   // optional
)

RCA assumes that the following fields exist in AnomalyEvent.Annotations.

View Source
const (
	META       = "metadata"
	CoreEntity = "core_entity_id"
)

RCA assumes that the following fields exist in AlgoParam.Task.

View Source
const (
	UnknownMessage byte = iota
	ErrorMessage
	FinishMessage
	ChunkResponseMessage
	AnalyzeResponseMessage
	QueryMessage
	IncQueryFinishMessage
	MessageEof
)
View Source
const (
	LeftSide = iota
	RightSide
)
View Source
const (
	BALANCED_DIFF = 5
	Aborted       = "aborted"
)
View Source
const (
	RowSizeBytes   int = 24 // Estimated size of each row
	TupleSizeBytes int = 16 // Estimated size of each tuple
)
View Source
const AggBufChunkNum = 2
View Source
const AlignPingPongChunkNum = 4
View Source
const (
	BinOpTransformName = "BinOpTransform"
)
View Source
const (
	CTETransformName = "CTETransform"
)
View Source
const CircularChunkNum = 4

CircularChunkNum Do not Modify. CircularChunks consists of one input chunk, two cached chunks(agg/fill and cached channel) and one chunk(cycle)

View Source
const DefaultTime = 0
View Source
const EulerNumber float64 = 2.7182818284
View Source
const FillBufChunkNum = 2
View Source
const GetTopoUidFuncName = "ms_get_node_id"
View Source
const HashAggTransformBufCap = 8192
View Source
const (
	HashJoinTransformName = "HashJoinTransform"
)
View Source
const HashMergeTransformBufCap = 1024
View Source
const (
	INVALID_NUMBER int = -1
)
View Source
const (
	InTransformName = "InTransform"
)
View Source
const LLMFuncPrefix = "llm_"
View Source
const MersennePower int = 31
View Source
const NilMarkByte = 0xFF
View Source
const OrderbyChunkNum = 6

OrderbyChunkNum consists of 3 orderbyTransform chunks and 3 hash_agg_transform chunks orderby_transform: 1 currChunk,1 bufChunk and 1 outCh hash_agg_transform: 1 inCh, 1 inputChunk and 1 bufChunk

View Source
const (
	PORT_CHAN_SIZE int = 1
)
View Source
const (
	PromInstantVectorTransformName = "PromInstantVectorTransform"
)
View Source
const (
	PromRangeVectorTransformName = "PromRangeVectorTransform"
)
View Source
const (
	RESULT_NAME string = "result"
)
View Source
const RcaFuncName = "ms_rca"
View Source
const SlidingWindowBufChunkNum = 2
View Source
const (
	SortMergeJoinTransformName = "SortMergeJoinTransform"
)
View Source
const TimeSuffix string = ".time"
View Source
const UDAFMaxRow = 100000

Variables

View Source
var DefaultDelta float64 = 1e-3
View Source
var DefaultEps float64 = DefaultPhi / 10
View Source
var DefaultPhi float64 = 3e-4
View Source
var DefaultSigma float64 = 1e-3
View Source
var GetColValsFn map[influxql.DataType]func(col Column, bmStart, bmEnd int, ckLen int, dst []interface{}) []interface{}
View Source
var IgnoreEmptyTag = false
View Source
var LLMFactoryInstance = map[string]LLMFunc{}
View Source
var MatchPlanFunc []func(hybridqp.Catalog) PlanType
View Source
var NotAggOnSeries = make(map[string]bool)
View Source
var OneShardStorePlanTemplate []*PlanTemplate
View Source
var PlanTypes []PlanType
View Source
var RegisterTable sync.Map
View Source
var RowSize = 1024
View Source
var SkipDefault = interface{}(0)
View Source
var SqlPlanTemplate []*PlanTemplate
View Source
var StorePlanTemplate []*PlanTemplate
View Source
var TemplateSql []string = []string{
	"SELECT max(f1) from mst WHERE (tag1 = 'tag1val') and time > 1 and time < 2 group by time(1ns)",
	"SELECT max(f1) from mst WHERE time < 2 group by time(1ns) limit 1",
	"SELECT * from mst where f1 > 1 and (tag1 = 'tag1val') and time > 1 and time < 2",
	"SELECT last(*) from mst group by tag1",
	"SELECT f1 from mst where tag1 = 'tag1val' order by time limit 1",
	"SELECT max(f1) from mst WHERE (tag1 = 'tag1val') and time > 1 and time < 2 group by time(1ns) fill(none)",
}

only for OptimizeAgg or call PreAggregateCallMapping, some calls like top/bottom/percentile_ogsketch which cancallspushdown but not use template plan

Functions

func ADDiffAbsReduce added in v1.5.0

func ADDiffAbsReduce[T util.NumberOnly](si *SliceItem[T]) (int, int64, float64, bool)

ADDiffAbsReduce computes the anomaly score of a data sequence to quantify the distribution difference between its first half and second half.

The calculation process: 1. Compare key statistical features (median, trimmed mean, interquartile center) between the first half and second half of the data sequence. 2. Synthesize a single-index anomaly score by weighted processing of the above feature differences.

Interpretation: A higher score indicates a greater distribution difference between the two segments, suggesting a higher possibility of anomaly in the data sequence.

func ADDiffTimeReduce

func ADDiffTimeReduce[T util.NumberOnly](si *SliceItem[T]) (index int, time int64, value float64, isNil bool)

func ADRMseExtReduce added in v1.5.0

func ADRMseExtReduce[T util.NumberOnly](si *SliceItem[T]) (int, int64, float64, bool)

func ADSlopeScoreReduce added in v1.5.0

func ADSlopeScoreReduce[T util.NumberOnly](si *SliceItem[T]) (int, int64, float64, bool)

ADSlopeScoreReduce assesses whether a data sequence has abnormal trends by integrating two key indicators: the overall trend slope (linear change trend) and the median difference between the first half and second half (magnitude of sudden change).

The function outputs a quantified trend anomaly score, which reflects both the degree of deviation in the data trend and its direction (positive/negative).

func AbsentMerge added in v1.3.0

func AbsentMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func AbsentReduce added in v1.3.0

func AbsentReduce(c Chunk, values []int64, ordinal, start, end int) (int, int64, bool)

func AbsentWithOutDataAlive added in v1.4.0

func AbsentWithOutDataAlive(opt *query.ProcessorOptions, outChunk Chunk)

func AbsoluteDiffFunc added in v1.3.0

func AbsoluteDiffFunc[T util.NumberOnly](prev, curr T) T

func AddRule added in v1.0.0

func AddRule(mapDescToRule map[string]OptRule, rule OptRule) (bool, map[string]OptRule)

func AdjustNils added in v1.0.0

func AdjustNils(dst Column, src Column, low int, high int)

func AppendBooleanValue added in v1.5.0

func AppendBooleanValue(outColumn Column, value *ValueKeeper)

func AppendFloatValue added in v1.5.0

func AppendFloatValue(outColumn Column, value *ValueKeeper)

func AppendIntegerValue added in v1.5.0

func AppendIntegerValue(outColumn Column, value *ValueKeeper)

func AppendRowValue added in v1.0.0

func AppendRowValue(column Column, value interface{})

func AppendStringValue added in v1.5.0

func AppendStringValue(outColumn Column, value *ValueKeeper)

func AuxTagKeyHandler added in v1.5.0

func AuxTagKeyHandler(auxTagIdx int, tags *influx.PointTags, rec *record.Record, start, end int)

func B2ui64 added in v1.4.0

func B2ui64(b bool) uint64

func BehindDiffFunc added in v1.3.0

func BehindDiffFunc[T util.NumberOnly](prev, curr T) T

func BinOpADD added in v1.3.0

func BinOpADD(lVal, rVal float64) (float64, bool)

func BinOpATAN2 added in v1.4.0

func BinOpATAN2(lVal, rVal float64) (float64, bool)

func BinOpDIV added in v1.3.0

func BinOpDIV(lVal, rVal float64) (float64, bool)

func BinOpEQLC added in v1.3.0

func BinOpEQLC(lVal, rVal float64) (float64, bool)

func BinOpGTE added in v1.3.0

func BinOpGTE(lVal, rVal float64) (float64, bool)

func BinOpGTR added in v1.3.0

func BinOpGTR(lVal, rVal float64) (float64, bool)

func BinOpLSS added in v1.3.0

func BinOpLSS(lVal, rVal float64) (float64, bool)

func BinOpLTE added in v1.3.0

func BinOpLTE(lVal, rVal float64) (float64, bool)

func BinOpMOD added in v1.3.0

func BinOpMOD(lVal, rVal float64) (float64, bool)

func BinOpMUL added in v1.3.0

func BinOpMUL(lVal, rVal float64) (float64, bool)

func BinOpNEQ added in v1.3.0

func BinOpNEQ(lVal, rVal float64) (float64, bool)

func BinOpPOW added in v1.3.0

func BinOpPOW(lVal, rVal float64) (float64, bool)

func BinOpSUB added in v1.3.0

func BinOpSUB(lVal, rVal float64) (float64, bool)

func BooleanAscendingAuxHelper

func BooleanAscendingAuxHelper(x, y Column, i, j int) (bool, bool)

func BooleanAuxHelpFunc

func BooleanAuxHelpFunc(input, output Column, rowIdx ...int)

func BooleanDescendingAuxHelper

func BooleanDescendingAuxHelper(x, y Column, i, j int) (bool, bool)

func BooleanFirstMerge

func BooleanFirstMerge(prevPoint, currPoint *Point[bool])

func BooleanFirstReduce

func BooleanFirstReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanFirstTimeColFastReduce

func BooleanFirstTimeColFastReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanFirstTimeColReduce

func BooleanFirstTimeColReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanFirstTimeColSlowReduce

func BooleanFirstTimeColSlowReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanLastMerge

func BooleanLastMerge(prevPoint, currPoint *Point[bool])

func BooleanLastReduce

func BooleanLastReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanLastTimeColFastReduce

func BooleanLastTimeColFastReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanLastTimeColReduce

func BooleanLastTimeColReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanLastTimeColSlowReduce

func BooleanLastTimeColSlowReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanMaxMerge

func BooleanMaxMerge(prevPoint, currPoint *Point[bool])

func BooleanMaxReduce

func BooleanMaxReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BooleanMinMerge

func BooleanMinMerge(prevPoint, currPoint *Point[bool])

func BooleanMinReduce

func BooleanMinReduce(c Chunk, values []bool, ordinal, start, end int) (int, bool, bool)

func BottomCmpByTimeReduce added in v1.3.0

func BottomCmpByTimeReduce[T util.NumberOnly](a, b *PointItem[T]) bool

func BottomCmpByValueReduce added in v1.3.0

func BottomCmpByValueReduce[T util.NumberOnly](a, b *PointItem[T]) bool

func BottomCmpByValueReduceProm added in v1.4.0

func BottomCmpByValueReduceProm[T util.NumberOnly](a, b *PointItem[T]) bool

func BuildBinOpQueryPlan added in v1.3.0

func BuildBinOpQueryPlan(ctx context.Context, qc query.LogicalPlanCreator, stmt *influxql.SelectStatement, schema *QuerySchema) (hybridqp.QueryNode, error)

func BuildCTELogicalPlan added in v1.5.0

func BuildCTELogicalPlan(ctx context.Context, outerSchema *QuerySchema, cte *influxql.CTE) (hybridqp.QueryNode, error)

func BuildHeuristicPlanner added in v1.0.0

func BuildHeuristicPlanner() hybridqp.Planner

func BuildHeuristicPlannerForStore added in v1.0.0

func BuildHeuristicPlannerForStore() hybridqp.Planner

func BuildInConditionPlan added in v1.1.0

func BuildInConditionPlan(ctx context.Context, outerQc query.LogicalPlanCreator, outerStmt *influxql.SelectStatement, outerSchema *QuerySchema,
	builder *LogicalPlanBuilderImpl) (hybridqp.QueryNode, error)

func BuildJoinQueryPlan added in v1.5.0

func BuildJoinQueryPlan(ctx context.Context, qc query.LogicalPlanCreator, stmt *influxql.SelectStatement, schema *QuerySchema) (hybridqp.QueryNode, error)

func BuildNodeExchange added in v1.5.0

func BuildNodeExchange(ctx context.Context, builder LogicalPlanBuilder, queryPlan hybridqp.QueryNode) error

func BuildSources added in v1.3.0

func BuildSources(ctx context.Context, qc query.LogicalPlanCreator, sources influxql.Sources, schema *QuerySchema, outerBinOp bool) (hybridqp.QueryNode, error)

func CalcChange added in v1.4.0

func CalcChange(preValues []float64, currValues []float64) float64

func CalcHoltWinters added in v1.4.0

func CalcHoltWinters(preV []float64, curV []float64, sf, tf float64) float64

func CalcLastTwoPoints added in v1.4.0

func CalcLastTwoPoints(preTimes []int64, currTimes []int64, preValues []float64, currValues []float64) (int64, int64, float64, float64)

func CalcLog2 added in v1.4.0

func CalcLog2(value float64) int

func CalcMad added in v1.4.0

func CalcMad(values []float64) float64

CalcMad calculates the Mean Absolute Deviation (MAD) of a given dataset. Parameters:

values: a slice of float64 numbers representing the dataset to calculate the MAD.

Return value:

The calculated MAD.

Note: This function modifies the input slice by changing its values in place.

func CalcMad2 added in v1.4.0

func CalcMad2(preValues []float64, currValues []float64) float64

func CalcQuantile added in v1.4.0

func CalcQuantile(q float64, values []float64) float64

CalcQuantile calculates the q-quantile of a given dataset. Parameters:

q: the quantile to calculate (should be between 0 and 1).
values: a slice of float64 numbers representing the dataset.

Return value:

The calculated q-quantile.

Note: This function modifies the input slice by sorting its elements.

If q is less than 0 or greater than 1, it returns positive or negative infinity. If the input slice is empty or q is NaN, it returns NaN.

func CalcQuantile2 added in v1.4.0

func CalcQuantile2(q float64, preValues []float64, currValues []float64) float64

func CalcReduceResult added in v1.3.0

func CalcReduceResult(prevT, currT []int64, prevV, currV []float64, isCounter bool) (int64, int64, float64, float64, float64)

func CalcResets added in v1.4.0

func CalcResets(preV []float64, curV []float64) float64

func ChangeCallExprForTimestamp added in v1.3.0

func ChangeCallExprForTimestamp(call *influxql.Call)

func ChangeOpsForTimestamp added in v1.3.0

func ChangeOpsForTimestamp(ops []hybridqp.ExprOptions) []hybridqp.ExprOptions

func ChunkToArrowRecords added in v1.0.0

func ChunkToArrowRecords(chunks []Chunk, taskId string, args []influxql.Expr) ([]arrow.Record, error)

ChunkToArrowRecords must release record after use

func ChunkToLastRowCache added in v1.5.0

func ChunkToLastRowCache(chunk Chunk, schema hybridqp.Catalog) *lastrowcache.CacheValue

func ColumnMapping added in v1.5.0

func ColumnMapping(chunk Chunk, colMap map[influxql.Expr]influxql.VarRef)

func ColumnStringValue added in v1.2.0

func ColumnStringValue(c Column, rowLoc int) string

func CompareBreakPoint

func CompareBreakPoint(name string, time int64, tag ChunkTags, b *BreakPoint, opt query.ProcessorOptions) bool

func CompareSortedAppendBreakPoint

func CompareSortedAppendBreakPoint(item Item, in int, tag ChunkTags, b *SortedBreakPoint, opt query.ProcessorOptions) bool

func CompareSortedMergeBreakPoint

func CompareSortedMergeBreakPoint(item Item, in int, tag ChunkTags, b *SortedBreakPoint, opt *query.ProcessorOptions) bool

func Connect

func Connect(from Port, to Port) error

func ContainDim

func ContainDim(des []string, src string) bool

func ContainsSameTagset added in v1.4.0

func ContainsSameTagset(chunk Chunk) bool

func ConvertToLabels added in v1.5.0

func ConvertToLabels(pts *influx.PointTags, dst []byte) []byte

func CopyArrowRecordToChunk added in v1.0.0

func CopyArrowRecordToChunk(r arrow.Record, c Chunk, fields map[string]struct{}) error

func CopyCastorADArrowRecordToChunk added in v1.5.0

func CopyCastorADArrowRecordToChunk(r arrow.Record, c Chunk, fields map[string]struct{}) error

func CountMerge added in v1.3.0

func CountMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func CountReduce added in v1.3.0

func CountReduce(c Chunk, values []int64, ordinal, start, end int) (int, int64, bool)

func DecodeBytes

func DecodeBytes(bytes []byte) []byte

func DoCalcChange added in v1.4.0

func DoCalcChange(curV []float64, prev float64, changes int) (float64, int)

func DoCalcResets added in v1.4.0

func DoCalcResets(prevV []float64, prev float64, resets int) (float64, int)

func DummyPipelineExecutorWithQuerySchema added in v1.5.0

func DummyPipelineExecutorWithQuerySchema(s query.PreparedStatement) (hybridqp.Executor, error)

for arrow flight protocel getFlightInfo & getSchema query, the query schema is returned instead of real data, therefore a dummy pipelineExecutor which will not be executed is created to carry just the query schema

func EnableFileCursor

func EnableFileCursor(en bool)

func ExpandColumnOffsets added in v1.2.0

func ExpandColumnOffsets(col Column, stringBytes []byte, offsets []uint32) ([]byte, []uint32)

func ExprHasCall added in v1.5.0

func ExprHasCall(expr influxql.Expr) bool

func ExprWalk

func ExprWalk(v influxql.Fields, node influxql.Node)

func FilterPushDown added in v1.5.0

func FilterPushDown(opt *query.ProcessorOptions, schema *QuerySchema)

func FirstMerge added in v1.3.0

func FirstMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func FirstReduce added in v1.3.0

func FirstReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func FirstTimeColFastReduce added in v1.3.0

func FirstTimeColFastReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func FirstTimeColMerge added in v1.3.0

func FirstTimeColMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func FirstTimeColReduce added in v1.3.0

func FirstTimeColReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func FirstTimeColSlowReduce added in v1.3.0

func FirstTimeColSlowReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func Float64AscendingAuxHelper

func Float64AscendingAuxHelper(x, y Column, i, j int) (bool, bool)

func Float64DescendingAuxHelper

func Float64DescendingAuxHelper(x, y Column, i, j int) (bool, bool)

func Float64Equal added in v1.4.0

func Float64Equal(a, b float64) bool

func FloatAuxHelpFunc

func FloatAuxHelpFunc(input, output Column, rowIdx ...int)

func FloatCountPromMerge added in v1.3.0

func FloatCountPromMerge(prevPoint, currPoint *Point[float64])

func FloatCountPromReduce added in v1.3.0

func FloatCountPromReduce(c Chunk, values []float64, ordinal, start, end int) (int, float64, bool)

func FrontDiffFunc added in v1.3.0

func FrontDiffFunc[T util.NumberOnly](prev, curr T) T

func GetAlignValue added in v1.5.0

func GetAlignValue[T util.BasicType](in Column, values []T, start int, end int) (T, bool)

func GetEnableFileCursor

func GetEnableFileCursor() bool

func GetFieldInfo added in v1.0.0

func GetFieldInfo(chunks []Chunk) (map[string]map[string]*fieldInfo, *errno.Error)

GetFieldInfo return every series' field info in form of 2 level map. Level 1 key is series key, level 2 key is field name.

func GetInnerDimensions added in v1.0.0

func GetInnerDimensions(outer, inner []string) []string

func GetMaxNumsOfGroups added in v1.2.0

func GetMaxNumsOfGroups() int

func GetOrdinal added in v1.3.0

func GetOrdinal(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (int, int)

func GetPlanSliceByTree added in v1.1.0

func GetPlanSliceByTree(plan hybridqp.QueryNode) []hybridqp.QueryNode

func GetPlanSliceByTreeDFS added in v1.1.0

func GetPlanSliceByTreeDFS(plan hybridqp.QueryNode, dstPlan *[]hybridqp.QueryNode) error

func GetPromTimeFuncInstance added in v1.3.0

func GetPromTimeFuncInstance() map[string]PromTimeFunction

func GetStorePlanTemplate added in v1.1.0

func GetStorePlanTemplate(shardNum int, planType PlanType) []hybridqp.QueryNode

func GetType

func GetType(i interface{}) string

func GetTypeName

func GetTypeName(i interface{}) string

func GroupMerge added in v1.3.0

func GroupMerge(prevPoint, currPoint *Point[float64])

func GroupReduce added in v1.3.0

func GroupReduce(c Chunk, values []float64, ordinal, start, end int) (int, float64, bool)

func HandleStandardAgg added in v1.5.0

func HandleStandardAgg(node *LogicalHashAgg)

HandleStandardAgg processes nodes for standard plans

func HandleUnifyPlanAgg added in v1.5.0

func HandleUnifyPlanAgg(node *LogicalHashAgg)

HandleUnifyPlanAgg processes nodes for unify plan

func IndexUnion

func IndexUnion(index1, index2 []int) []int

func InitLocalStoreTemplatePlan added in v1.2.0

func InitLocalStoreTemplatePlan()

func InitMstName added in v1.0.0

func InitMstName(item *AppendHeapItems) string

func InitNagtPool added in v1.5.0

func InitNagtPool(cap int)

func IntegerAscendingAuxHelper

func IntegerAscendingAuxHelper(x, y Column, i, j int) (bool, bool)

func IntegerAuxHelpFunc

func IntegerAuxHelpFunc(input, output Column, rowIdx ...int)

func IntegerDescendingAuxHelper

func IntegerDescendingAuxHelper(x, y Column, i, j int) (bool, bool)

func IntervalIndexGen

func IntervalIndexGen(ck Chunk, opt *query.ProcessorOptions)

func IrateFastReduce added in v1.3.0

func IrateFastReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func IrateFinalReduce added in v1.3.0

func IrateFinalReduce[T util.NumberOnly](ft int64, st int64, fv T, sv T,
	interval *hybridqp.Interval) (float64, bool)

func IrateMerge added in v1.3.0

func IrateMerge[T util.NumberOnly](prevPoints [2]*Point[T], interval *hybridqp.Interval) (float64, bool)

func IrateMiddleReduce added in v1.3.0

func IrateMiddleReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func IrateSlowReduce added in v1.3.0

func IrateSlowReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func IrateUpdate added in v1.3.0

func IrateUpdate[T util.NumberOnly](prevPoints, currPoints [2]*Point[T])

func IsEnableFileCursor added in v1.3.0

func IsEnableFileCursor(schema hybridqp.Catalog) bool

func IsOneShardExchange added in v1.5.0

func IsOneShardExchange(exchange Exchange) bool

IsOneShardExchange used for serially scheduling shards in one PT.

func IsSubTreeEqual

func IsSubTreeEqual(node hybridqp.QueryNode, comparedNode hybridqp.QueryNode) bool

func KahanSumInc added in v1.4.0

func KahanSumInc(inc, sum, c float64) (newSum, newC float64)

func LastMerge added in v1.3.0

func LastMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func LastReduce added in v1.3.0

func LastReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func LastRowFieldKey added in v1.5.0

func LastRowFieldKey(field influxql.Expr, mapping map[influxql.Expr]influxql.VarRef) string

LastRowFieldKey The input `field` is the value in the `mapping`. The function is used to recursively search for the `VarRef key` based on the `field`.

func LastTimeColFastReduce added in v1.3.0

func LastTimeColFastReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func LastTimeColMerge added in v1.3.0

func LastTimeColMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func LastTimeColReduce added in v1.3.0

func LastTimeColReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func LastTimeColSlowReduce added in v1.3.0

func LastTimeColSlowReduce[T util.ExceptBool](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func LengthOfPrefixForField added in v1.5.0

func LengthOfPrefixForField(name string, aliases []string) int

func Marshal added in v1.0.0

func Marshal(plan LogicalPlaner, extMarshal func(pb *internal.QueryNode), nodes ...hybridqp.QueryNode) ([]byte, error)

func MarshalBinary added in v1.0.0

func MarshalBinary(q hybridqp.QueryNode) ([]byte, error)

func MarshalPtQuerys added in v1.2.0

func MarshalPtQuerys(ptQuerys []PtQuery) []*proto2.PtQuery

func MarshalQueryNode

func MarshalQueryNode(node hybridqp.QueryNode) ([]byte, error)

MarshalQueryNode The schema of all nodes are the same, only one codec is required. |8 byte schema size|schema buffer|node buffer|

func MarshalShardInfos added in v1.2.0

func MarshalShardInfos(shardInfos []ShardInfo) []*proto2.ShardInfo

func MatchSortMergeJoin added in v1.5.0

func MatchSortMergeJoin(join *influxql.Join, schema hybridqp.Catalog) (bool, error)

MatchSortMergeJoin used to check whether the sort merge join algorithm is matched. Requirements for the sort merge join algorithm: 1. Only join on tag is supported. Join on field is not supported. 2. The join key belongs to the group by subset. 3. Join on join equal value comparison.

func MaxMerge added in v1.3.0

func MaxMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func MaxPromMerge added in v1.3.0

func MaxPromMerge(prevPoint, currPoint *Point[float64])

func MaxPromReduce added in v1.3.0

func MaxPromReduce(c Chunk, values []float64, ordinal, start, end int) (int, float64, bool)

func MaxReduce added in v1.3.0

func MaxReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func MergeRecordColToChunk added in v1.5.0

func MergeRecordColToChunk(r arrow.Record, c Chunk) error

func MinMerge added in v1.3.0

func MinMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func MinPromMerge added in v1.3.0

func MinPromMerge(prevPoint, currPoint *Point[float64])

func MinPromReduce added in v1.3.0

func MinPromReduce(c Chunk, values []float64, ordinal, start, end int) (int, float64, bool)

func MinReduce added in v1.3.0

func MinReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func NewAggFunc added in v1.1.0

func NewAggFunc(aggType AggFuncType, fn NewAggOperator, inIdx int, outIdx int, p any) *aggFunc

func NewAnalyzeResponse

func NewAnalyzeResponse(trace *tracing.Trace) *rpc.Message

func NewBoolSortEle added in v1.1.0

func NewBoolSortEle() sortEleMsg

func NewBooleanModeReduce

func NewBooleanModeReduce(BooleanSliceItem *BooleanSliceItem) (int, int64, float64, bool)

func NewCallLLMApiFunc added in v1.5.0

func NewCallLLMApiFunc(ctx context.Context, client openai.Client, modelName string, temperature float64, maxTokens int, logger *logger.Logger) func(prompt string, question string) ([]string, error)

NewCallLLMApiFunc used to generate callLLMAPI that calls the LLM API for processing

func NewChunkResponse

func NewChunkResponse(chunk Chunk) *rpc.Message

func NewCountFunc added in v1.1.0

func NewCountFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewCountOperator added in v1.1.1

func NewCountOperator() aggOperator

func NewCountPromFunc added in v1.3.0

func NewCountPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewCountPromOperator added in v1.3.0

func NewCountPromOperator() aggOperator

func NewErrorMessage

func NewErrorMessage(errCode errno.Errno, err string) *rpc.Message

func NewFinishMessage

func NewFinishMessage(queryIndexState int32) *rpc.Message

func NewFirstBooleanOperator added in v1.1.1

func NewFirstBooleanOperator() aggOperator

func NewFirstFloatOperator added in v1.1.1

func NewFirstFloatOperator() aggOperator

func NewFirstFunc added in v1.1.0

func NewFirstFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewFirstIntegerOperator added in v1.1.1

func NewFirstIntegerOperator() aggOperator

func NewFirstStringOperator added in v1.1.1

func NewFirstStringOperator() aggOperator

func NewFloatSortEle added in v1.1.0

func NewFloatSortEle() sortEleMsg

func NewGroupPromFunc added in v1.4.0

func NewGroupPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewGroupPromOperator added in v1.4.0

func NewGroupPromOperator() aggOperator

func NewHeapFloatOperator added in v1.2.0

func NewHeapFloatOperator() aggOperator

func NewHeapFunc added in v1.2.0

func NewHeapFunc(inRowDataType, outRowDataType hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, sortIdx int, sortAsc bool) (*aggFunc, error)

func NewHeapIntegerOperator added in v1.2.0

func NewHeapIntegerOperator() aggOperator

func NewHeapParam added in v1.2.0

func NewHeapParam(topN int64, sortFuncs []func() sortEleMsg, sorKeyIdx []int, sortAsc []bool, m map[int]int) *heapParam

func NewHeapStringOperator added in v1.2.0

func NewHeapStringOperator() aggOperator

func NewHoltWintersReducer added in v1.0.0

func NewHoltWintersReducer(h, m int, includeFitData bool, interval time.Duration) *holtWintersReducer

func NewIncQueryFinishMessage added in v1.2.0

func NewIncQueryFinishMessage(isIncQuery, getFailed bool, queryID string, iterMaxNum int32, rowCount int64) *rpc.Message

func NewIntegerSortEle added in v1.1.0

func NewIntegerSortEle() sortEleMsg

func NewLastBooleanOperator added in v1.1.1

func NewLastBooleanOperator() aggOperator

func NewLastFloatOperator added in v1.1.1

func NewLastFloatOperator() aggOperator

func NewLastFunc added in v1.1.0

func NewLastFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewLastIntegerOperator added in v1.1.1

func NewLastIntegerOperator() aggOperator

func NewLastStringOperator added in v1.1.1

func NewLastStringOperator() aggOperator

func NewLocalStoreSqlPlanTypePool added in v1.2.0

func NewLocalStoreSqlPlanTypePool(planType PlanType) ([]hybridqp.QueryNode, error)

func NewMaxBooleanOperator added in v1.2.0

func NewMaxBooleanOperator() aggOperator

func NewMaxFloatOperator added in v1.1.1

func NewMaxFloatOperator() aggOperator

func NewMaxFunc added in v1.1.0

func NewMaxFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewMaxIntegerOperator added in v1.1.1

func NewMaxIntegerOperator() aggOperator

func NewMaxPromFunc added in v1.3.0

func NewMaxPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewMaxPromOperator added in v1.3.0

func NewMaxPromOperator() aggOperator

func NewMedianReduce added in v1.3.0

func NewMedianReduce[T util.NumberOnly](si *SliceItem[T]) (int, int64, float64, bool)

func NewMinBooleanOperator added in v1.2.0

func NewMinBooleanOperator() aggOperator

func NewMinFloatOperator added in v1.1.1

func NewMinFloatOperator() aggOperator

func NewMinFunc added in v1.1.0

func NewMinFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewMinIntegerOperator added in v1.1.1

func NewMinIntegerOperator() aggOperator

func NewMinPromFunc added in v1.3.0

func NewMinPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewMinPromOperator added in v1.3.0

func NewMinPromOperator() aggOperator

func NewModeReduce added in v1.3.0

func NewModeReduce[T util.ExceptBool](si *SliceItem[T]) (int, int64, float64, bool)

func NewOneShardStorePlanTypePool added in v1.1.0

func NewOneShardStorePlanTypePool(planType PlanType) []hybridqp.QueryNode

func NewPercentileFloatOperator added in v1.1.1

func NewPercentileFloatOperator() aggOperator

func NewPercentileFunc added in v1.1.0

func NewPercentileFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewPercentileIntegerOperator added in v1.1.1

func NewPercentileIntegerOperator() aggOperator

func NewPlanBySchemaAndSrcPlan added in v1.1.0

func NewPlanBySchemaAndSrcPlan(schema hybridqp.Catalog, srcPlan []hybridqp.QueryNode, eTrait []hybridqp.Trait, localStore bool) (hybridqp.QueryNode, error)

func NewPreparedStatement

func NewPreparedStatement(stmt *influxql.SelectStatement, opt hybridqp.Options,
	shards interface {
		query.LogicalPlanCreator
		io.Closer
	}, columns []string, MaxPointN int, now time.Time) *preparedStatement

func NewPrevValuesFunc

func NewPrevValuesFunc(rowDataType hybridqp.RowDataType) []func(prev Chunk, prevValues []interface{}, ordinal int)

func NewPrevWindowFunc

func NewPrevWindowFunc(rowDataType hybridqp.RowDataType) ([]func(prev Chunk, window *prevWindow, ordinal int),
	[]func(input Chunk, window *prevWindow, prevValues []interface{}, ordinal int))

func NewProcessors

func NewProcessors(inRowDataType, outRowDataType hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, opt *query.ProcessorOptions, isSubQuery bool) (*processorResults, error)

func NewPromFunc added in v1.3.0

func NewPromFunc(inRowDataType hybridqp.RowDataType, typ AggFuncType, op func() aggOperator, inOrdinal, outOrdinal int) (*aggFunc, error)

func NewRPCMessage

func NewRPCMessage(typ uint8) transport.Codec

func NewSortPartition added in v1.1.0

func NewSortPartition(i int, sortKeysIdxs []int, ascending []bool) *sortPartition

func NewSortRowMsg added in v1.1.0

func NewSortRowMsg(eles []sortEleMsg) *sortRowMsg

func NewSqlPlanTypePool added in v1.1.0

func NewSqlPlanTypePool(planType PlanType) ([]hybridqp.QueryNode, error)

func NewStdPromOperator added in v1.4.0

func NewStdPromOperator(flag bool) aggOperator

func NewStddevPromFunc added in v1.4.0

func NewStddevPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewStdvarPromFunc added in v1.4.0

func NewStdvarPromFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewStorePlanTypePool added in v1.1.0

func NewStorePlanTypePool(planType PlanType) []hybridqp.QueryNode

func NewStringSortEle added in v1.1.0

func NewStringSortEle() sortEleMsg

func NewSumFloatOperator added in v1.1.1

func NewSumFloatOperator() aggOperator

func NewSumFunc added in v1.1.0

func NewSumFunc(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (*aggFunc, error)

func NewSumIntegerOperator added in v1.1.1

func NewSumIntegerOperator() aggOperator

func NewfloatTuple added in v1.0.0

func NewfloatTuple(tuple []float64) *floatTuple

func OperateParam added in v1.5.0

func OperateParam(e influxql.Expr, param *util.Param) error

func PrintPlan added in v1.2.0

func PrintPlan(planName string, plan hybridqp.QueryNode)

func ProcessHashAgg added in v1.5.0

func ProcessHashAgg(node *LogicalHashAgg, isUnifyPlan bool)

ProcessHashAgg handles the specific logic for LogicalHashAgg nodes

func PutIncAggChunk added in v1.2.0

func PutIncAggChunk(queryID string, iterID int32, chunk Chunk)

func PutIncHashAggItem added in v1.2.0

func PutIncHashAggItem(queryID string, item *IncHashAggItem)

func RateFastReduce added in v1.3.0

func RateFastReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func RateFinalReduce added in v1.3.0

func RateFinalReduce[T util.NumberOnly](firstTime int64, lastTime int64, firstValue T, lastValue T,
	interval *hybridqp.Interval) (float64, bool)

func RateLowReduce added in v1.3.0

func RateLowReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func RateMerge added in v1.3.0

func RateMerge[T util.NumberOnly](prevPoints [2]*Point[T], interval *hybridqp.Interval) (float64, bool)

func RateMiddleReduce added in v1.3.0

func RateMiddleReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, int, T, T, bool)

func RateUpdate added in v1.3.0

func RateUpdate[T util.NumberOnly](prevPoints, currPoints [2]*Point[T])

func ReWriteArgs added in v1.1.0

func ReWriteArgs(best hybridqp.QueryNode, isUnifyPlan bool)

RewriteArgs rewrites the arguments for the query plan nodes recursively based on the specified plan type

func RebuildAggNodes added in v1.1.0

func RebuildAggNodes(plan hybridqp.QueryNode)

func RebuildColumnStorePlan added in v1.1.0

func RebuildColumnStorePlan(plan hybridqp.QueryNode) []hybridqp.QueryNode

func RegistryAggOp added in v1.3.0

func RegistryAggOp(name string, aggOp AggOperator)

func RegistryLLMFunc added in v1.5.0

func RegistryLLMFunc(name string, op LLMFunc)

func RegistryLabelFunction added in v1.3.0

func RegistryLabelFunction(name string, labelFunc LabelFunction)

func RegistryPromTimeFunction added in v1.3.0

func RegistryPromTimeFunction(name string, timeFunc PromTimeFunction)

func RegistryReaderCreator added in v1.1.0

func RegistryReaderCreator(plan LogicalPlan, creator ReaderCreator) bool

func RegistryTableFunctionOp added in v1.5.0

func RegistryTableFunctionOp(name string, tableFunctionOp TableFunctionOperator)

func RegistryTransformCreator

func RegistryTransformCreator(plan LogicalPlan, creator TransformCreator) bool

func RegrSlopeReduce added in v1.5.0

func RegrSlopeReduce[T util.NumberOnly](si *SliceItem[T]) (int, int64, float64, bool)

func RemoveCommonValues added in v1.3.0

func RemoveCommonValues(prev, curr [][]interface{}) [][]interface{}

func ReplaceSortAggMergeWithHashAgg added in v1.2.0

func ReplaceSortAggMergeWithHashAgg(plan hybridqp.QueryNode) []hybridqp.QueryNode

func ReplaceSortAggWithHashAgg added in v1.2.0

func ReplaceSortAggWithHashAgg(plan hybridqp.QueryNode) []hybridqp.QueryNode

func ReplaceSortMergeWithHashMerge added in v1.2.0

func ReplaceSortMergeWithHashMerge(plan hybridqp.QueryNode) []hybridqp.QueryNode

func ResetTime added in v1.3.0

func ResetTime(srcCk, dstCk Chunk, ts int64)

func ResetTimeForCompare added in v1.5.0

func ResetTimeForCompare(oChunk Chunk, offset time.Duration)

func ResetTimeForProm added in v1.3.0

func ResetTimeForProm(srcCk, dstCk Chunk, ts int64)

func RmseCol added in v1.5.0

func RmseCol[T util.NumberOnly](actual, predicted []T) float64

func Select

Select compiles, prepares, and then initiates execution of the query using the default compile options.

func SetLocalStorageForQuery added in v1.2.0

func SetLocalStorageForQuery(storage hybridqp.StoreEngine)

func SetMaxGroupsNums added in v1.2.0

func SetMaxGroupsNums(numsOfGroups int)

func SetResource added in v1.5.0

func SetResource(qc query.LogicalPlanCreator, schema hybridqp.Catalog) error

func SetTimeZero

func SetTimeZero(schema *QuerySchema) bool

func SlidingWindowMergeFunc added in v1.3.0

func SlidingWindowMergeFunc[T util.ExceptString](prevWindow, currWindow *SlidingWindow[T], fpm PointMerge[T])

func SortByTimeAsc added in v1.4.0

func SortByTimeAsc[T util.NumberOnly](f *HeapItem[T])

func SortByValueDsc added in v1.4.0

func SortByValueDsc[T util.NumberOnly](f *HeapItem[T])

func Str2bytes

func Str2bytes(s string) []byte

func StringAscendingAuxHelper

func StringAscendingAuxHelper(x, y Column, i, j int) (bool, bool)

func StringAuxHelpFunc

func StringAuxHelpFunc(input, output Column, rowIdx ...int)

func StringDescendingAuxHelper

func StringDescendingAuxHelper(x, y Column, i, j int) (bool, bool)

func StringFirstMerge

func StringFirstMerge(prevPoint, currPoint *StringPoint)

func StringLastMerge

func StringLastMerge(prevPoint, currPoint *StringPoint)

func SumMerge added in v1.3.0

func SumMerge[T util.NumberOnly](prevPoint, currPoint *Point[T])

func SumReduce added in v1.3.0

func SumReduce[T util.NumberOnly](c Chunk, values []T, ordinal, start, end int) (int, T, bool)

func TopCmpByTimeReduce added in v1.3.0

func TopCmpByTimeReduce[T util.NumberOnly](a, b *PointItem[T]) bool

func TopCmpByValueReduce added in v1.3.0

func TopCmpByValueReduce[T util.NumberOnly](a, b *PointItem[T]) bool

func TopCmpByValueReduceProm added in v1.4.0

func TopCmpByValueReduceProm[T util.NumberOnly](a, b *PointItem[T]) bool

func TransForwardBoolean added in v1.5.0

func TransForwardBoolean(dst Column, src Column) error

func TransForwardFloat added in v1.5.0

func TransForwardFloat(dst Column, src Column) error

func TransForwardInteger added in v1.5.0

func TransForwardInteger(dst Column, src Column) error

func TransForwardString added in v1.5.0

func TransForwardString(dst Column, src Column) error

func TransMath added in v1.0.0

func TransMath(c *influxql.Call) (func(dst Column, src Chunk, index []int, dstChunk Chunk), error)

func TransNodeTraits added in v1.5.0

func TransNodeTraits(nodeTraits []hybridqp.Trait) ([]hybridqp.Trait, error)

func TransparentForwardBoolean added in v1.0.0

func TransparentForwardBoolean(dst Column, src Chunk, index []int, dstChunk Chunk)

func TransparentForwardBooleanColumn

func TransparentForwardBooleanColumn(dst Column, src Column, srcChunk Chunk, dstChunk Chunk)

func TransparentForwardFloat added in v1.0.0

func TransparentForwardFloat(dst Column, src Chunk, index []int, dstChunk Chunk)

func TransparentForwardFloatColumn

func TransparentForwardFloatColumn(dst Column, src Column, srcChunk Chunk, dstChunk Chunk)

func TransparentForwardGraph added in v1.5.0

func TransparentForwardGraph(dst Column, src Chunk, index []int, dstChunk Chunk)

func TransparentForwardGraphColumn added in v1.5.0

func TransparentForwardGraphColumn(dst Column, src Column, srcChunk Chunk, dstChunk Chunk)

func TransparentForwardInteger added in v1.0.0

func TransparentForwardInteger(dst Column, src Chunk, index []int, dstChunk Chunk)

func TransparentForwardIntegerColumn

func TransparentForwardIntegerColumn(dst Column, src Column, srcChunk Chunk, dstChunk Chunk)

func TransparentForwardString added in v1.0.0

func TransparentForwardString(dst Column, src Chunk, index []int, dstChunk Chunk)

func TransparentForwardStringColumn

func TransparentForwardStringColumn(dst Column, src Column, srcChunk Chunk, dstChunk Chunk)

func TranverseBinTreeForTimestamp added in v1.3.0

func TranverseBinTreeForTimestamp(expr influxql.Expr)

func UnionBitMapArray

func UnionBitMapArray(b1, b2 []uint16) []uint16

func UnionColumns

func UnionColumns(cols ...Column) []uint16

func UnmarshalBinary added in v1.0.0

func UnmarshalBinary(buf []byte, schema hybridqp.Catalog) (hybridqp.QueryNode, error)

func UnmarshalBinaryNode added in v1.0.0

func UnmarshalBinaryNode(pb *internal.QueryNode, schema hybridqp.Catalog) (hybridqp.QueryNode, error)

func UnmarshalQueryNode

func UnmarshalQueryNode(buf []byte, shardNum int, opt hybridqp.Options) (hybridqp.QueryNode, error)

func UpdateHashBooleanMaxFast added in v1.3.0

func UpdateHashBooleanMaxFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashBooleanMaxSlow added in v1.3.0

func UpdateHashBooleanMaxSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashBooleanMinFast added in v1.3.0

func UpdateHashBooleanMinFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashBooleanMinSlow added in v1.3.0

func UpdateHashBooleanMinSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatMaxFast added in v1.3.0

func UpdateHashFloatMaxFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatMaxSlow added in v1.3.0

func UpdateHashFloatMaxSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatMinFast added in v1.3.0

func UpdateHashFloatMinFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatMinSlow added in v1.3.0

func UpdateHashFloatMinSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatSumFast added in v1.3.0

func UpdateHashFloatSumFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashFloatSumSlow added in v1.3.0

func UpdateHashFloatSumSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterMaxFast added in v1.3.0

func UpdateHashInterMaxFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterMaxSlow added in v1.3.0

func UpdateHashInterMaxSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterMinFast added in v1.3.0

func UpdateHashInterMinFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterMinSlow added in v1.3.0

func UpdateHashInterMinSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterSumFast added in v1.3.0

func UpdateHashInterSumFast(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateHashInterSumSlow added in v1.3.0

func UpdateHashInterSumSlow(dstChunk, srcChunk Chunk, dstCol, srcCol, dstRow, srcRow int, bitmap *Bitmap)

func UpdateIncAggFunc added in v1.2.0

func UpdateIncAggFunc(_, _ cache.Entry) bool

func UpdateIncHashAggFunc added in v1.2.0

func UpdateIncHashAggFunc(_, _ cache.Entry) bool

func ValidRpcMessageType added in v1.1.0

func ValidRpcMessageType(typ byte) bool

func ValidateFieldsFromPlans

func ValidateFieldsFromPlans(plans []hybridqp.QueryNode) bool

func Walk

Types

type ADDiffAbsOp added in v1.5.0

type ADDiffAbsOp struct{}

func (*ADDiffAbsOp) CreateRoutine added in v1.5.0

func (c *ADDiffAbsOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type ADDiffTimeOp

type ADDiffTimeOp struct{}

func (*ADDiffTimeOp) CreateRoutine

func (c *ADDiffTimeOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type ADRmseExtOp added in v1.5.0

type ADRmseExtOp struct{}

func (*ADRmseExtOp) CreateRoutine added in v1.5.0

func (c *ADRmseExtOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type ADSlopeScoreOp added in v1.5.0

type ADSlopeScoreOp struct{}

func (*ADSlopeScoreOp) CreateRoutine added in v1.5.0

func (c *ADSlopeScoreOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type Abort

type Abort struct {
	ClientID    uint64
	QueryID     uint64
	NoMarkCrash bool
}

func NewAbort

func NewAbort(queryID uint64, clientID uint64, noMarkCrash bool) *Abort

func (*Abort) Instance

func (e *Abort) Instance() transport.Codec

func (*Abort) Marshal

func (e *Abort) Marshal(buf []byte) ([]byte, error)

func (*Abort) Size

func (e *Abort) Size() int

func (*Abort) Unmarshal

func (e *Abort) Unmarshal(buf []byte) error

type AbortProcessor added in v1.3.0

type AbortProcessor interface {
	AbortSinkTransform()
}

type AbsentIterator added in v1.4.0

type AbsentIterator struct {
	// contains filtered or unexported fields
}

func NewAbsentIterator added in v1.4.0

func NewAbsentIterator(inOrdinal, outOrdinal int, opt *query.ProcessorOptions) *AbsentIterator

func (*AbsentIterator) Next added in v1.4.0

type AggCallFuncParams added in v1.3.0

type AggCallFuncParams struct {
	InRowDataType, OutRowDataType hybridqp.RowDataType    // dataType
	ExprOpt                       hybridqp.ExprOptions    // aggregate column information
	IsSingleCall                  bool                    // is a single call flag, used for optimize performance
	AuxProcessor                  []*AuxProcessor         // eg: select first(v1),v2 from xxx,then v2 is aux
	Opt                           *query.ProcessorOptions // ProcessorOptions is an object passed to CreateIterator to specify creation options.
	ProRes                        *processorResults       //processor response
	IsSubQuery                    bool
	Name                          string
}

type AggFactory added in v1.3.0

type AggFactory map[string]AggOperator

type AggFuncType added in v1.1.0

type AggFuncType uint32

type AggLevel added in v1.0.0

type AggLevel uint8
const (
	UnknownLevel AggLevel = iota
	SourceLevel
	MiddleLevel
	SinkLevel
)

type AggOperator added in v1.3.0

type AggOperator interface {
	CreateRoutine(params *AggCallFuncParams) (Routine, error)
}

func GetAggOperator added in v1.3.0

func GetAggOperator(name string) AggOperator

type AggOperatorMsgs added in v1.1.0

type AggOperatorMsgs struct {
	// contains filtered or unexported fields
}

func NewAggOperatorMsgs added in v1.1.0

func NewAggOperatorMsgs(size int) *AggOperatorMsgs

func (*AggOperatorMsgs) Alloc added in v1.1.0

func (a *AggOperatorMsgs) Alloc(size int) []*aggOperatorMsg

func (*AggOperatorMsgs) Free added in v1.1.0

func (a *AggOperatorMsgs) Free()

type AggPushDownToColumnStoreReaderRule added in v1.1.0

type AggPushDownToColumnStoreReaderRule struct {
	OptRuleBase
}

func NewAggPushDownToColumnStoreReaderRule added in v1.1.0

func NewAggPushDownToColumnStoreReaderRule(description string) *AggPushDownToColumnStoreReaderRule

func (*AggPushDownToColumnStoreReaderRule) Category added in v1.5.0

func (*AggPushDownToColumnStoreReaderRule) Equals added in v1.1.0

func (*AggPushDownToColumnStoreReaderRule) OnMatch added in v1.1.0

func (*AggPushDownToColumnStoreReaderRule) ToString added in v1.1.0

type AggPushDownToSubQueryRule

type AggPushDownToSubQueryRule struct {
	OptRuleBase
}

func NewAggPushDownToSubQueryRule

func NewAggPushDownToSubQueryRule(description string) *AggPushDownToSubQueryRule

func (*AggPushDownToSubQueryRule) Category added in v1.5.0

func (*AggPushDownToSubQueryRule) Equals

func (r *AggPushDownToSubQueryRule) Equals(rhs OptRule) bool

func (*AggPushDownToSubQueryRule) OnMatch

func (r *AggPushDownToSubQueryRule) OnMatch(call *OptRuleCall)

func (*AggPushDownToSubQueryRule) ToString

func (r *AggPushDownToSubQueryRule) ToString() string

type AggPushdownToExchangeRule

type AggPushdownToExchangeRule struct {
	OptRuleBase
}

func NewAggPushdownToExchangeRule

func NewAggPushdownToExchangeRule(description string) *AggPushdownToExchangeRule

func (*AggPushdownToExchangeRule) Category added in v1.5.0

func (*AggPushdownToExchangeRule) Equals

func (r *AggPushdownToExchangeRule) Equals(rhs OptRule) bool

func (*AggPushdownToExchangeRule) OnMatch

func (r *AggPushdownToExchangeRule) OnMatch(call *OptRuleCall)

func (*AggPushdownToExchangeRule) ToString

func (r *AggPushdownToExchangeRule) ToString() string

type AggPushdownToReaderRule

type AggPushdownToReaderRule struct {
	OptRuleBase
}

func NewAggPushdownToReaderRule

func NewAggPushdownToReaderRule(description string) *AggPushdownToReaderRule

func (*AggPushdownToReaderRule) Category added in v1.5.0

func (*AggPushdownToReaderRule) Equals

func (r *AggPushdownToReaderRule) Equals(rhs OptRule) bool

func (*AggPushdownToReaderRule) OnMatch

func (r *AggPushdownToReaderRule) OnMatch(call *OptRuleCall)

func (*AggPushdownToReaderRule) ToString

func (r *AggPushdownToReaderRule) ToString() string

type AggPushdownToSeriesRule

type AggPushdownToSeriesRule struct {
	OptRuleBase
}

func NewAggPushdownToSeriesRule

func NewAggPushdownToSeriesRule(description string) *AggPushdownToSeriesRule

func (*AggPushdownToSeriesRule) Category added in v1.5.0

func (*AggPushdownToSeriesRule) Equals

func (r *AggPushdownToSeriesRule) Equals(rhs OptRule) bool

func (*AggPushdownToSeriesRule) OnMatch

func (r *AggPushdownToSeriesRule) OnMatch(call *OptRuleCall)

func (*AggPushdownToSeriesRule) ToString

func (r *AggPushdownToSeriesRule) ToString() string

type AggSpreadToExchangeRule

type AggSpreadToExchangeRule struct {
	OptRuleBase
}

func NewAggSpreadToExchangeRule

func NewAggSpreadToExchangeRule(description string) *AggSpreadToExchangeRule

func (*AggSpreadToExchangeRule) Category added in v1.5.0

func (*AggSpreadToExchangeRule) Equals

func (r *AggSpreadToExchangeRule) Equals(rhs OptRule) bool

func (*AggSpreadToExchangeRule) OnMatch

func (r *AggSpreadToExchangeRule) OnMatch(call *OptRuleCall)

func (*AggSpreadToExchangeRule) ToString

func (r *AggSpreadToExchangeRule) ToString() string

type AggSpreadToReaderRule

type AggSpreadToReaderRule struct {
	OptRuleBase
}

func NewAggSpreadToReaderRule

func NewAggSpreadToReaderRule(description string) *AggSpreadToReaderRule

func (*AggSpreadToReaderRule) Category added in v1.5.0

func (r *AggSpreadToReaderRule) Category() OptRuleCategory

func (*AggSpreadToReaderRule) Equals

func (r *AggSpreadToReaderRule) Equals(rhs OptRule) bool

func (*AggSpreadToReaderRule) OnMatch

func (r *AggSpreadToReaderRule) OnMatch(call *OptRuleCall)

func (*AggSpreadToReaderRule) ToString

func (r *AggSpreadToReaderRule) ToString() string

type AggSpreadToSortAppendRule

type AggSpreadToSortAppendRule struct {
	OptRuleBase
}

func NewAggSpreadToSortAppendRule

func NewAggSpreadToSortAppendRule(description string) *AggSpreadToSortAppendRule

func (*AggSpreadToSortAppendRule) Category added in v1.5.0

func (*AggSpreadToSortAppendRule) Equals

func (r *AggSpreadToSortAppendRule) Equals(rhs OptRule) bool

func (*AggSpreadToSortAppendRule) OnMatch

func (r *AggSpreadToSortAppendRule) OnMatch(call *OptRuleCall)

func (*AggSpreadToSortAppendRule) ToString

func (r *AggSpreadToSortAppendRule) ToString() string

type AggToProjectInSubQueryRule

type AggToProjectInSubQueryRule struct {
	OptRuleBase
}

func NewAggToProjectInSubQueryRule

func NewAggToProjectInSubQueryRule(description string) *AggToProjectInSubQueryRule

func (*AggToProjectInSubQueryRule) Category added in v1.5.0

func (*AggToProjectInSubQueryRule) Equals

func (r *AggToProjectInSubQueryRule) Equals(rhs OptRule) bool

func (*AggToProjectInSubQueryRule) OnMatch

func (r *AggToProjectInSubQueryRule) OnMatch(call *OptRuleCall)

func (*AggToProjectInSubQueryRule) ToString

func (r *AggToProjectInSubQueryRule) ToString() string

type AlgoParam added in v1.5.0

type AlgoParam struct {
	HopCount  int                    `json:"hop_count"`
	BFSNarrow bool                   `json:"bfs_narrow"`
	Task      map[string]interface{} `json:"task"`
}

type AlignTransform

type AlignTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewAlignTransform

func NewAlignTransform(inRowDataType []hybridqp.RowDataType, outRowDataType []hybridqp.RowDataType, opt *query.ProcessorOptions) *AlignTransform

func (*AlignTransform) Close

func (trans *AlignTransform) Close()

func (*AlignTransform) Explain

func (trans *AlignTransform) Explain() []ValuePair

func (*AlignTransform) GetInputNumber

func (trans *AlignTransform) GetInputNumber(port Port) int

func (*AlignTransform) GetInputs

func (trans *AlignTransform) GetInputs() Ports

func (*AlignTransform) GetOutputNumber

func (trans *AlignTransform) GetOutputNumber(port Port) int

func (*AlignTransform) GetOutputs

func (trans *AlignTransform) GetOutputs() Ports

func (*AlignTransform) Name

func (trans *AlignTransform) Name() string

func (*AlignTransform) NextChunk added in v1.5.0

func (trans *AlignTransform) NextChunk()

func (*AlignTransform) SendChunk

func (trans *AlignTransform) SendChunk()

func (*AlignTransform) Work

func (trans *AlignTransform) Work(ctx context.Context) error

type AlignTransformCreator

type AlignTransformCreator struct{}

func (*AlignTransformCreator) Create

type AnalyzeResponse

type AnalyzeResponse struct {
	// contains filtered or unexported fields
}

func (AnalyzeResponse) Instance

func (a AnalyzeResponse) Instance() transport.Codec

func (*AnalyzeResponse) Marshal

func (a *AnalyzeResponse) Marshal(buf []byte) ([]byte, error)

func (*AnalyzeResponse) Size

func (a *AnalyzeResponse) Size() int

func (*AnalyzeResponse) Unmarshal

func (a *AnalyzeResponse) Unmarshal(buf []byte) error

type AppendColumnValueFunc added in v1.5.0

type AppendColumnValueFunc func(outColumn Column, value *ValueKeeper)

type AppendHeapItems

type AppendHeapItems struct {
	Items []*Item
	// contains filtered or unexported fields
}

func (*AppendHeapItems) GetBreakPoint added in v1.0.0

func (h *AppendHeapItems) GetBreakPoint() BaseBreakPoint

func (*AppendHeapItems) GetOption added in v1.0.0

func (h *AppendHeapItems) GetOption() *query.ProcessorOptions

func (*AppendHeapItems) Len

func (h *AppendHeapItems) Len() int

func (*AppendHeapItems) Less

func (h *AppendHeapItems) Less(i, j int) bool

func (*AppendHeapItems) Pop

func (h *AppendHeapItems) Pop() interface{}

func (*AppendHeapItems) Push

func (h *AppendHeapItems) Push(x interface{})

func (*AppendHeapItems) Swap

func (h *AppendHeapItems) Swap(i, j int)

type ArrowChunkSender added in v1.5.0

type ArrowChunkSender struct {
	Logger *logger.Logger
	// contains filtered or unexported fields
}

func NewArrowChunkSender added in v1.5.0

func NewArrowChunkSender(opt *query.ProcessorOptions) *ArrowChunkSender

func (*ArrowChunkSender) GenRecords added in v1.5.0

func (w *ArrowChunkSender) GenRecords(chunk Chunk, lastChunk bool)

func (*ArrowChunkSender) Release added in v1.5.0

func (w *ArrowChunkSender) Release()

func (*ArrowChunkSender) SetAbortProcessor added in v1.5.0

func (w *ArrowChunkSender) SetAbortProcessor(trans AbortProcessor)

func (*ArrowChunkSender) Write added in v1.5.0

func (w *ArrowChunkSender) Write(chunk Chunk, lastChunk bool) bool

type AttachedFrags added in v1.2.0

type AttachedFrags struct {
	BaseFrags
	// contains filtered or unexported fields
}

func NewAttachedFrags added in v1.2.0

func NewAttachedFrags(basePath string, cap int) *AttachedFrags

func (*AttachedFrags) AppendIndexes added in v1.2.0

func (s *AttachedFrags) AppendIndexes(tsspFiles ...interface{})

func (*AttachedFrags) IndexCount added in v1.2.0

func (s *AttachedFrags) IndexCount() int

func (*AttachedFrags) Indexes added in v1.2.0

func (s *AttachedFrags) Indexes() interface{}

type AttachedIndexInfo added in v1.2.0

type AttachedIndexInfo struct {
	// contains filtered or unexported fields
}

func NewAttachedIndexInfo added in v1.2.0

func NewAttachedIndexInfo(files []immutable.TSSPFile, infos []*colstore.PKInfo) *AttachedIndexInfo

func (*AttachedIndexInfo) Files added in v1.2.0

func (a *AttachedIndexInfo) Files() []immutable.TSSPFile

func (*AttachedIndexInfo) Infos added in v1.2.0

func (a *AttachedIndexInfo) Infos() []*colstore.PKInfo

type AuxProcessor

type AuxProcessor struct {
	// contains filtered or unexported fields
}

func NewAuxCoProcessor

func NewAuxCoProcessor(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) *AuxProcessor

type BaseBreakPoint added in v1.0.0

type BaseBreakPoint interface {
}

type BaseFrags added in v1.2.0

type BaseFrags struct {
	// contains filtered or unexported fields
}

func NewBaseFrags added in v1.2.0

func NewBaseFrags(basePath string, fileMode FileMode) *BaseFrags

func (*BaseFrags) AddFragCount added in v1.2.0

func (s *BaseFrags) AddFragCount(fragCount int64)

func (*BaseFrags) AppendFragRanges added in v1.2.0

func (s *BaseFrags) AppendFragRanges(frs ...fragment.FragmentRanges)

func (*BaseFrags) BasePath added in v1.2.0

func (s *BaseFrags) BasePath() string

func (*BaseFrags) FileMode added in v1.2.0

func (s *BaseFrags) FileMode() FileMode

func (*BaseFrags) FragCount added in v1.2.0

func (s *BaseFrags) FragCount() int64

func (*BaseFrags) FragRanges added in v1.2.0

func (s *BaseFrags) FragRanges() []fragment.FragmentRanges

func (*BaseFrags) GetErr added in v1.2.0

func (s *BaseFrags) GetErr() error

func (*BaseFrags) SetErr added in v1.2.0

func (s *BaseFrags) SetErr(err error)

func (*BaseFrags) Size added in v1.2.0

func (s *BaseFrags) Size() int

type BaseHeapItems added in v1.0.0

type BaseHeapItems interface {
	Len() int
	Less(i, j int) bool
	Swap(i, j int)
	Push(x interface{})
	Pop() interface{}
	GetBreakPoint() BaseBreakPoint
	GetOption() *query.ProcessorOptions
}

type BaseLLMFunc added in v1.5.0

type BaseLLMFunc struct {
	// contains filtered or unexported fields
}

func NewBaseLLMFunc added in v1.5.0

func NewBaseLLMFunc(
	name string,
	prompt string,
	callLLM func(prompt string, input string) ([]string, error)) *BaseLLMFunc

func (*BaseLLMFunc) Process added in v1.5.0

func (b *BaseLLMFunc) Process(dst, src Column) error

type BaseMergeIterator added in v1.1.0

type BaseMergeIterator struct {
	// contains filtered or unexported fields
}

type BaseProcessor

type BaseProcessor struct {
	// contains filtered or unexported fields
}

func (*BaseProcessor) Abort added in v1.0.1

func (bp *BaseProcessor) Abort()

func (*BaseProcessor) Analyze

func (bp *BaseProcessor) Analyze(span *tracing.Span)

func (*BaseProcessor) BaseSpan

func (bp *BaseProcessor) BaseSpan() *tracing.Span

func (*BaseProcessor) FinishSpan

func (bp *BaseProcessor) FinishSpan()

func (*BaseProcessor) Interrupt added in v1.3.0

func (bp *BaseProcessor) Interrupt()

func (*BaseProcessor) InterruptWithoutMark added in v1.4.0

func (bp *BaseProcessor) InterruptWithoutMark()

func (*BaseProcessor) IsSink

func (bp *BaseProcessor) IsSink() bool

func (*BaseProcessor) Once

func (bp *BaseProcessor) Once(fn func())

func (*BaseProcessor) Release

func (bp *BaseProcessor) Release() error

func (*BaseProcessor) StartSpan

func (bp *BaseProcessor) StartSpan(name string, withPP bool) *tracing.Span

type BasePromOp added in v1.3.0

type BasePromOp struct {
	// contains filtered or unexported fields
}

func NewBasePromOp added in v1.3.0

func NewBasePromOp(op string, fn ColReduceFunc[float64], fv ColMergeFunc[float64]) BasePromOp

func (*BasePromOp) CreateRoutine added in v1.3.0

func (c *BasePromOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type BaseTransData

type BaseTransData struct {
	// contains filtered or unexported fields
}

type BatchMPool added in v1.1.0

type BatchMPool struct {
	// contains filtered or unexported fields
}

func NewBatchMPool added in v1.1.0

func NewBatchMPool(size int) *BatchMPool

func (*BatchMPool) AllocBatchEndLocs added in v1.1.0

func (b *BatchMPool) AllocBatchEndLocs() []int

func (*BatchMPool) FreeBatchEndLocs added in v1.1.0

func (b *BatchMPool) FreeBatchEndLocs(batchEndLocs []int)

type BinOpTransform added in v1.3.0

type BinOpTransform struct {
	BaseProcessor

	OpType                   int
	On                       bool     // true: on; false: ignore
	MatchKeysForMatchCompute []string // on(MatchKeys)/ ignore(MatchKeys)
	MatchKeys                []string

	IncludeKeys []string // group_left/group_right(IncludeKeys)
	ReturnBool  bool
	NilMst      influxql.NilMstState

	BinOpHelper func(ctx context.Context, errs *errno.Errs)
	// contains filtered or unexported fields
}

func NewBinOpTransform added in v1.3.0

func NewBinOpTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, schema *QuerySchema, para *influxql.BinOp, lExpr, rExpr influxql.Expr) (*BinOpTransform, error)

func (*BinOpTransform) AddPrePrimaryGroup added in v1.5.0

func (trans *BinOpTransform) AddPrePrimaryGroup(prePrimaryGroups *GroupLocs)

func (*BinOpTransform) AddResult added in v1.3.0

func (trans *BinOpTransform) AddResult(secondaryChunk Chunk) error

func (*BinOpTransform) AddResultBase added in v1.5.0

func (trans *BinOpTransform) AddResultBase(secondaryChunk Chunk, noMatchFn func(Chunk, int, *ChunkTags), addPrePrimaryGroupFn func(*GroupLocs)) error

func (*BinOpTransform) AddResultLand added in v1.5.0

func (trans *BinOpTransform) AddResultLand(secondaryChunk Chunk) error

func (*BinOpTransform) AddResultLor added in v1.5.0

func (trans *BinOpTransform) AddResultLor(secondaryChunk Chunk) error

func (*BinOpTransform) AddResultLunless added in v1.5.0

func (trans *BinOpTransform) AddResultLunless(secondaryChunk Chunk) error

func (*BinOpTransform) AddResultSimple added in v1.3.0

func (trans *BinOpTransform) AddResultSimple(secondaryChunk Chunk)

1.no compute resultTags; 2.no computeMatchResult by both side input; 3.no resultMap dup err check

func (*BinOpTransform) BinOpHelperConditionBoth added in v1.3.0

func (trans *BinOpTransform) BinOpHelperConditionBoth(ctx context.Context, errs *errno.Errs)

func (*BinOpTransform) BinOpHelperConditionSingle added in v1.3.0

func (trans *BinOpTransform) BinOpHelperConditionSingle(ctx context.Context, errs *errno.Errs)

func (*BinOpTransform) BinOpHelperConditionSingleExpr added in v1.4.0

func (trans *BinOpTransform) BinOpHelperConditionSingleExpr(ctx context.Context, errs *errno.Errs)

func (*BinOpTransform) BinOpHelperOperator added in v1.3.0

func (trans *BinOpTransform) BinOpHelperOperator(ctx context.Context, errs *errno.Errs)

func (*BinOpTransform) Close added in v1.3.0

func (trans *BinOpTransform) Close()

func (*BinOpTransform) ComputeResultTags added in v1.3.0

func (trans *BinOpTransform) ComputeResultTags(pTagKeys, pTagValues, sTagKeys, sTagValues []string) ([]string, []string, string)

func (*BinOpTransform) Explain added in v1.3.0

func (trans *BinOpTransform) Explain() []ValuePair

func (*BinOpTransform) GetInputNumber added in v1.3.0

func (trans *BinOpTransform) GetInputNumber(_ Port) int

func (*BinOpTransform) GetInputs added in v1.3.0

func (trans *BinOpTransform) GetInputs() Ports

func (*BinOpTransform) GetOutputNumber added in v1.3.0

func (trans *BinOpTransform) GetOutputNumber(_ Port) int

func (*BinOpTransform) GetOutputs added in v1.3.0

func (trans *BinOpTransform) GetOutputs() Ports

func (*BinOpTransform) Name added in v1.3.0

func (trans *BinOpTransform) Name() string

func (*BinOpTransform) SendChunk added in v1.3.0

func (trans *BinOpTransform) SendChunk()

func (*BinOpTransform) Work added in v1.3.0

func (trans *BinOpTransform) Work(ctx context.Context) error

type BinOpTransformCreator added in v1.3.0

type BinOpTransformCreator struct {
}

func (*BinOpTransformCreator) Create added in v1.3.0

type Bitmap

type Bitmap struct {
	// contains filtered or unexported fields
}

Bitmap for column

func NewBitmap

func NewBitmap() *Bitmap

func (*Bitmap) Clear

func (b *Bitmap) Clear()

func (*Bitmap) CopyTo

func (b *Bitmap) CopyTo(dst *Bitmap)

func (*Bitmap) GetArray added in v1.1.1

func (b *Bitmap) GetArray() []uint16

func (*Bitmap) GetBit added in v1.1.1

func (b *Bitmap) GetBit() []byte

func (*Bitmap) GetLength added in v1.1.1

func (b *Bitmap) GetLength() int

func (*Bitmap) Instance

func (b *Bitmap) Instance() transport.Codec

func (*Bitmap) Marshal

func (b *Bitmap) Marshal(buf []byte) ([]byte, error)

func (*Bitmap) Reverse

func (b *Bitmap) Reverse()

func (*Bitmap) SetArray

func (b *Bitmap) SetArray(arr []uint16)

func (*Bitmap) SetLen

func (b *Bitmap) SetLen(len int)

func (*Bitmap) SetNilCount

func (b *Bitmap) SetNilCount(nilCount int)

func (*Bitmap) Size

func (b *Bitmap) Size() int

func (*Bitmap) String

func (b *Bitmap) String() string

func (*Bitmap) Unmarshal

func (b *Bitmap) Unmarshal(buf []byte) error

func (*Bitmap) UpdateBitWithArray

func (b *Bitmap) UpdateBitWithArray(dat []bool)

type BlockChunkPool added in v1.0.0

type BlockChunkPool struct {
	// contains filtered or unexported fields
}

BlockChunkPool fixed-capacity memory pool that blocks when the pool is empty or full.

func NewBlockChunkPool added in v1.0.0

func NewBlockChunkPool(chunkNum int, chunkBuilder *ChunkBuilder) *BlockChunkPool

func (*BlockChunkPool) Get added in v1.0.0

func (cp *BlockChunkPool) Get() Chunk

func (*BlockChunkPool) Put added in v1.0.0

func (cp *BlockChunkPool) Put(c Chunk)

func (*BlockChunkPool) Release added in v1.0.0

func (cp *BlockChunkPool) Release()

type BooleanAppendIterator

type BooleanAppendIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanAppendIterator

func NewBooleanAppendIterator() *BooleanAppendIterator

func (*BooleanAppendIterator) Next

func (f *BooleanAppendIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type BooleanColBooleanDistinctIterator

type BooleanColBooleanDistinctIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanColBooleanDistinctIterator

func NewBooleanColBooleanDistinctIterator(
	inOrdinal, outOrdinal int,
) *BooleanColBooleanDistinctIterator

func (*BooleanColBooleanDistinctIterator) Next

type BooleanColBooleanSampleIterator

type BooleanColBooleanSampleIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanColBooleanSampleIterator

func NewBooleanColBooleanSampleIterator(sampleNum int,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *BooleanColBooleanSampleIterator

func (*BooleanColBooleanSampleIterator) Next

type BooleanColBooleanSliceIterator

type BooleanColBooleanSliceIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanColBooleanSliceIterator

func NewBooleanColBooleanSliceIterator(fn BooleanColReduceSliceReduce,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *BooleanColBooleanSliceIterator

func (*BooleanColBooleanSliceIterator) Next

type BooleanColBooleanWindowReduce

type BooleanColBooleanWindowReduce func(c Chunk, values []bool, ordinal, start, end int) (index int, value bool, isNil bool)

type BooleanColReduceSliceReduce

type BooleanColReduceSliceReduce func(booleanItem *BooleanSliceItem) (index int, time int64, value float64, isNil bool)

type BooleanDistinctItem

type BooleanDistinctItem struct {
	// contains filtered or unexported fields
}

func NewBooleanDistinctItem

func NewBooleanDistinctItem() *BooleanDistinctItem

func (*BooleanDistinctItem) Len

func (f *BooleanDistinctItem) Len() int

func (*BooleanDistinctItem) Less

func (f *BooleanDistinctItem) Less(i, j int) bool

func (*BooleanDistinctItem) Nil

func (f *BooleanDistinctItem) Nil() bool

func (*BooleanDistinctItem) Reset

func (f *BooleanDistinctItem) Reset()

func (*BooleanDistinctItem) Swap

func (f *BooleanDistinctItem) Swap(i, j int)

type BooleanFieldValuer added in v1.0.0

type BooleanFieldValuer struct {
	// contains filtered or unexported fields
}

func (*BooleanFieldValuer) At added in v1.0.0

func (valuer *BooleanFieldValuer) At(col Column, pos int, field *influx.Field) bool

type BooleanIntervalIterator

type BooleanIntervalIterator struct{}

func NewBooleanIntervalIterator

func NewBooleanIntervalIterator() *BooleanIntervalIterator

func (*BooleanIntervalIterator) Next

type BooleanIterator added in v1.3.0

type BooleanIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanIterator added in v1.3.0

func NewBooleanIterator(fn BooleanReduce, fv PointMerge[bool],
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *BooleanIterator

func (*BooleanIterator) Next added in v1.3.0

type BooleanLimitIterator

type BooleanLimitIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanLimitIterator

func NewBooleanLimitIterator() *BooleanLimitIterator

func (*BooleanLimitIterator) Next

func (f *BooleanLimitIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type BooleanLinearFillProcessor

type BooleanLinearFillProcessor struct {
	// contains filtered or unexported fields
}

func NewBooleanLinearFillProcessor

func NewBooleanLinearFillProcessor(inOrdinal, outOrdinal int) *BooleanLinearFillProcessor

type BooleanMergeIterator

type BooleanMergeIterator struct {
	BaseMergeIterator
	// contains filtered or unexported fields
}

func NewBooleanMergeIterator

func NewBooleanMergeIterator() *BooleanMergeIterator

func (*BooleanMergeIterator) Next

func (f *BooleanMergeIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type BooleanNullFillProcessor

type BooleanNullFillProcessor struct {
	// contains filtered or unexported fields
}

func NewBooleanNullFillProcessor

func NewBooleanNullFillProcessor(inOrdinal, outOrdinal int) *BooleanNullFillProcessor

type BooleanNumberFillProcessor

type BooleanNumberFillProcessor struct {
	// contains filtered or unexported fields
}

func NewBooleanNumberFillProcessor

func NewBooleanNumberFillProcessor(inOrdinal, outOrdinal int) *BooleanNumberFillProcessor

type BooleanPreviousFillProcessor

type BooleanPreviousFillProcessor struct {
	// contains filtered or unexported fields
}

func NewBooleanPreviousFillProcessor

func NewBooleanPreviousFillProcessor(inOrdinal, outOrdinal int) *BooleanPreviousFillProcessor

type BooleanReduce added in v1.3.0

type BooleanReduce func(c Chunk, values []bool, ordinal, start, end int) (index int, value bool, isNil bool)

type BooleanSliceItem

type BooleanSliceItem struct {
	// contains filtered or unexported fields
}

func NewBooleanSliceItem

func NewBooleanSliceItem() *BooleanSliceItem

func (*BooleanSliceItem) AppendItem

func (f *BooleanSliceItem) AppendItem(c Chunk, ordinal, start, end int)

func (*BooleanSliceItem) Len

func (f *BooleanSliceItem) Len() int

func (*BooleanSliceItem) Reset

func (f *BooleanSliceItem) Reset()

type BooleanTimeColBooleanIterator

type BooleanTimeColBooleanIterator struct {
	// contains filtered or unexported fields
}

func NewBooleanTimeColBooleanIterator

func NewBooleanTimeColBooleanIterator(
	fn BooleanTimeColBooleanReduce, fv BooleanTimeColBooleanMerge, inOrdinal, outOrdinal int,
) *BooleanTimeColBooleanIterator

func (*BooleanTimeColBooleanIterator) Next

type BooleanTimeColBooleanMerge

type BooleanTimeColBooleanMerge func(prevPoint, currPoint *Point[bool])

type BooleanTimeColBooleanReduce

type BooleanTimeColBooleanReduce func(c Chunk, values []bool, ordinal, start, end int) (index int, value bool, isNil bool)

type BreakPoint

type BreakPoint struct {
	Name      string
	Tag       ChunkTags
	TimeEnd   int64
	TimeStart int64
}

BreakPoint is the point we peek from the 2nd. less chunk, if the value is bigger than the BreakPoint, which means we Need to change the chunk.

type BufEle added in v1.3.0

type BufEle struct {
	// contains filtered or unexported fields
}

func NewBufEle added in v1.3.0

func NewBufEle() *BufEle

func (*BufEle) Append added in v1.3.0

func (be *BufEle) Append(times []int64, values []float64)

func (*BufEle) AppendBuf added in v1.3.0

func (be *BufEle) AppendBuf(buf *BufEle)

func (*BufEle) Clear added in v1.3.0

func (be *BufEle) Clear()

func (*BufEle) Set added in v1.3.0

func (be *BufEle) Set(times []int64, values []float64, loc int, endLoc int)

type Byte

type Byte byte

type CONTEXT_IDENTIFIER added in v1.0.0

type CONTEXT_IDENTIFIER int
const (
	WRITER_CONTEXT CONTEXT_IDENTIFIER = 0x01
)

type CSIndexInfo added in v1.2.0

type CSIndexInfo struct {
	AttachedIndexInfo
	// contains filtered or unexported fields
}

func NewCSIndexInfo added in v1.2.0

func NewCSIndexInfo(shardPath string, info *AttachedIndexInfo, version uint32) *CSIndexInfo

func (*CSIndexInfo) ShardPath added in v1.2.0

func (cs *CSIndexInfo) ShardPath() string

func (*CSIndexInfo) Version added in v1.2.0

func (cs *CSIndexInfo) Version() uint32

type CTEExecutorBuilder added in v1.5.0

type CTEExecutorBuilder interface {
	Analyze(span *tracing.Span)
	Build(node hybridqp.QueryNode) (hybridqp.Executor, error)
}

type CTETransform added in v1.5.0

type CTETransform struct {
	BaseProcessor

	ExecutorBuilder CTEExecutorBuilder
	// contains filtered or unexported fields
}

func NewCTETransform added in v1.5.0

func NewCTETransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType,
	schema hybridqp.Catalog, ctePlan hybridqp.QueryNode, cte *influxql.CTE, ops []hybridqp.ExprOptions) (*CTETransform, error)

func (*CTETransform) Close added in v1.5.0

func (trans *CTETransform) Close()

func (*CTETransform) Explain added in v1.5.0

func (trans *CTETransform) Explain() []ValuePair

func (*CTETransform) GetInputNumber added in v1.5.0

func (trans *CTETransform) GetInputNumber(port Port) int

func (*CTETransform) GetInputs added in v1.5.0

func (trans *CTETransform) GetInputs() Ports

func (*CTETransform) GetOutputNumber added in v1.5.0

func (trans *CTETransform) GetOutputNumber(port Port) int

func (*CTETransform) GetOutputs added in v1.5.0

func (trans *CTETransform) GetOutputs() Ports

func (*CTETransform) Name added in v1.5.0

func (trans *CTETransform) Name() string

func (*CTETransform) RunPlan added in v1.5.0

func (trans *CTETransform) RunPlan(ctx context.Context, span *tracing.Span) error

func (*CTETransform) SendChunk added in v1.5.0

func (trans *CTETransform) SendChunk(c Chunk)

func (*CTETransform) TagValueFromChunk added in v1.5.0

func (trans *CTETransform) TagValueFromChunk(chunk Chunk, name string) Column

func (*CTETransform) Work added in v1.5.0

func (trans *CTETransform) Work(ctx context.Context) error

type CTETransformCreator added in v1.5.0

type CTETransformCreator struct {
}

func (*CTETransformCreator) Create added in v1.5.0

type CallFn added in v1.3.0

type CallFn func([]int64, []int64, []float64, []float64, int64, *influxql.PromSubCall) (float64, bool)

type CancelOnlySource

type CancelOnlySource struct {
	BaseProcessor

	Output *ChunkPort
}

func NewCancelOnlySource

func NewCancelOnlySource(rowDataType hybridqp.RowDataType) *CancelOnlySource

func (*CancelOnlySource) Close

func (source *CancelOnlySource) Close()

func (*CancelOnlySource) Explain

func (source *CancelOnlySource) Explain() []ValuePair

func (*CancelOnlySource) GetInputNumber

func (source *CancelOnlySource) GetInputNumber(_ Port) int

func (*CancelOnlySource) GetInputs

func (source *CancelOnlySource) GetInputs() Ports

func (*CancelOnlySource) GetOutputNumber

func (source *CancelOnlySource) GetOutputNumber(_ Port) int

func (*CancelOnlySource) GetOutputs

func (source *CancelOnlySource) GetOutputs() Ports

func (*CancelOnlySource) Name

func (source *CancelOnlySource) Name() string

func (*CancelOnlySource) Work

func (source *CancelOnlySource) Work(ctx context.Context) error

type CastorAggCutRule added in v0.2.0

type CastorAggCutRule struct {
	OptRuleBase
}

func NewCastorAggCutRule added in v0.2.0

func NewCastorAggCutRule(description string) *CastorAggCutRule

func (*CastorAggCutRule) Category added in v1.5.0

func (r *CastorAggCutRule) Category() OptRuleCategory

func (*CastorAggCutRule) Equals added in v0.2.0

func (r *CastorAggCutRule) Equals(rhs OptRule) bool

func (*CastorAggCutRule) OnMatch added in v0.2.0

func (r *CastorAggCutRule) OnMatch(call *OptRuleCall)

func (*CastorAggCutRule) ToString added in v0.2.0

func (r *CastorAggCutRule) ToString() string

type CastorParams added in v1.5.0

type CastorParams struct {
	// contains filtered or unexported fields
}

type Chunk

Chunk consists of seven functionally distinct components, each responsible for managing metadata, tags, time, basic operations and serialization and Graph management.

func FilterEventChunks added in v1.5.0

func FilterEventChunks(chunks []Chunk, nodes map[string]GraphNode, colMap map[string]int) (filterChunks []Chunk, err error)

func GetIncAggChunk added in v1.2.0

func GetIncAggChunk(queryID string, iterID int32) (Chunk, bool)

func PromStepInvariant added in v1.5.0

func PromStepInvariant(src, dst Chunk, typ PromStepInvariantType, preGroup []byte, nextGroup []byte, step, startT, endT int64) Chunk

func PromStepInvariantOnlyLastGroup added in v1.5.0

func PromStepInvariantOnlyLastGroup(src Chunk, preGroup []byte, nextGroup []byte, step, startT, endT int64) Chunk

func PromStepInvariantSkipLastGroup added in v1.5.0

func PromStepInvariantSkipLastGroup(src, dst Chunk, preGroup []byte, step, startT, endT int64) Chunk

type ChunkBuilder

type ChunkBuilder struct {
	// contains filtered or unexported fields
}

func NewChunkBuilder

func NewChunkBuilder(rowDataType hybridqp.RowDataType) *ChunkBuilder

func (*ChunkBuilder) NewChunk

func (b *ChunkBuilder) NewChunk(name string) Chunk

func (*ChunkBuilder) SetDim added in v1.1.0

func (b *ChunkBuilder) SetDim(dimDataType hybridqp.RowDataType)

type ChunkColumn added in v1.5.0

type ChunkColumn interface {
	Columns() []Column
	Column(int) Column
	Dims() []Column
	AddDims([]string)
	NewDims(size int)
	SetColumn(Column, int)
	AddColumn(...Column)
	Dim(int) Column
	AddDim(...Column)
}

ChunkColumn is an interface related to columns and dimensions.

type ChunkGraph added in v1.5.0

type ChunkGraph interface {
	GetGraph() IGraph
	SetGraph(g IGraph)
}

ChunkGraph is a graph-related interface.

type ChunkImpl

type ChunkImpl struct {
	*record.Record
	// contains filtered or unexported fields
}

ChunkImpl DO NOT ADD ADDITIONAL FIELDS TO THIS STRUCT. Memory layout of ChunkImpl:

+------------------------------------------------+ | rowDataType: Data type of the row | | - Description: Stores the data type of the row | | - Usage: Determines the type and structure of the data | +------------------------------------------------+ | name: Measurement name | | - Description: Stores the name of the measurement | | - Usage: Identifies the measurement to which the data belongs | +------------------------------------------------+ | tags: List of tag data | | - Description: Stores a list of tag data | | - Usage: Manages tag information of the data | +------------------------------------------------+ | tagIndex: List of tag indices | | - Description: Stores index information for tags | | - Usage:Quick access and locate tag data | +------------------------------------------------+ | time: List of timestamps | | - Description: Stores a list of timestamp data | | - Usage: Records and manages the time information of the data | +------------------------------------------------+ | intervalIndex: List of interval indices | | - Description: Stores a list of interval index data | | - Usage: Manages the time interval indices of the data | +------------------------------------------------+ | columns: List of column data | | - Description: Stores a list of column data | | - Usage: Manages the column information of the data | +------------------------------------------------+ | dims: List of dimension columns | | - Description: Stores a list of dimension column data | | - Usage: Manages the dimension information of the data | +------------------------------------------------+ | Record: Record data | | - Description: Stores record data | | - Usage: Manages the record information of the data | +------------------------------------------------+ | graph: Graph data | | - Description: Stores graph data | | - Usage: Manages the graph structure information of the data | +------------------------------------------------+

func NewChunkImpl

func NewChunkImpl(rowDataType hybridqp.RowDataType, name string) *ChunkImpl

NewChunkImpl FIXME: memory pool

func (*ChunkImpl) AddColumn

func (c *ChunkImpl) AddColumn(cols ...Column)

func (*ChunkImpl) AddDim added in v1.1.0

func (c *ChunkImpl) AddDim(cols ...Column)

func (*ChunkImpl) AddDims added in v1.1.0

func (c *ChunkImpl) AddDims(dimsVals []string)

func (*ChunkImpl) Append added in v1.5.0

func (c *ChunkImpl) Append(ck Chunk, start, end int)

Append used to append one chunk with the same number of columns and the same data type to other one based on the specified row interval.

func (*ChunkImpl) AppendIntervalIndex

func (c *ChunkImpl) AppendIntervalIndex(intervalIndex int)

func (*ChunkImpl) AppendIntervalIndexes added in v1.1.0

func (c *ChunkImpl) AppendIntervalIndexes(intervalIndex []int)

func (*ChunkImpl) AppendTagsAndIndex

func (c *ChunkImpl) AppendTagsAndIndex(tag ChunkTags, tagIndex int)

func (*ChunkImpl) AppendTagsAndIndexes

func (c *ChunkImpl) AppendTagsAndIndexes(tags []ChunkTags, tagIndex []int)

func (*ChunkImpl) AppendTime

func (c *ChunkImpl) AppendTime(t ...int64)

func (*ChunkImpl) AppendTimes added in v1.1.0

func (c *ChunkImpl) AppendTimes(ts []int64)

Note: This method will be deprecated; it is recommended to use AppendTime instead.

func (*ChunkImpl) CheckChunk

func (c *ChunkImpl) CheckChunk()

func (*ChunkImpl) Clone

func (c *ChunkImpl) Clone() Chunk

func (*ChunkImpl) Column

func (c *ChunkImpl) Column(i int) Column

func (*ChunkImpl) Columns

func (c *ChunkImpl) Columns() []Column

func (*ChunkImpl) CopyByRowDataType added in v1.1.0

func (c *ChunkImpl) CopyByRowDataType(dst Chunk, fromRt hybridqp.RowDataType, dstRt hybridqp.RowDataType) error

func (*ChunkImpl) CopyTo added in v1.0.0

func (c *ChunkImpl) CopyTo(dstChunk Chunk)

func (*ChunkImpl) Dim added in v1.1.0

func (c *ChunkImpl) Dim(i int) Column

func (*ChunkImpl) Dims added in v1.1.0

func (c *ChunkImpl) Dims() []Column

func (*ChunkImpl) FilterRowsByIndexes added in v1.5.0

func (c *ChunkImpl) FilterRowsByIndexes(indexes ...int) Chunk

Convention: The indexes parameter is a list of valid row IDs in ascending order

func (*ChunkImpl) GetGraph added in v1.5.0

func (c *ChunkImpl) GetGraph() IGraph

func (*ChunkImpl) GetRecord added in v1.0.0

func (c *ChunkImpl) GetRecord() *record.Record

func (*ChunkImpl) InitTimeWindow added in v1.2.0

func (c *ChunkImpl) InitTimeWindow(minTime, maxTime, intervalTime int64, hasInterval, ascending bool, tag ChunkTags)

func (*ChunkImpl) Instance

func (c *ChunkImpl) Instance() transport.Codec

func (*ChunkImpl) IntervalIndex

func (c *ChunkImpl) IntervalIndex() []int

func (*ChunkImpl) IntervalLen

func (c *ChunkImpl) IntervalLen() int

func (*ChunkImpl) IsNil deprecated

func (c *ChunkImpl) IsNil() bool

Deprecated: Do not use

func (*ChunkImpl) Len

func (c *ChunkImpl) Len() int

func (*ChunkImpl) Marshal

func (c *ChunkImpl) Marshal(buf []byte) ([]byte, error)

func (*ChunkImpl) Name

func (c *ChunkImpl) Name() string

func (*ChunkImpl) NewDims added in v1.1.0

func (c *ChunkImpl) NewDims(size int)

func (*ChunkImpl) NumberOfCols

func (c *ChunkImpl) NumberOfCols() int

func (*ChunkImpl) NumberOfRows

func (c *ChunkImpl) NumberOfRows() int

func (*ChunkImpl) Reset

func (c *ChunkImpl) Reset()

func (*ChunkImpl) ResetIntervalIndex

func (c *ChunkImpl) ResetIntervalIndex(intervalIndex ...int)

func (*ChunkImpl) ResetTagsAndIndexes

func (c *ChunkImpl) ResetTagsAndIndexes(tags []ChunkTags, tagIndex []int)

func (*ChunkImpl) ResetTime

func (c *ChunkImpl) ResetTime(idx int, time int64)

func (*ChunkImpl) RowDataType

func (c *ChunkImpl) RowDataType() hybridqp.RowDataType

func (*ChunkImpl) SetColumn

func (c *ChunkImpl) SetColumn(col Column, i int)

func (*ChunkImpl) SetGraph added in v1.5.0

func (c *ChunkImpl) SetGraph(g IGraph)

func (*ChunkImpl) SetName

func (c *ChunkImpl) SetName(name string)

func (*ChunkImpl) SetRowDataType

func (c *ChunkImpl) SetRowDataType(r hybridqp.RowDataType)

func (*ChunkImpl) SetTime

func (c *ChunkImpl) SetTime(time []int64)

func (*ChunkImpl) Size

func (c *ChunkImpl) Size() int

func (*ChunkImpl) SlimChunk

func (c *ChunkImpl) SlimChunk(ridIdx []int) Chunk

SlimChunk filter the ridIdx columns to slim chunk

func (*ChunkImpl) String

func (c *ChunkImpl) String() string

func (*ChunkImpl) TagIndex

func (c *ChunkImpl) TagIndex() []int

func (*ChunkImpl) TagLen

func (c *ChunkImpl) TagLen() int

func (*ChunkImpl) Tags

func (c *ChunkImpl) Tags() []ChunkTags

func (*ChunkImpl) Time

func (c *ChunkImpl) Time() []int64

func (*ChunkImpl) TimeByIndex

func (c *ChunkImpl) TimeByIndex(i int) int64

func (*ChunkImpl) TruncateTime

func (c *ChunkImpl) TruncateTime(idx int)

func (*ChunkImpl) Unmarshal

func (c *ChunkImpl) Unmarshal(buf []byte) error

type ChunkIterator added in v1.0.0

type ChunkIterator struct {
	// contains filtered or unexported fields
}

func NewChunkIteratorFromValuer added in v1.0.0

func NewChunkIteratorFromValuer(chunk Chunk, name string, valuer *FieldsValuer) *ChunkIterator

func (*ChunkIterator) GetNext added in v1.0.0

func (iter *ChunkIterator) GetNext(row *influx.Row, tuple *TargetTuple)

func (*ChunkIterator) HasMore added in v1.0.0

func (iter *ChunkIterator) HasMore() bool

type ChunkMeta added in v1.5.0

type ChunkMeta interface {
	RowDataType() hybridqp.RowDataType
	SetRowDataType(hybridqp.RowDataType)
	CopyByRowDataType(c Chunk, fromRt hybridqp.RowDataType, dstRt hybridqp.RowDataType) error
	Name() string
	SetName(string)
}

ChunkMeta is data types and name interface.

type ChunkOperator added in v1.5.0

type ChunkOperator interface {
	IsNil() bool
	NumberOfRows() int
	NumberOfCols() int
	Len() int
	Reset()
	SlimChunk(ridIdx []int) Chunk
	Clone() Chunk
	CopyTo(Chunk)
	FilterRowsByIndexes(indexes ...int) Chunk
	CheckChunk()
	GetRecord() *record.Record
	Append(ck Chunk, start, end int)
	String() string
}

ChunkOperator is an interface related to data operations.

type ChunkPort

type ChunkPort struct {
	RowDataType hybridqp.RowDataType
	State       chan Chunk
	OrigiState  chan Chunk
	Redirected  bool
	// contains filtered or unexported fields
}

func NewChunkPort

func NewChunkPort(rowDataType hybridqp.RowDataType) *ChunkPort

func (*ChunkPort) Close

func (p *ChunkPort) Close()

func (*ChunkPort) Connect

func (p *ChunkPort) Connect(to Port)

func (*ChunkPort) ConnectNoneCache added in v1.0.0

func (p *ChunkPort) ConnectNoneCache(to Port)

func (*ChunkPort) ConnectionId

func (p *ChunkPort) ConnectionId() uintptr

func (*ChunkPort) Equal

func (p *ChunkPort) Equal(to Port) bool

func (*ChunkPort) Redirect

func (p *ChunkPort) Redirect(to Port)

func (*ChunkPort) Release

func (p *ChunkPort) Release()

type ChunkPorts

type ChunkPorts []*ChunkPort

func (ChunkPorts) Close

func (ps ChunkPorts) Close()

type ChunkPortsWithMeta added in v1.5.0

type ChunkPortsWithMeta struct {
	ChunkPort *ChunkPort
	Chunks    []Chunk
	IGraph    IGraph
	Meta      map[string]int
}

type ChunkSender added in v1.5.0

type ChunkSender interface {
	Write(Chunk, bool) bool
	SetAbortProcessor(AbortProcessor)
	Release()
}

func NewChunkSender added in v1.5.0

func NewChunkSender(opt *query.ProcessorOptions) ChunkSender

type ChunkSerialization added in v1.5.0

type ChunkSerialization interface {
	Marshal([]byte) ([]byte, error)
	Unmarshal([]byte) error
	Instance() transport.Codec
	Size() int
}

ChunkSerialization is an interface related to serialization and deserialization.

type ChunkTag added in v1.5.0

type ChunkTag interface {
	Tags() []ChunkTags
	TagIndex() []int
	TagLen() int
	AppendTagsAndIndex(ChunkTags, int)
	AppendTagsAndIndexes([]ChunkTags, []int)
	ResetTagsAndIndexes(tags []ChunkTags, tagIndex []int)
}

TagsManager is label-related interface.

type ChunkTags

type ChunkTags struct {
	// contains filtered or unexported fields
}

func NewChunkTags

func NewChunkTags(pts influx.PointTags, dimensions []string) *ChunkTags

func NewChunkTagsByBytes added in v1.3.0

func NewChunkTagsByBytes(bytes []byte) *ChunkTags

func NewChunkTagsByTagKVs added in v1.0.0

func NewChunkTagsByTagKVs(k []string, v []string) *ChunkTags

func NewChunkTagsDeepCopy added in v1.3.0

func NewChunkTagsDeepCopy(subset []byte, offsets []uint16) *ChunkTags

func NewChunkTagsV2

func NewChunkTagsV2(subset []byte) *ChunkTags

func NewChunkTagsWithoutDims added in v1.3.0

func NewChunkTagsWithoutDims(pts influx.PointTags, withoutDims []string) *ChunkTags

func (*ChunkTags) DecodeTagsWithoutTag added in v1.3.0

func (ct *ChunkTags) DecodeTagsWithoutTag(tagName string) ([]byte, string)

func (*ChunkTags) GetChunkTagAndValues

func (ct *ChunkTags) GetChunkTagAndValues() ([]string, []string)

func (*ChunkTags) GetChunkTagKVStrings

func (ct *ChunkTags) GetChunkTagKVStrings() string

func (*ChunkTags) GetChunkTagValue

func (ct *ChunkTags) GetChunkTagValue(name string) (string, bool)

func (*ChunkTags) GetOffsets added in v1.0.0

func (c *ChunkTags) GetOffsets() []uint16

func (*ChunkTags) GetTag

func (ct *ChunkTags) GetTag() []byte

func (*ChunkTags) Instance

func (ct *ChunkTags) Instance() transport.Codec

func (*ChunkTags) KeepKeys

func (ct *ChunkTags) KeepKeys(keys []string) *ChunkTags

func (*ChunkTags) KeyValues

func (ct *ChunkTags) KeyValues() map[string]string

func (*ChunkTags) Marshal

func (ct *ChunkTags) Marshal(buf []byte) ([]byte, error)

func (*ChunkTags) PointTags added in v1.0.0

func (ct *ChunkTags) PointTags() influx.PointTags

func (*ChunkTags) RemoveKeys added in v1.3.0

func (ct *ChunkTags) RemoveKeys(keys []string) *ChunkTags

func (*ChunkTags) Reset

func (ct *ChunkTags) Reset()

func (*ChunkTags) Size

func (ct *ChunkTags) Size() int

func (*ChunkTags) Subset

func (ct *ChunkTags) Subset(keys []string) []byte

func (*ChunkTags) Unmarshal

func (ct *ChunkTags) Unmarshal(buf []byte) error

type ChunkTime added in v1.5.0

type ChunkTime interface {
	InitTimeWindow(minTime, maxTime, intervalTime int64, hasInterval, ascending bool, tag ChunkTags)
	Time() []int64
	TruncateTime(int)
	SetTime(time []int64)
	ResetTime(int, int64)
	AppendTime(...int64)
	AppendTimes([]int64)
	TimeByIndex(int) int64
	IntervalIndex() []int
	AppendIntervalIndexes([]int)
	AppendIntervalIndex(int)
	ResetIntervalIndex(...int)
	IntervalLen() int
}

ChunkTime is a time window-related interface.

type ChunkValuer

type ChunkValuer struct {
	// contains filtered or unexported fields
}

ChunkValuer is a valuer that substitutes values for the mapped interface.

func NewChunkValuer

func NewChunkValuer(isPromQuery bool) *ChunkValuer

func (*ChunkValuer) AtChunkRow

func (c *ChunkValuer) AtChunkRow(chunk Chunk, index int)

func (*ChunkValuer) SetValueFnOnlyPromTime added in v1.4.0

func (c *ChunkValuer) SetValueFnOnlyPromTime()

func (*ChunkValuer) SetValuer

func (c *ChunkValuer) SetValuer(_ influxql.Valuer, _ int)

func (*ChunkValuer) Value

func (c *ChunkValuer) Value(key string) (interface{}, bool)

Value returns the value for a key in the MapValuer.

func (*ChunkValuer) ValueNormal added in v1.3.0

func (c *ChunkValuer) ValueNormal(key string) (interface{}, bool)

func (*ChunkValuer) ValueProm added in v1.3.0

func (c *ChunkValuer) ValueProm(key string) (interface{}, bool)

func (*ChunkValuer) ValuePromTime added in v1.4.0

func (c *ChunkValuer) ValuePromTime(key string) (interface{}, bool)

type ChunkWriter

type ChunkWriter interface {
	Write(Chunk)
	Close()
}

type CircularChunkPool

type CircularChunkPool struct {
	// contains filtered or unexported fields
}

func NewCircularChunkPool

func NewCircularChunkPool(chunkNum int, chunkBuilder *ChunkBuilder) *CircularChunkPool

func (*CircularChunkPool) GetChunk

func (cp *CircularChunkPool) GetChunk() Chunk

func (*CircularChunkPool) Release

func (cp *CircularChunkPool) Release()

type Cluster added in v1.0.0

type Cluster struct {
	Mean   float64
	Weight float64
}

func NewCluster added in v1.0.0

func NewCluster() *Cluster

func (*Cluster) Reset added in v1.0.0

func (c *Cluster) Reset()

func (*Cluster) Set added in v1.0.0

func (c *Cluster) Set(m, w float64)

type ClusterPool added in v1.0.0

type ClusterPool struct {
	// contains filtered or unexported fields
}

func NewClusterPool added in v1.0.0

func NewClusterPool() *ClusterPool

func (*ClusterPool) Get added in v1.0.0

func (p *ClusterPool) Get() *Cluster

func (*ClusterPool) Put added in v1.0.0

func (p *ClusterPool) Put(cluster *Cluster)

type ClusterSet added in v1.0.0

type ClusterSet []Cluster

func (ClusterSet) Len added in v1.0.0

func (l ClusterSet) Len() int

func (ClusterSet) Less added in v1.0.0

func (l ClusterSet) Less(i, j int) bool

func (ClusterSet) Swap added in v1.0.0

func (l ClusterSet) Swap(i, j int)

type CoProcessor

type CoProcessor interface {
	WorkOnChunk(Chunk, Chunk, *IteratorParams)
}

func AppendColumnsIteratorHelper

func AppendColumnsIteratorHelper(rowDataType hybridqp.RowDataType) CoProcessor

func FixedColumnsIteratorHelper

func FixedColumnsIteratorHelper(rowDataType hybridqp.RowDataType) CoProcessor

func FixedMergeColumnsIteratorHelper

func FixedMergeColumnsIteratorHelper(rowDataType hybridqp.RowDataType) CoProcessor

func NewAlignCoProcessor

func NewAlignCoProcessor(rowDataType hybridqp.RowDataType) CoProcessor

func NewIntervalCoProcessor

func NewIntervalCoProcessor(rowDataType hybridqp.RowDataType) CoProcessor

func NewSlidingWindowProcessors

func NewSlidingWindowProcessors(
	inRowDataType, outRowDataType hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, opt *query.ProcessorOptions, schema hybridqp.Catalog,
) (CoProcessor, int, int)

func NewWideProcessorImpl added in v0.2.0

func NewWideProcessorImpl(inRowDataType, outRowDataType hybridqp.RowDataType, exprOpts []hybridqp.ExprOptions) (CoProcessor, error)

type CoProcessorImpl

type CoProcessorImpl struct {
	Routines []Routine
}

func NewCoProcessorImpl

func NewCoProcessorImpl(routines ...Routine) *CoProcessorImpl

func (*CoProcessorImpl) AppendRoutine

func (p *CoProcessorImpl) AppendRoutine(routines ...Routine)

func (*CoProcessorImpl) WorkOnChunk

func (p *CoProcessorImpl) WorkOnChunk(in Chunk, out Chunk, params *IteratorParams)

type ColMergeFunc added in v1.3.0

type ColMergeFunc[T util.ExceptString] func(prevPoint, currPoint *Point[T])

type ColReduceFunc added in v1.3.0

type ColReduceFunc[T util.ExceptBool] func(c Chunk, values []T, ordinal, start, end int) (index int, value T, isNil bool)

type Column

type Column interface {
	DataType() influxql.DataType
	Length() int
	NilCount() int
	IsEmpty() bool

	ColumnTime(int) int64
	ColumnTimes() []int64
	AppendColumnTime(int64)
	AppendColumnTimes([]int64)
	SetColumnTimes([]int64)

	IsNilV2(int) bool
	NilsV2() *Bitmap
	SetNilsBitmap(new *Bitmap)
	AppendNil()
	AppendNotNil()
	AppendNilsV2(dat ...bool)
	AppendManyNotNil(num int)
	AppendManyNil(num int)
	GetValueIndexV2(start int) int
	GetRangeValueIndexV2(bmStart, bmEnd int) (s int, e int)
	StringValuesRangeV2(dst []string, start, end int) []string
	GetTimeIndex(valIdx int) int
	Reset()
	FilterColumnByRowIndexes(indexes ...int) Column

	FloatTuple(int) floatTuple
	FloatTuples() []floatTuple
	GetFloatTupleValues(int) []float64
	AppendFloatTuple(floatTuple)
	AppendFloatTuples([]floatTuple)
	SetFloatTuples([]floatTuple)

	FloatValue(int) float64
	FloatValues() []float64
	AppendFloatValue(float64)
	AppendFloatValues([]float64)
	SetFloatValues([]float64)
	UpdateFloatValueFast(v float64, row int)

	IntegerValue(int) int64
	IntegerValues() []int64
	AppendIntegerValue(int64)
	AppendIntegerValues([]int64)
	SetIntegerValues([]int64)
	UpdateIntegerValueFast(v int64, row int)

	StringValue(int) string
	StringValuesV2(dst []string) []string
	StringValuesRange(dst []string, start, end int) []string
	StringValuesWithOffset(start, end int, offsets []uint32) ([]byte, []uint32)
	AppendStringValue(string)
	AppendStringValues([]string)
	AppendStringBytes([]byte, []uint32)
	SetStringValues([]byte, []uint32)
	GetStringBytes() ([]byte, []uint32)
	CloneStringValues([]byte, []uint32)
	GetStringValueBytes(valueBits []byte, value []string, start, end int) ([]byte, []string)

	BooleanValue(int) bool
	BooleanValues() []bool
	AppendBooleanValue(bool)
	AppendBooleanValues([]bool)
	SetBooleanValues([]bool)
	UpdateBooleanValueFast(v bool, row int)

	//TODO:CheckColumn used to check the chunk's structure
	// Remember to remove it!
	CheckColumn(int)

	Marshal([]byte) ([]byte, error)
	Unmarshal([]byte) error
	Size() int
	BitMap() *Bitmap
}

type ColumnImpl

type ColumnImpl struct {
	// contains filtered or unexported fields
}

func NewColumnImpl

func NewColumnImpl(dataType influxql.DataType) *ColumnImpl

func (*ColumnImpl) AppendBooleanValue added in v1.1.0

func (c *ColumnImpl) AppendBooleanValue(value bool)

func (*ColumnImpl) AppendBooleanValues

func (c *ColumnImpl) AppendBooleanValues(values []bool)

func (*ColumnImpl) AppendColumnTime added in v1.1.0

func (c *ColumnImpl) AppendColumnTime(value int64)

func (*ColumnImpl) AppendColumnTimes

func (c *ColumnImpl) AppendColumnTimes(values []int64)

func (*ColumnImpl) AppendFloatTuple added in v1.1.0

func (c *ColumnImpl) AppendFloatTuple(tuple floatTuple)

func (*ColumnImpl) AppendFloatTuples added in v1.0.0

func (c *ColumnImpl) AppendFloatTuples(tuples []floatTuple)

func (*ColumnImpl) AppendFloatValue added in v1.1.0

func (c *ColumnImpl) AppendFloatValue(value float64)

func (*ColumnImpl) AppendFloatValues

func (c *ColumnImpl) AppendFloatValues(values []float64)

func (*ColumnImpl) AppendIntegerValue added in v1.1.0

func (c *ColumnImpl) AppendIntegerValue(value int64)

func (*ColumnImpl) AppendIntegerValues

func (c *ColumnImpl) AppendIntegerValues(values []int64)

func (*ColumnImpl) AppendManyNil

func (c *ColumnImpl) AppendManyNil(num int)

func (*ColumnImpl) AppendManyNotNil

func (c *ColumnImpl) AppendManyNotNil(num int)

func (*ColumnImpl) AppendNil

func (c *ColumnImpl) AppendNil()

func (*ColumnImpl) AppendNilsV2

func (c *ColumnImpl) AppendNilsV2(dat ...bool)

func (*ColumnImpl) AppendNotNil added in v1.1.0

func (c *ColumnImpl) AppendNotNil()

func (*ColumnImpl) AppendStringBytes

func (c *ColumnImpl) AppendStringBytes(val []byte, offset []uint32)

func (*ColumnImpl) AppendStringValue deprecated added in v1.1.0

func (c *ColumnImpl) AppendStringValue(value string)

Deprecated: please do not use. recommend to use AppendStringBytes

func (*ColumnImpl) AppendStringValues deprecated

func (c *ColumnImpl) AppendStringValues(values []string)

Deprecated: please do not use. recommend to use AppendStringBytes

func (*ColumnImpl) BitMap

func (c *ColumnImpl) BitMap() *Bitmap

func (*ColumnImpl) BooleanValue

func (c *ColumnImpl) BooleanValue(idx int) bool

func (*ColumnImpl) BooleanValues

func (c *ColumnImpl) BooleanValues() []bool

func (*ColumnImpl) CheckColumn

func (c *ColumnImpl) CheckColumn(length int)

func (*ColumnImpl) CloneStringValues

func (c *ColumnImpl) CloneStringValues(val []byte, offset []uint32)

func (*ColumnImpl) ColumnTime

func (c *ColumnImpl) ColumnTime(idx int) int64

func (*ColumnImpl) ColumnTimes

func (c *ColumnImpl) ColumnTimes() []int64

func (*ColumnImpl) DataType

func (c *ColumnImpl) DataType() influxql.DataType

func (*ColumnImpl) FilterColumnByRowIndexes added in v1.5.0

func (c *ColumnImpl) FilterColumnByRowIndexes(indexes ...int) Column

Convention: The indexes parameter is a list of valid row IDs in ascending order

func (*ColumnImpl) FloatTuple added in v1.0.0

func (c *ColumnImpl) FloatTuple(idx int) floatTuple

func (*ColumnImpl) FloatTuples added in v1.0.0

func (c *ColumnImpl) FloatTuples() []floatTuple

func (*ColumnImpl) FloatValue

func (c *ColumnImpl) FloatValue(idx int) float64

func (*ColumnImpl) FloatValues

func (c *ColumnImpl) FloatValues() []float64

func (*ColumnImpl) GetFloatTupleValues added in v1.5.0

func (c *ColumnImpl) GetFloatTupleValues(idx int) []float64

func (*ColumnImpl) GetRangeValueIndexV2

func (c *ColumnImpl) GetRangeValueIndexV2(bmStart, bmEnd int) (int, int)

func (*ColumnImpl) GetStringBytes

func (c *ColumnImpl) GetStringBytes() ([]byte, []uint32)

func (*ColumnImpl) GetStringValueBytes

func (c *ColumnImpl) GetStringValueBytes(valueBits []byte, value []string, start, end int) ([]byte, []string)

func (*ColumnImpl) GetTimeIndex

func (c *ColumnImpl) GetTimeIndex(valIdx int) int

func (*ColumnImpl) GetValueIndexV2

func (c *ColumnImpl) GetValueIndexV2(start int) int

func (*ColumnImpl) Instance

func (c *ColumnImpl) Instance() transport.Codec

func (*ColumnImpl) IntegerValue

func (c *ColumnImpl) IntegerValue(idx int) int64

func (*ColumnImpl) IntegerValues

func (c *ColumnImpl) IntegerValues() []int64

func (*ColumnImpl) IsEmpty

func (c *ColumnImpl) IsEmpty() bool

func (*ColumnImpl) IsNilV2

func (c *ColumnImpl) IsNilV2(idx int) bool

func (*ColumnImpl) Length

func (c *ColumnImpl) Length() int

func (*ColumnImpl) Marshal

func (c *ColumnImpl) Marshal(buf []byte) ([]byte, error)

func (*ColumnImpl) NilCount

func (c *ColumnImpl) NilCount() int

func (*ColumnImpl) NilsV2

func (c *ColumnImpl) NilsV2() *Bitmap

func (*ColumnImpl) Reset

func (c *ColumnImpl) Reset()

func (*ColumnImpl) SetBooleanValues

func (c *ColumnImpl) SetBooleanValues(values []bool)

func (*ColumnImpl) SetColumnTimes

func (c *ColumnImpl) SetColumnTimes(values []int64)

func (*ColumnImpl) SetFloatTuples added in v1.0.0

func (c *ColumnImpl) SetFloatTuples(tuples []floatTuple)

func (*ColumnImpl) SetFloatValues

func (c *ColumnImpl) SetFloatValues(values []float64)

func (*ColumnImpl) SetIntegerValues

func (c *ColumnImpl) SetIntegerValues(values []int64)

func (*ColumnImpl) SetNilsBitmap

func (c *ColumnImpl) SetNilsBitmap(new *Bitmap)

SetNilsBitmap just for test use now.

func (*ColumnImpl) SetStringValues

func (c *ColumnImpl) SetStringValues(val []byte, offset []uint32)

func (*ColumnImpl) Size

func (c *ColumnImpl) Size() int

func (*ColumnImpl) StringValue

func (c *ColumnImpl) StringValue(idx int) string

func (*ColumnImpl) StringValuesRange

func (c *ColumnImpl) StringValuesRange(dst []string, start, end int) []string

func (*ColumnImpl) StringValuesRangeV2 added in v1.1.0

func (c *ColumnImpl) StringValuesRangeV2(dst []string, start, end int) []string

func (*ColumnImpl) StringValuesV2

func (c *ColumnImpl) StringValuesV2(dst []string) []string

StringValuesV2 just use for test.

func (*ColumnImpl) StringValuesWithOffset added in v1.1.0

func (c *ColumnImpl) StringValuesWithOffset(start, end int, offsets []uint32) ([]byte, []uint32)

func (*ColumnImpl) Unmarshal

func (c *ColumnImpl) Unmarshal(buf []byte) error

func (*ColumnImpl) UpdateBooleanValueFast added in v1.3.0

func (c *ColumnImpl) UpdateBooleanValueFast(v bool, row int)

func (*ColumnImpl) UpdateFloatValueFast added in v1.2.0

func (c *ColumnImpl) UpdateFloatValueFast(v float64, row int)

func (*ColumnImpl) UpdateIntegerValueFast added in v1.2.0

func (c *ColumnImpl) UpdateIntegerValueFast(v int64, row int)

type ConditionExprVisitor

type ConditionExprVisitor struct {
	// contains filtered or unexported fields
}

func NewConditionExprVisitor

func NewConditionExprVisitor() *ConditionExprVisitor

func (*ConditionExprVisitor) Visit

type ContextKey

type ContextKey string
const (
	NowKey ContextKey = "now"
)

type CountValuesIterator added in v1.3.0

type CountValuesIterator struct {
	// contains filtered or unexported fields
}

func NewCountValuesIterator added in v1.3.0

func NewCountValuesIterator(inOrdinal, outOrdinal int, tagName string) *CountValuesIterator

func (*CountValuesIterator) Next added in v1.3.0

type CountValuesOp added in v1.3.0

type CountValuesOp struct{}

func (*CountValuesOp) CreateRoutine added in v1.3.0

func (c *CountValuesOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type Crash added in v1.3.0

type Crash struct {
	ClientID    uint64
	QueryID     uint64
	NoMarkCrash bool
}

func NewCrash added in v1.3.0

func NewCrash(queryID uint64, clientID uint64, noMarkCrash bool) *Crash

func (*Crash) Instance added in v1.3.0

func (c *Crash) Instance() transport.Codec

func (*Crash) Marshal added in v1.3.0

func (c *Crash) Marshal(buf []byte) ([]byte, error)

func (*Crash) Size added in v1.3.0

func (c *Crash) Size() int

func (*Crash) Unmarshal added in v1.3.0

func (c *Crash) Unmarshal(buf []byte) error

type CsStoreExchangeTraits added in v1.2.0

type CsStoreExchangeTraits struct {
	// contains filtered or unexported fields
}

func NewCsStoreExchangeTraits added in v1.2.0

func NewCsStoreExchangeTraits(w spdy.Responser, PtQuerys []PtQuery) *CsStoreExchangeTraits

type DAG

type DAG struct {
	ProcessorMap map[Processor]VertexId
	Vertexs      Vertexs
}

func NewDAG

func NewDAG(processors Processors) *DAG

func (*DAG) AddEdge

func (dag *DAG) AddEdge(edges Edges, edge *Edge, from Processor, to Processor) (Edges, error)

func (*DAG) AddEdges

func (dag *DAG) AddEdges(id VertexId, inputmap map[uintptr]VertexId, outputmap map[uintptr]VertexId) error

func (*DAG) CyclicGraph

func (dag *DAG) CyclicGraph() bool

func (*DAG) Explain

func (dag *DAG) Explain(fn func(*DAG, *strings.Builder) VertexWriter) VertexWriter

func (*DAG) OrphanVertexs

func (dag *DAG) OrphanVertexs() []VertexId

func (*DAG) Path

func (dag *DAG) Path() int

func (*DAG) Processors

func (dag *DAG) Processors() Processors

func (*DAG) SinkVertexs

func (dag *DAG) SinkVertexs() []VertexId

func (*DAG) Size

func (dag *DAG) Size() int

func (*DAG) SourceVertexs

func (dag *DAG) SourceVertexs() []VertexId

func (*DAG) Span

func (dag *DAG) Span() int

func (*DAG) Walk

func (dag *DAG) Walk(id VertexId, backward bool, fn WalkFn, m map[VertexId]int) error

type DAGBuilder

type DAGBuilder struct {
	// contains filtered or unexported fields
}

func NewDAGBuilder

func NewDAGBuilder(opt query.ProcessorOptions) *DAGBuilder

func (*DAGBuilder) Build

func (b *DAGBuilder) Build(plan hybridqp.QueryNode) (*DAG, error)

func (*DAGBuilder) Visit

type DataCacheCenter added in v1.5.0

type DataCacheCenter struct {
	// contains filtered or unexported fields
}

func NewDataCacheCenter added in v1.5.0

func NewDataCacheCenter() DataCacheCenter

func (*DataCacheCenter) ClearDataStoreByKey added in v1.5.0

func (dataCacheCenter *DataCacheCenter) ClearDataStoreByKey(keyPrefix string, CTEs influxql.CTES)

type DeleteClusterSet added in v1.0.0

type DeleteClusterSet map[float64]float64

type DetachedFrags added in v1.2.0

type DetachedFrags struct {
	BaseFrags
	// contains filtered or unexported fields
}

func NewDetachedFrags added in v1.2.0

func NewDetachedFrags(basePath string, cap int) *DetachedFrags

func (*DetachedFrags) AppendIndexes added in v1.2.0

func (s *DetachedFrags) AppendIndexes(metaIndexes ...interface{})

func (*DetachedFrags) IndexCount added in v1.2.0

func (s *DetachedFrags) IndexCount() int

func (*DetachedFrags) Indexes added in v1.2.0

func (s *DetachedFrags) Indexes() interface{}

func (*DetachedFrags) Size added in v1.2.0

func (s *DetachedFrags) Size() int

type DetachedIndexInfo added in v1.2.0

type DetachedIndexInfo struct {
	// contains filtered or unexported fields
}

func NewDetachedIndexInfo added in v1.2.0

func NewDetachedIndexInfo(files []*immutable.MetaIndex, infos []*colstore.DetachedPKInfo) *DetachedIndexInfo

func (*DetachedIndexInfo) Files added in v1.2.0

func (a *DetachedIndexInfo) Files() []*immutable.MetaIndex

func (*DetachedIndexInfo) Infos added in v1.2.0

type DistinctItem added in v1.3.0

type DistinctItem[T util.ExceptBool] struct {
	// contains filtered or unexported fields
}

func NewDistinctItem added in v1.3.0

func NewDistinctItem[T util.ExceptBool]() *DistinctItem[T]

func (*DistinctItem[T]) Len added in v1.3.0

func (f *DistinctItem[T]) Len() int

func (*DistinctItem[T]) Less added in v1.3.0

func (f *DistinctItem[T]) Less(i, j int) bool

func (*DistinctItem[T]) Nil added in v1.3.0

func (f *DistinctItem[T]) Nil() bool

func (*DistinctItem[T]) Reset added in v1.3.0

func (f *DistinctItem[T]) Reset()

func (*DistinctItem[T]) Swap added in v1.3.0

func (f *DistinctItem[T]) Swap(i, j int)

type DistinctPushDownToExchangeRule added in v1.5.0

type DistinctPushDownToExchangeRule struct {
	OptRuleBase
}

func NewDistinctPushDownToExchangeRule added in v1.5.0

func NewDistinctPushDownToExchangeRule(description string) *DistinctPushDownToExchangeRule

func (*DistinctPushDownToExchangeRule) Category added in v1.5.0

func (*DistinctPushDownToExchangeRule) Equals added in v1.5.0

func (*DistinctPushDownToExchangeRule) OnMatch added in v1.5.0

func (r *DistinctPushDownToExchangeRule) OnMatch(call *OptRuleCall)

func (*DistinctPushDownToExchangeRule) ToString added in v1.5.0

func (r *DistinctPushDownToExchangeRule) ToString() string

type DistinctTransform added in v1.5.0

type DistinctTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewDistinctTransform added in v1.5.0

func NewDistinctTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, schema hybridqp.Catalog) *DistinctTransform

func (*DistinctTransform) Close added in v1.5.0

func (d *DistinctTransform) Close()

func (*DistinctTransform) Explain added in v1.5.0

func (d *DistinctTransform) Explain() []ValuePair

func (*DistinctTransform) GetInputNumber added in v1.5.0

func (d *DistinctTransform) GetInputNumber(port Port) int

func (*DistinctTransform) GetInputs added in v1.5.0

func (d *DistinctTransform) GetInputs() Ports

func (*DistinctTransform) GetOutputNumber added in v1.5.0

func (d *DistinctTransform) GetOutputNumber(port Port) int

func (*DistinctTransform) GetOutputs added in v1.5.0

func (d *DistinctTransform) GetOutputs() Ports

func (*DistinctTransform) Name added in v1.5.0

func (d *DistinctTransform) Name() string

func (*DistinctTransform) Work added in v1.5.0

func (d *DistinctTransform) Work(ctx context.Context) error

type DistinctTransformCreator added in v1.5.0

type DistinctTransformCreator struct {
}

func (*DistinctTransformCreator) Create added in v1.5.0

type DownSampleState added in v1.0.0

type DownSampleState struct {
	// contains filtered or unexported fields
}

func NewDownSampleState added in v1.0.0

func NewDownSampleState(taskID int, err error, newFiles []immutable.TSSPFile) *DownSampleState

func (*DownSampleState) GetErr added in v1.0.0

func (p *DownSampleState) GetErr() error

func (*DownSampleState) GetNewFiles added in v1.0.0

func (p *DownSampleState) GetNewFiles() []immutable.TSSPFile

func (*DownSampleState) GetTaskID added in v1.0.0

func (p *DownSampleState) GetTaskID() int

type DownSampleStatePort added in v1.0.0

type DownSampleStatePort struct {
	RowDataType hybridqp.RowDataType
	State       chan *DownSampleState
	OrigiState  chan *DownSampleState
	Redirected  bool
	// contains filtered or unexported fields
}

func NewDownSampleStatePort added in v1.0.0

func NewDownSampleStatePort(rowDataType hybridqp.RowDataType) *DownSampleStatePort

func (*DownSampleStatePort) Close added in v1.0.0

func (p *DownSampleStatePort) Close()

func (*DownSampleStatePort) Connect added in v1.0.0

func (p *DownSampleStatePort) Connect(to Port)

func (*DownSampleStatePort) ConnectStateReserve added in v1.0.0

func (p *DownSampleStatePort) ConnectStateReserve(to Port)

func (*DownSampleStatePort) ConnectionId added in v1.0.0

func (p *DownSampleStatePort) ConnectionId() uintptr

func (*DownSampleStatePort) Equal added in v1.0.0

func (p *DownSampleStatePort) Equal(to Port) bool

func (*DownSampleStatePort) Redirect added in v1.0.0

func (p *DownSampleStatePort) Redirect(to Port)

func (*DownSampleStatePort) Release added in v1.0.0

func (p *DownSampleStatePort) Release()

type Edge

type Edge struct {
	To           VertexId
	Backward     bool
	InputNumber  int
	OutputNumber int
}

func NewEdge

func NewEdge(to VertexId, backward bool, inputNumber int, outputNumber int) *Edge

type EdgeMetaData added in v1.5.0

type EdgeMetaData struct {
	Kind          string            `json:"kind"`
	SourceTopoKey string            `json:"sourceTopoKey"`
	SourceUid     string            `json:"sourceUid"`
	TargetTopoKey string            `json:"targetTopoKey"`
	TargetUid     string            `json:"targetUid"`
	Tags          map[string]string `json:"tags"`
}

type Edges

type Edges []*Edge

type ElapsedItem

type ElapsedItem struct {
	// contains filtered or unexported fields
}

func NewElapsedItem

func NewElapsedItem(interval hybridqp.Interval) *ElapsedItem

func (*ElapsedItem) AppendItem

func (f *ElapsedItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*ElapsedItem) AppendItemFastFunc

func (f *ElapsedItem) AppendItemFastFunc(c Chunk, _ int, start, end int, sameInterval bool)

func (*ElapsedItem) AppendItemSlowFunc

func (f *ElapsedItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*ElapsedItem) GetBaseTransData

func (f *ElapsedItem) GetBaseTransData() BaseTransData

func (*ElapsedItem) Len

func (f *ElapsedItem) Len() int

func (*ElapsedItem) PrevNil

func (f *ElapsedItem) PrevNil() bool

func (*ElapsedItem) Reset

func (f *ElapsedItem) Reset()

func (*ElapsedItem) ResetPrev

func (f *ElapsedItem) ResetPrev()

type EndPointPair

type EndPointPair struct {
	Chunk   Chunk
	Ordinal int
}

type EquivalenceExprRewriter

type EquivalenceExprRewriter struct {
}

func (*EquivalenceExprRewriter) Visit

type Error

type Error struct {
	// contains filtered or unexported fields
}

func (*Error) Instance

func (e *Error) Instance() transport.Codec

func (*Error) Marshal

func (e *Error) Marshal(buf []byte) ([]byte, error)

func (*Error) Size

func (e *Error) Size() int

func (*Error) Unmarshal

func (e *Error) Unmarshal(buf []byte) error

type Exchange added in v1.2.0

type Exchange interface {
	hybridqp.QueryNode
	Schema() hybridqp.Catalog
	EType() ExchangeType
	ERole() ExchangeRole
	ETraits() []hybridqp.Trait
	AddTrait(trait interface{})
	ToProducer()
}

type ExchangeRole

type ExchangeRole uint8
const (
	UNKNOWN_ROLE ExchangeRole = iota
	CONSUMER_ROLE
	PRODUCER_ROLE
)

type ExchangeType

type ExchangeType uint8
const (
	UNKNOWN_EXCHANGE ExchangeType = iota
	NODE_EXCHANGE
	SHARD_EXCHANGE
	SINGLE_SHARD_EXCHANGE
	READER_EXCHANGE
	SERIES_EXCHANGE
	SEGMENT_EXCHANGE
	PARTITION_EXCHANGE
	SUBQUERY_EXCHANGE
)

type ExecutorBuilder

type ExecutorBuilder struct {
	// contains filtered or unexported fields
}

func NewColStoreScanExecutorBuilder added in v1.2.0

func NewColStoreScanExecutorBuilder(traits *StoreExchangeTraits, indexInfo interface{}, info *IndexScanExtraInfo) *ExecutorBuilder

func NewCsStoreExecutorBuilder added in v1.2.0

func NewCsStoreExecutorBuilder(traits *CsStoreExchangeTraits, s hybridqp.StoreEngine,
	req *RemoteQuery, ctx context.Context, limitSize int) *ExecutorBuilder

func NewIndexScanExecutorBuilder

func NewIndexScanExecutorBuilder(traits *StoreExchangeTraits, enableBinaryTreeMerge int64) *ExecutorBuilder

func NewMocStoreExecutorBuilder

func NewMocStoreExecutorBuilder(traits *StoreExchangeTraits, csTraits *CsStoreExchangeTraits, info *IndexScanExtraInfo,
	enableBinaryTreeMerge int64) *ExecutorBuilder

func NewQueryExecutorBuilder

func NewQueryExecutorBuilder(enableBinaryTreeMerge int64) *ExecutorBuilder

func NewScannerStoreExecutorBuilder

func NewScannerStoreExecutorBuilder(traits *StoreExchangeTraits, s hybridqp.StoreEngine,
	req *RemoteQuery, ctx context.Context, limitSize int) *ExecutorBuilder

func NewSparseIndexScanExecutorBuilder added in v1.1.0

func NewSparseIndexScanExecutorBuilder(frags *ShardsFragmentsGroups, info *IndexScanExtraInfo) *ExecutorBuilder

func NewStoreExecutorBuilder

func NewStoreExecutorBuilder(traits *StoreExchangeTraits, enableBinaryTreeMerge int64) *ExecutorBuilder

func (*ExecutorBuilder) Analyze

func (builder *ExecutorBuilder) Analyze(span *tracing.Span)

func (*ExecutorBuilder) Build

func (builder *ExecutorBuilder) Build(node hybridqp.QueryNode) (hybridqp.Executor, error)

func (*ExecutorBuilder) CanOptimizeExchange added in v1.4.0

func (builder *ExecutorBuilder) CanOptimizeExchange(node hybridqp.QueryNode, children []*TransformVertex) (*TransformVertex, bool)

CanOptimizeExchange used for optimizing one shard or multiple shards in the same prom PT. Eliminate redundant Merge and Agg on the IndexScan upper layer.

func (*ExecutorBuilder) IsMultiMstPlanNode added in v1.3.0

func (builder *ExecutorBuilder) IsMultiMstPlanNode(node hybridqp.QueryNode) bool

func (*ExecutorBuilder) NextInfo added in v1.2.0

func (builder *ExecutorBuilder) NextInfo()

ts-server + multiPt nextInfo is use for nextPt

func (*ExecutorBuilder) NextMst added in v1.2.0

func (builder *ExecutorBuilder) NextMst()

func (*ExecutorBuilder) SetInfo added in v1.0.0

func (builder *ExecutorBuilder) SetInfo(info *IndexScanExtraInfo)

func (*ExecutorBuilder) SetInfosAndTraits added in v1.2.0

func (builder *ExecutorBuilder) SetInfosAndTraits(ctx context.Context, mstsReqs []*MultiMstReqs, store hybridqp.StoreEngine, w spdy.Responser)

func (*ExecutorBuilder) SetMultiMstInfosForLocalStore added in v1.2.0

func (builder *ExecutorBuilder) SetMultiMstInfosForLocalStore(t []*IndexScanExtraInfo)

func (*ExecutorBuilder) SetMultiMstTraitsForLocalStore added in v1.2.0

func (builder *ExecutorBuilder) SetMultiMstTraitsForLocalStore(t []*StoreExchangeTraits)

func (*ExecutorBuilder) SetTraits added in v1.2.0

func (builder *ExecutorBuilder) SetTraits(t *StoreExchangeTraits)

type ExprRewriteRule

type ExprRewriteRule interface {
	Rewrite(hybridqp.Catalog)
	String() string
}

type ExprRewriteRules

type ExprRewriteRules []ExprRewriteRule

func (ExprRewriteRules) Rewrite

func (rs ExprRewriteRules) Rewrite(schema hybridqp.Catalog)

type FieldValuer added in v1.0.0

type FieldValuer interface {
	At(Column, int, *influx.Field) bool
}

func NewFieldValuer added in v1.0.0

func NewFieldValuer(ref *influxql.VarRef) (FieldValuer, error)

type FieldsValuer added in v1.0.0

type FieldsValuer struct {
	// contains filtered or unexported fields
}

func NewFieldsValuer added in v1.0.0

func NewFieldsValuer(rdt hybridqp.RowDataType) (*FieldsValuer, error)

func (*FieldsValuer) At added in v1.0.0

func (valuer *FieldsValuer) At(chunk Chunk, pos int, tuple *TargetTuple)

type FileFragment added in v1.1.0

type FileFragment interface {
	GetFile() immutable.TSSPFile
	GetFragmentRanges() fragment.FragmentRanges
	GetFragmentRange(int) *fragment.FragmentRange
	AppendFragmentRange(fragment.FragmentRanges)
	AppendFragmentRangeDetails(fragment.FragmentRangeDetails)
	FragmentCount() int64
	CutTo(num int64) FileFragment
	GetFragmentRangeDetails() fragment.FragmentRangeDetails
}

type FileFragmentImpl added in v1.1.0

type FileFragmentImpl struct {
	// contains filtered or unexported fields
}

func NewFileFragment added in v1.1.0

func NewFileFragmentForTest added in v1.5.0

func NewFileFragmentForTest(f immutable.TSSPFile, fr fragment.FragmentRanges, fc int64) *FileFragmentImpl

func (*FileFragmentImpl) AppendFragmentRange added in v1.1.0

func (f *FileFragmentImpl) AppendFragmentRange(frs fragment.FragmentRanges)

func (*FileFragmentImpl) AppendFragmentRangeDetails added in v1.5.0

func (f *FileFragmentImpl) AppendFragmentRangeDetails(frds fragment.FragmentRangeDetails)

func (*FileFragmentImpl) CutTo added in v1.1.0

func (f *FileFragmentImpl) CutTo(num int64) FileFragment

func (*FileFragmentImpl) FragmentCount added in v1.1.0

func (f *FileFragmentImpl) FragmentCount() int64

func (*FileFragmentImpl) GetFile added in v1.1.0

func (f *FileFragmentImpl) GetFile() immutable.TSSPFile

func (*FileFragmentImpl) GetFragmentRange added in v1.1.0

func (f *FileFragmentImpl) GetFragmentRange(i int) *fragment.FragmentRange

func (*FileFragmentImpl) GetFragmentRangeDetails added in v1.5.0

func (f *FileFragmentImpl) GetFragmentRangeDetails() fragment.FragmentRangeDetails

func (*FileFragmentImpl) GetFragmentRanges added in v1.1.0

func (f *FileFragmentImpl) GetFragmentRanges() fragment.FragmentRanges

type FileFragments added in v1.1.0

type FileFragments struct {
	FragmentCount int64
	FileMarks     map[string]FileFragment
}

func NewFileFragments added in v1.1.0

func NewFileFragments() *FileFragments

func (*FileFragments) AddFileFragment added in v1.1.0

func (fms *FileFragments) AddFileFragment(filePath string, fm FileFragment, fc int64)

type FileMode added in v1.2.0

type FileMode uint8
const (
	Detached FileMode = iota
	Attached
)

type FillItem

type FillItem struct {
	// contains filtered or unexported fields
}

func NewFillItem

func NewFillItem() *FillItem

func (*FillItem) ReSet

func (f *FillItem) ReSet()

func (*FillItem) Set

func (f *FillItem) Set(prevReadAt, inputReadAt int, interval, start int64, fillValue interface{})

type FillProcessor

type FillProcessor interface {
	// contains filtered or unexported methods
}

func NewFillProcessor

func NewFillProcessor(rowDataType hybridqp.RowDataType, schema *QuerySchema) ([]FillProcessor, error)

func NewLinearFillProcessor

func NewLinearFillProcessor(rowDataType hybridqp.RowDataType) []FillProcessor

func NewNullFillProcessor

func NewNullFillProcessor(rowDataType hybridqp.RowDataType, schema *QuerySchema) []FillProcessor

func NewNumberFillProcessor

func NewNumberFillProcessor(rowDataType hybridqp.RowDataType) []FillProcessor

func NewPreviousFillProcessor

func NewPreviousFillProcessor(rowDataType hybridqp.RowDataType) []FillProcessor

type FillTransform

type FillTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewFillTransform

func NewFillTransform(inRowDataType []hybridqp.RowDataType, outRowDataType []hybridqp.RowDataType,
	_ []hybridqp.ExprOptions, schema *QuerySchema) (*FillTransform, error)

func (*FillTransform) Close

func (trans *FillTransform) Close()

func (*FillTransform) Explain

func (trans *FillTransform) Explain() []ValuePair

func (*FillTransform) GetInputNumber

func (trans *FillTransform) GetInputNumber(port Port) int

func (*FillTransform) GetInputs

func (trans *FillTransform) GetInputs() Ports

func (*FillTransform) GetOutputNumber

func (trans *FillTransform) GetOutputNumber(port Port) int

func (*FillTransform) GetOutputs

func (trans *FillTransform) GetOutputs() Ports

func (*FillTransform) Name

func (trans *FillTransform) Name() string

func (*FillTransform) NextChunk added in v1.4.0

func (trans *FillTransform) NextChunk()

func (*FillTransform) Work

func (trans *FillTransform) Work(ctx context.Context) error

type FillTransformCreator

type FillTransformCreator struct{}

func (*FillTransformCreator) Create

type FilterBlankTransform

type FilterBlankTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewFilterBlankTransform

func NewFilterBlankTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, opt *query.ProcessorOptions) *FilterBlankTransform

func (*FilterBlankTransform) Close

func (trans *FilterBlankTransform) Close()

func (*FilterBlankTransform) Explain

func (trans *FilterBlankTransform) Explain() []ValuePair

func (*FilterBlankTransform) GetInputNumber

func (trans *FilterBlankTransform) GetInputNumber(port Port) int

func (*FilterBlankTransform) GetInputs

func (trans *FilterBlankTransform) GetInputs() Ports

func (*FilterBlankTransform) GetOutputNumber

func (trans *FilterBlankTransform) GetOutputNumber(port Port) int

func (*FilterBlankTransform) GetOutputs

func (trans *FilterBlankTransform) GetOutputs() Ports

func (*FilterBlankTransform) Name

func (trans *FilterBlankTransform) Name() string

func (*FilterBlankTransform) Work

func (trans *FilterBlankTransform) Work(ctx context.Context) error

type FilterBlankTransformCreator

type FilterBlankTransformCreator struct {
}

func (*FilterBlankTransformCreator) Create

type FilterTransform

type FilterTransform struct {
	BaseProcessor

	Input  *ChunkPort
	Output *ChunkPort

	ResultChunkPool *CircularChunkPool
	CoProcessor     CoProcessor
	// contains filtered or unexported fields
}

func NewFilterTransform

func NewFilterTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, schema *QuerySchema, opt *query.ProcessorOptions) *FilterTransform

func (*FilterTransform) Close

func (trans *FilterTransform) Close()

func (*FilterTransform) Explain

func (trans *FilterTransform) Explain() []ValuePair

func (*FilterTransform) GetInputNumber

func (trans *FilterTransform) GetInputNumber(port Port) int

func (*FilterTransform) GetInputs

func (trans *FilterTransform) GetInputs() Ports

func (*FilterTransform) GetOutputNumber

func (trans *FilterTransform) GetOutputNumber(port Port) int

func (*FilterTransform) GetOutputs

func (trans *FilterTransform) GetOutputs() Ports

func (*FilterTransform) Name

func (trans *FilterTransform) Name() string

func (*FilterTransform) Release

func (trans *FilterTransform) Release() error

func (*FilterTransform) Work

func (trans *FilterTransform) Work(ctx context.Context) error

type FilterTransformCreator

type FilterTransformCreator struct {
}

func (*FilterTransformCreator) Create

type Finish

type Finish struct {
	// contains filtered or unexported fields
}

func (*Finish) Instance

func (e *Finish) Instance() transport.Codec

func (*Finish) Marshal

func (e *Finish) Marshal(buf []byte) ([]byte, error)

func (*Finish) Size

func (e *Finish) Size() int

func (*Finish) Unmarshal

func (e *Finish) Unmarshal(buf []byte) error

type Float64AppendIterator

type Float64AppendIterator struct {
	// contains filtered or unexported fields
}

func NewFloat64AppendIterator

func NewFloat64AppendIterator() *Float64AppendIterator

func (*Float64AppendIterator) Next

func (f *Float64AppendIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type Float64LimitIterator

type Float64LimitIterator struct {
	// contains filtered or unexported fields
}

func NewFloat64LimitIterator

func NewFloat64LimitIterator() *Float64LimitIterator

func (*Float64LimitIterator) Next

func (f *Float64LimitIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type Float64MergeIterator

type Float64MergeIterator struct {
	// contains filtered or unexported fields
}

func NewFloat64MergeIterator

func NewFloat64MergeIterator() *Float64MergeIterator

func (*Float64MergeIterator) Next

func (f *Float64MergeIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type FloatColFloatDistinctIterator

type FloatColFloatDistinctIterator struct {
	// contains filtered or unexported fields
}

func NewFloatColFloatDistinctIterator

func NewFloatColFloatDistinctIterator(
	inOrdinal, outOrdinal int,
) *FloatColFloatDistinctIterator

func (*FloatColFloatDistinctIterator) Next

type FloatColFloatHeapIterator

type FloatColFloatHeapIterator struct {
	// contains filtered or unexported fields
}

func NewFloatColFloatHeapIterator

func NewFloatColFloatHeapIterator(
	inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType, FloatHeapItem *HeapItem[float64],
) *FloatColFloatHeapIterator

func (*FloatColFloatHeapIterator) Next

type FloatColFloatHistogramIterator added in v1.3.0

type FloatColFloatHistogramIterator struct {
	// contains filtered or unexported fields
}

func NewFloatColFloatHistogramIterator added in v1.3.0

func NewFloatColFloatHistogramIterator(fn FloatColReduceHistogramReduce, inOrdinal, outOrdinal int, rowDataType hybridqp.RowDataType) *FloatColFloatHistogramIterator

func (*FloatColFloatHistogramIterator) Next added in v1.3.0

type FloatColFloatRateIterator

type FloatColFloatRateIterator struct {
	// contains filtered or unexported fields
}

func NewFloatRateIterator added in v1.3.0

func NewFloatRateIterator(fn RateMiddleReduceFunc[float64], fv RateFinalReduceFunc[float64],
	fu RateUpdateFunc[float64], fm RateMergeFunc[float64],
	isSingleCall bool, inOrdinal, outOrdinal int, rowDataType hybridqp.RowDataType,
	interval *hybridqp.Interval,
) *FloatColFloatRateIterator

func (*FloatColFloatRateIterator) Next

type FloatColFloatSampleIterator

type FloatColFloatSampleIterator struct {
	// contains filtered or unexported fields
}

func NewFloatColFloatSampleIterator

func NewFloatColFloatSampleIterator(sampleNum int,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *FloatColFloatSampleIterator

func (*FloatColFloatSampleIterator) Next

type FloatColFloatSliceIterator

type FloatColFloatSliceIterator struct {
	// contains filtered or unexported fields
}

func NewFloatColFloatSliceIterator

func NewFloatColFloatSliceIterator(fn SliceReduce[float64],
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *FloatColFloatSliceIterator

func (*FloatColFloatSliceIterator) Next

type FloatColReduceHistogramReduce added in v1.3.0

type FloatColReduceHistogramReduce func(floatItem []bucket) (value float64)

func FloatHistogramQuantilePromReduce added in v1.3.0

func FloatHistogramQuantilePromReduce(p float64) FloatColReduceHistogramReduce

type FloatCountPromOp added in v1.3.0

type FloatCountPromOp struct {
	BasePromOp
}

func (*FloatCountPromOp) CreateRoutine added in v1.3.0

func (c *FloatCountPromOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type FloatCumulativeSumItem

type FloatCumulativeSumItem struct {
	// contains filtered or unexported fields
}

func NewFloatCumulativeSumItem

func NewFloatCumulativeSumItem() *FloatCumulativeSumItem

func (*FloatCumulativeSumItem) AppendItem

func (f *FloatCumulativeSumItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatCumulativeSumItem) AppendItemFastFunc

func (f *FloatCumulativeSumItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatCumulativeSumItem) AppendItemSlowFunc

func (f *FloatCumulativeSumItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatCumulativeSumItem) GetBaseTransData

func (f *FloatCumulativeSumItem) GetBaseTransData() BaseTransData

func (*FloatCumulativeSumItem) Len

func (f *FloatCumulativeSumItem) Len() int

func (*FloatCumulativeSumItem) PrevNil

func (f *FloatCumulativeSumItem) PrevNil() bool

func (*FloatCumulativeSumItem) Reset

func (f *FloatCumulativeSumItem) Reset()

func (*FloatCumulativeSumItem) ResetPrev

func (f *FloatCumulativeSumItem) ResetPrev()

type FloatDerivativeItem

type FloatDerivativeItem struct {
	// contains filtered or unexported fields
}

func NewFloatDerivativeItem

func NewFloatDerivativeItem(isNonNegative, ascending bool, interval hybridqp.Interval) *FloatDerivativeItem

func (*FloatDerivativeItem) AppendItem

func (f *FloatDerivativeItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDerivativeItem) AppendItemFastFunc

func (f *FloatDerivativeItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDerivativeItem) AppendItemSlowFunc

func (f *FloatDerivativeItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDerivativeItem) GetBaseTransData

func (f *FloatDerivativeItem) GetBaseTransData() BaseTransData

func (*FloatDerivativeItem) Len

func (f *FloatDerivativeItem) Len() int

func (*FloatDerivativeItem) PrevNil

func (f *FloatDerivativeItem) PrevNil() bool

func (*FloatDerivativeItem) Reset

func (f *FloatDerivativeItem) Reset()

func (*FloatDerivativeItem) ResetPrev

func (f *FloatDerivativeItem) ResetPrev()

type FloatDifferenceItem

type FloatDifferenceItem struct {
	// contains filtered or unexported fields
}

func NewFloatDifferenceItem

func NewFloatDifferenceItem(isNonNegative bool, diff floatDifference) *FloatDifferenceItem

func (*FloatDifferenceItem) AppendItem

func (f *FloatDifferenceItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDifferenceItem) AppendItemFastFunc

func (f *FloatDifferenceItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDifferenceItem) AppendItemSlowFunc

func (f *FloatDifferenceItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatDifferenceItem) GetBaseTransData

func (f *FloatDifferenceItem) GetBaseTransData() BaseTransData

func (*FloatDifferenceItem) Len

func (f *FloatDifferenceItem) Len() int

func (*FloatDifferenceItem) PrevNil

func (f *FloatDifferenceItem) PrevNil() bool

func (*FloatDifferenceItem) Reset

func (f *FloatDifferenceItem) Reset()

func (*FloatDifferenceItem) ResetPrev

func (f *FloatDifferenceItem) ResetPrev()

type FloatFieldValuer added in v1.0.0

type FloatFieldValuer struct {
	// contains filtered or unexported fields
}

func (*FloatFieldValuer) At added in v1.0.0

func (valuer *FloatFieldValuer) At(col Column, pos int, field *influx.Field) bool

type FloatIntegralIterator added in v1.3.0

type FloatIntegralIterator struct {
	// contains filtered or unexported fields
}

func NewFloatIntegralIterator added in v1.3.0

func NewFloatIntegralIterator(
	isSingleCall bool, inOrdinal, outOrdinal int, interval hybridqp.Interval,
	opt *query.ProcessorOptions,
) *FloatIntegralIterator

func (*FloatIntegralIterator) Next added in v1.3.0

type FloatIntervalIterator

type FloatIntervalIterator struct{}

func NewFloatIntervalIterator

func NewFloatIntervalIterator() *FloatIntervalIterator

func (*FloatIntervalIterator) Next

func (f *FloatIntervalIterator) Next(endpoint *IteratorEndpoint, _ *IteratorParams)

type FloatIterator added in v1.3.0

type FloatIterator struct {
	// contains filtered or unexported fields
}

func NewFloatIterator added in v1.3.0

func NewFloatIterator(fn ColReduceFunc[float64], fv ColMergeFunc[float64],
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *FloatIterator

func (*FloatIterator) Next added in v1.3.0

type FloatLinearFillProcessor

type FloatLinearFillProcessor struct {
	// contains filtered or unexported fields
}

func NewFloatLinearFillProcessor

func NewFloatLinearFillProcessor(inOrdinal, outOrdinal int) *FloatLinearFillProcessor

type FloatMovingAverageItem

type FloatMovingAverageItem struct {
	// contains filtered or unexported fields
}

func NewFloatMovingAverageItem

func NewFloatMovingAverageItem(n int) *FloatMovingAverageItem

func (*FloatMovingAverageItem) AppendItem

func (f *FloatMovingAverageItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatMovingAverageItem) AppendItemFastFunc

func (f *FloatMovingAverageItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatMovingAverageItem) AppendItemSlowFunc

func (f *FloatMovingAverageItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*FloatMovingAverageItem) GetBaseTransData

func (f *FloatMovingAverageItem) GetBaseTransData() BaseTransData

func (*FloatMovingAverageItem) Len

func (f *FloatMovingAverageItem) Len() int

func (*FloatMovingAverageItem) PrevNil

func (f *FloatMovingAverageItem) PrevNil() bool

func (*FloatMovingAverageItem) Reset

func (f *FloatMovingAverageItem) Reset()

func (*FloatMovingAverageItem) ResetPrev

func (f *FloatMovingAverageItem) ResetPrev()

type FloatNullFillProcessor

type FloatNullFillProcessor struct {
	// contains filtered or unexported fields
}

func NewFloatNullFillProcessor

func NewFloatNullFillProcessor(inOrdinal, outOrdinal int) *FloatNullFillProcessor

type FloatNumberFillProcessor

type FloatNumberFillProcessor struct {
	// contains filtered or unexported fields
}

func NewFloatNumberFillProcessor

func NewFloatNumberFillProcessor(inOrdinal, outOrdinal int) *FloatNumberFillProcessor

type FloatOGSketchInsertItem added in v1.0.0

type FloatOGSketchInsertItem struct {
	// contains filtered or unexported fields
}

func NewFloatOGSketchInsertIem added in v1.0.0

func NewFloatOGSketchInsertIem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *FloatOGSketchInsertItem

func (*FloatOGSketchInsertItem) IsNil added in v1.0.0

func (o *FloatOGSketchInsertItem) IsNil() bool

func (*FloatOGSketchInsertItem) Reset added in v1.0.0

func (o *FloatOGSketchInsertItem) Reset()

func (*FloatOGSketchInsertItem) UpdateCluster added in v1.0.0

func (o *FloatOGSketchInsertItem) UpdateCluster(inChunk Chunk, start, end int)

func (*FloatOGSketchInsertItem) WriteResult added in v1.0.0

func (o *FloatOGSketchInsertItem) WriteResult(outChunk Chunk, time int64)

type FloatOGSketchPercentileItem added in v1.0.0

type FloatOGSketchPercentileItem struct {
	// contains filtered or unexported fields
}

func NewFloatOGSketchPercentileItem added in v1.0.0

func NewFloatOGSketchPercentileItem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *FloatOGSketchPercentileItem

func (*FloatOGSketchPercentileItem) IsNil added in v1.0.0

func (o *FloatOGSketchPercentileItem) IsNil() bool

func (*FloatOGSketchPercentileItem) Reset added in v1.0.0

func (o *FloatOGSketchPercentileItem) Reset()

func (*FloatOGSketchPercentileItem) UpdateCluster added in v1.0.0

func (o *FloatOGSketchPercentileItem) UpdateCluster(inChunk Chunk, start, end int)

func (*FloatOGSketchPercentileItem) WriteResult added in v1.0.0

func (o *FloatOGSketchPercentileItem) WriteResult(outChunk Chunk, time int64)

type FloatPercentileApproxItem added in v1.0.0

type FloatPercentileApproxItem struct {
	// contains filtered or unexported fields
}

func NewFloatPercentileApproxItem added in v1.0.0

func NewFloatPercentileApproxItem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *FloatPercentileApproxItem

func (*FloatPercentileApproxItem) IsNil added in v1.0.0

func (o *FloatPercentileApproxItem) IsNil() bool

func (*FloatPercentileApproxItem) Reset added in v1.0.0

func (o *FloatPercentileApproxItem) Reset()

func (*FloatPercentileApproxItem) UpdateCluster added in v1.0.0

func (o *FloatPercentileApproxItem) UpdateCluster(inChunk Chunk, start, end int)

func (*FloatPercentileApproxItem) WriteResult added in v1.0.0

func (o *FloatPercentileApproxItem) WriteResult(outChunk Chunk, time int64)

type FloatPreviousFillProcessor

type FloatPreviousFillProcessor struct {
	// contains filtered or unexported fields
}

func NewFloatPreviousFillProcessor

func NewFloatPreviousFillProcessor(inOrdinal, outOrdinal int) *FloatPreviousFillProcessor

type FloatTimeColFloatIterator

type FloatTimeColFloatIterator struct {
	// contains filtered or unexported fields
}

func NewFloatTimeColFloatIterator

func NewFloatTimeColFloatIterator(
	fn TimeColReduceFunc[float64], fv ColMergeFunc[float64], inOrdinal, outOrdinal int,
) *FloatTimeColFloatIterator

func (*FloatTimeColFloatIterator) Next

type FloatTransIterator added in v1.3.0

type FloatTransIterator struct {
	// contains filtered or unexported fields
}

func NewFloatTransIterator added in v1.3.0

func NewFloatTransIterator(
	isSingleCall bool, inOrdinal, outOrdinal int, transItem TransItem,
) *FloatTransIterator

func (*FloatTransIterator) Next added in v1.3.0

type FloatTupleMergeIterator added in v1.0.0

type FloatTupleMergeIterator struct {
	BaseMergeIterator
	// contains filtered or unexported fields
}

func NewFloatTupleMergeIterator added in v1.0.0

func NewFloatTupleMergeIterator() *FloatTupleMergeIterator

func (*FloatTupleMergeIterator) Next added in v1.0.0

func (f *FloatTupleMergeIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type FrequentResult added in v1.4.0

type FrequentResult struct {
	// contains filtered or unexported fields
}

func (*FrequentResult) Len added in v1.4.0

func (fr *FrequentResult) Len() int

func (*FrequentResult) Less added in v1.4.0

func (fr *FrequentResult) Less(i, j int) bool

func (*FrequentResult) Swap added in v1.4.0

func (fr *FrequentResult) Swap(i, j int)

type FullJoinTransform added in v1.0.0

type FullJoinTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewFullJoinTransform added in v1.0.0

func NewFullJoinTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataType hybridqp.RowDataType,
	joinCase *influxql.Join, schema *QuerySchema) (*FullJoinTransform, error)

func (*FullJoinTransform) Close added in v1.0.0

func (trans *FullJoinTransform) Close()

func (*FullJoinTransform) Explain added in v1.0.0

func (trans *FullJoinTransform) Explain() []ValuePair

func (*FullJoinTransform) GetInputNumber added in v1.0.0

func (trans *FullJoinTransform) GetInputNumber(_ Port) int

func (*FullJoinTransform) GetInputs added in v1.0.0

func (trans *FullJoinTransform) GetInputs() Ports

func (*FullJoinTransform) GetOutputNumber added in v1.0.0

func (trans *FullJoinTransform) GetOutputNumber(_ Port) int

func (*FullJoinTransform) GetOutputs added in v1.0.0

func (trans *FullJoinTransform) GetOutputs() Ports

func (*FullJoinTransform) Name added in v1.0.0

func (trans *FullJoinTransform) Name() string

func (*FullJoinTransform) NewChunkElem added in v1.0.0

func (trans *FullJoinTransform) NewChunkElem(chunk Chunk, seriesKeyLoc int, seriesValLoc int) *chunkElem

func (*FullJoinTransform) SendChunk added in v1.0.0

func (trans *FullJoinTransform) SendChunk()

func (*FullJoinTransform) Work added in v1.0.0

func (trans *FullJoinTransform) Work(ctx context.Context) error

type FullJoinTransformCreator added in v1.0.0

type FullJoinTransformCreator struct {
}

func (*FullJoinTransformCreator) Create added in v1.0.0

type GenericAlignIterator added in v1.5.0

type GenericAlignIterator struct {
	// contains filtered or unexported fields
}

func NewGenericAlignIterator added in v1.5.0

func NewGenericAlignIterator(fn GetColumnValueFunc, fv AppendColumnValueFunc) *GenericAlignIterator

func (*GenericAlignIterator) Next added in v1.5.0

func (f *GenericAlignIterator) Next(endpoint *IteratorEndpoint, p *IteratorParams)

type GetColumnValueFunc added in v1.5.0

type GetColumnValueFunc func(inColumn Column, start int, end int) (*ValueKeeper, bool)

type GetNodeIdOp added in v1.5.0

type GetNodeIdOp struct{}

func (*GetNodeIdOp) Run added in v1.5.0

func (c *GetNodeIdOp) Run(params *TableFunctionParams) ([]Chunk, error)

type Graph added in v1.5.0

type Graph struct {
	Nodes map[string]GraphNode
	Edges map[string]GraphEdge
}

func FaultDemarcation added in v1.5.0

func FaultDemarcation(chunks []Chunk, subTopo *Graph, algoParams AlgoParam, colMap map[string]int) (graph *Graph, err error)

func NewGraph added in v1.5.0

func NewGraph() *Graph

func (*Graph) BatchInsertEdges added in v1.5.0

func (G *Graph) BatchInsertEdges(graphData GraphData) (bool, error)

func (*Graph) BatchInsertNodes added in v1.5.0

func (G *Graph) BatchInsertNodes(graphData GraphData) bool

func (*Graph) CreateGraph added in v1.5.0

func (G *Graph) CreateGraph(jsonGraphData string) (bool, error)

func (*Graph) GetEdgeInfo added in v1.5.0

func (G *Graph) GetEdgeInfo(id string) *GraphEdge

func (*Graph) GetNodeInfo added in v1.5.0

func (G *Graph) GetNodeInfo(id string) *GraphNode

func (*Graph) GraphToRows added in v1.5.0

func (G *Graph) GraphToRows() models.Rows

func (*Graph) MultiHopFilter added in v1.5.0

func (G *Graph) MultiHopFilter(startNodeId string, hopNum int, nodeCondition influxql.Expr, edgeCondition influxql.Expr) (*Graph, error)

type GraphData added in v1.5.0

type GraphData struct {
	ResultUid string   `json:"resultUid"`
	MetaData  MetaData `json:"metadata"`
	Graph     TopoInfo `json:"graph"`
}

type GraphEdge added in v1.5.0

type GraphEdge struct {
	Uid      string       `json:"uid"`
	MetaData EdgeMetaData `json:"metadata"`
}

type GraphFilterFn added in v1.5.0

type GraphFilterFn func(chunks []Chunk, subTopo *Graph, algoParams AlgoParam, colMap map[string]int) (*Graph, error)

type GraphFilterIterator added in v1.5.0

type GraphFilterIterator struct {
	// contains filtered or unexported fields
}

func NewGraphFilterIterator added in v1.5.0

func NewGraphFilterIterator(fn GraphFilterFn, outOrdinal int, params AlgoParam, ordinalMap map[int]int) *GraphFilterIterator

func (*GraphFilterIterator) Next added in v1.5.0

type GraphIterator

type GraphIterator struct {
	// contains filtered or unexported fields
}

func NewGraphIterator

func NewGraphIterator(capacity int) *GraphIterator

func (*GraphIterator) HasNext

func (iter *GraphIterator) HasNext() bool

func (*GraphIterator) Next

func (iter *GraphIterator) Next() *HeuVertex

func (*GraphIterator) Reset

func (iter *GraphIterator) Reset()

func (*GraphIterator) Size

func (iter *GraphIterator) Size() int

func (*GraphIterator) Visit

func (iter *GraphIterator) Visit(vertex *HeuVertex) HeuDagVisitor

type GraphNode added in v1.5.0

type GraphNode struct {
	Uid      string       `json:"uid"`
	MetaData NodeMetaData `json:"metadata"`
	OutEdges []string
	InEdges  []string
}

type GraphTransform added in v1.5.0

type GraphTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewGraphTransform added in v1.5.0

func NewGraphTransform(stmt *influxql.GraphStatement) (*GraphTransform, error)

func (*GraphTransform) Close added in v1.5.0

func (trans *GraphTransform) Close()

func (*GraphTransform) Explain added in v1.5.0

func (trans *GraphTransform) Explain() []ValuePair

func (*GraphTransform) GetInputNumber added in v1.5.0

func (trans *GraphTransform) GetInputNumber(_ Port) int

func (*GraphTransform) GetInputs added in v1.5.0

func (trans *GraphTransform) GetInputs() Ports

func (*GraphTransform) GetOutputNumber added in v1.5.0

func (trans *GraphTransform) GetOutputNumber(_ Port) int

func (*GraphTransform) GetOutputs added in v1.5.0

func (trans *GraphTransform) GetOutputs() Ports

func (*GraphTransform) Name added in v1.5.0

func (trans *GraphTransform) Name() string

func (*GraphTransform) ParseGraphOtherCondition added in v1.5.0

func (trans *GraphTransform) ParseGraphOtherCondition(cond string) (influxql.Expr, string, string, error)

func (*GraphTransform) Work added in v1.5.0

func (trans *GraphTransform) Work(ctx context.Context) error

type GraphTransformCreator added in v1.5.0

type GraphTransformCreator struct {
}

func (*GraphTransformCreator) Create added in v1.5.0

type GroupByTransform

type GroupByTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewGroupByTransform

func NewGroupByTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions) *GroupByTransform

func (*GroupByTransform) Close

func (trans *GroupByTransform) Close()

func (*GroupByTransform) Explain

func (trans *GroupByTransform) Explain() []ValuePair

func (*GroupByTransform) GetInputNumber

func (trans *GroupByTransform) GetInputNumber(_ Port) int

func (*GroupByTransform) GetInputs

func (trans *GroupByTransform) GetInputs() Ports

func (*GroupByTransform) GetOutputNumber

func (trans *GroupByTransform) GetOutputNumber(_ Port) int

func (*GroupByTransform) GetOutputs

func (trans *GroupByTransform) GetOutputs() Ports

func (*GroupByTransform) Name

func (trans *GroupByTransform) Name() string

func (*GroupByTransform) Release

func (trans *GroupByTransform) Release() error

func (*GroupByTransform) Work

func (trans *GroupByTransform) Work(ctx context.Context) error

type GroupByTransformCreator

type GroupByTransformCreator struct {
}

func (*GroupByTransformCreator) Create

type GroupKeysMPool added in v1.1.0

type GroupKeysMPool struct {
	// contains filtered or unexported fields
}

func NewGroupKeysPool added in v1.1.0

func NewGroupKeysPool(size int) *GroupKeysMPool

func (*GroupKeysMPool) AllocGroupKeys added in v1.1.0

func (gkp *GroupKeysMPool) AllocGroupKeys(size int) [][]byte

func (*GroupKeysMPool) AllocGroupTags added in v1.1.0

func (gkp *GroupKeysMPool) AllocGroupTags(size int) []*ChunkTags

func (*GroupKeysMPool) AllocValues added in v1.1.0

func (gkp *GroupKeysMPool) AllocValues(size int) []uint64

func (*GroupKeysMPool) AllocZValues added in v1.1.0

func (gkp *GroupKeysMPool) AllocZValues(size int) []int64

func (*GroupKeysMPool) FreeGroupKeys added in v1.1.0

func (gkp *GroupKeysMPool) FreeGroupKeys(groupKeys [][]byte)

func (*GroupKeysMPool) FreeGroupTags added in v1.1.0

func (gkp *GroupKeysMPool) FreeGroupTags(groupTags []*ChunkTags)

func (*GroupKeysMPool) FreeValues added in v1.1.0

func (gkp *GroupKeysMPool) FreeValues(values []uint64)

func (*GroupKeysMPool) FreeZValue added in v1.1.0

func (gkp *GroupKeysMPool) FreeZValue(spillState []int64)

type GroupLocs added in v1.3.0

type GroupLocs struct {
	Locs []*Loc
	Loc  int
	// contains filtered or unexported fields
}

type Handler

type Handler func(interface{}) error

type HashAggTransform added in v1.1.0

type HashAggTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func (*HashAggTransform) Close added in v1.1.0

func (trans *HashAggTransform) Close()

func (*HashAggTransform) Explain added in v1.1.0

func (trans *HashAggTransform) Explain() []ValuePair

func (*HashAggTransform) GetFuncs added in v1.1.0

func (trans *HashAggTransform) GetFuncs() []aggFunc

func (*HashAggTransform) GetInputNumber added in v1.1.0

func (trans *HashAggTransform) GetInputNumber(_ Port) int

func (*HashAggTransform) GetInputs added in v1.1.0

func (trans *HashAggTransform) GetInputs() Ports

func (*HashAggTransform) GetOutputNumber added in v1.1.0

func (trans *HashAggTransform) GetOutputNumber(_ Port) int

func (*HashAggTransform) GetOutputs added in v1.1.0

func (trans *HashAggTransform) GetOutputs() Ports

func (*HashAggTransform) InitFuncs added in v1.1.0

func (trans *HashAggTransform) InitFuncs(inRowDataType, outRowDataType hybridqp.RowDataType,
	exprOpt []hybridqp.ExprOptions) error

func (*HashAggTransform) Name added in v1.1.0

func (trans *HashAggTransform) Name() string

func (*HashAggTransform) Work added in v1.1.0

func (trans *HashAggTransform) Work(ctx context.Context) error

type HashAggTransformCreator added in v1.1.0

type HashAggTransformCreator struct {
}

func (*HashAggTransformCreator) Create added in v1.1.0

type HashAggType added in v1.1.0

type HashAggType uint32
const (
	Fill HashAggType = iota
	Normal
	SubQuery
	TopN
)

type HashJoinTransform added in v1.5.0

type HashJoinTransform struct {
	BaseProcessor

	BuildSide int
	ProbeSide int
	// contains filtered or unexported fields
}

func NewHashJoinTransform added in v1.5.0

func NewHashJoinTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataType hybridqp.RowDataType,
	joinCase *influxql.Join, schema hybridqp.Catalog) (*HashJoinTransform, error)

func (*HashJoinTransform) Close added in v1.5.0

func (trans *HashJoinTransform) Close()

func (*HashJoinTransform) Explain added in v1.5.0

func (trans *HashJoinTransform) Explain() []ValuePair

func (*HashJoinTransform) GetInputNumber added in v1.5.0

func (trans *HashJoinTransform) GetInputNumber(_ Port) int

func (*HashJoinTransform) GetInputs added in v1.5.0

func (trans *HashJoinTransform) GetInputs() Ports

func (*HashJoinTransform) GetOutputNumber added in v1.5.0

func (trans *HashJoinTransform) GetOutputNumber(_ Port) int

func (*HashJoinTransform) GetOutputs added in v1.5.0

func (trans *HashJoinTransform) GetOutputs() Ports

func (*HashJoinTransform) Name added in v1.5.0

func (trans *HashJoinTransform) Name() string

func (*HashJoinTransform) SendChunk added in v1.5.0

func (trans *HashJoinTransform) SendChunk()

func (*HashJoinTransform) Work added in v1.5.0

func (trans *HashJoinTransform) Work(ctx context.Context) error

type HashMergeBooleanColumn added in v1.1.1

type HashMergeBooleanColumn struct {
	// contains filtered or unexported fields
}

func (*HashMergeBooleanColumn) AppendValues added in v1.1.1

func (m *HashMergeBooleanColumn) AppendValues(col Column, start int, end int)

func (*HashMergeBooleanColumn) SetOutPut added in v1.1.1

func (m *HashMergeBooleanColumn) SetOutPut(col Column)

type HashMergeColumn added in v1.1.1

type HashMergeColumn interface {
	AppendValues(col Column, start int, end int)
	SetOutPut(col Column)
}

func NewHashMergeBooleanColumn added in v1.1.1

func NewHashMergeBooleanColumn() HashMergeColumn

func NewHashMergeFloatColumn added in v1.1.1

func NewHashMergeFloatColumn() HashMergeColumn

func NewHashMergeIntegerColumn added in v1.1.1

func NewHashMergeIntegerColumn() HashMergeColumn

func NewHashMergeStringColumn added in v1.1.1

func NewHashMergeStringColumn() HashMergeColumn

type HashMergeFloatColumn added in v1.1.1

type HashMergeFloatColumn struct {
	// contains filtered or unexported fields
}

func (*HashMergeFloatColumn) AppendValues added in v1.1.1

func (m *HashMergeFloatColumn) AppendValues(col Column, start int, end int)

func (*HashMergeFloatColumn) SetOutPut added in v1.1.1

func (m *HashMergeFloatColumn) SetOutPut(col Column)

type HashMergeIntegerColumn added in v1.1.1

type HashMergeIntegerColumn struct {
	// contains filtered or unexported fields
}

func (*HashMergeIntegerColumn) AppendValues added in v1.1.1

func (m *HashMergeIntegerColumn) AppendValues(col Column, start int, end int)

func (*HashMergeIntegerColumn) SetOutPut added in v1.1.1

func (m *HashMergeIntegerColumn) SetOutPut(col Column)

type HashMergeMsg added in v1.1.1

type HashMergeMsg struct {
	// contains filtered or unexported fields
}

type HashMergeResult added in v1.1.1

type HashMergeResult struct {
	// contains filtered or unexported fields
}

func (*HashMergeResult) AppendResult added in v1.1.1

func (mr *HashMergeResult) AppendResult(chunk Chunk, start int, end int)

type HashMergeStringColumn added in v1.1.1

type HashMergeStringColumn struct {
	// contains filtered or unexported fields
}

func (*HashMergeStringColumn) AppendValues added in v1.1.1

func (m *HashMergeStringColumn) AppendValues(col Column, start int, end int)

func (*HashMergeStringColumn) SetOutPut added in v1.1.1

func (m *HashMergeStringColumn) SetOutPut(col Column)

type HashMergeTransform added in v1.1.0

type HashMergeTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewHashMergeHashTypeTransform added in v1.1.0

func NewHashMergeHashTypeTransform(inRowDataType, outRowDataType []hybridqp.RowDataType, s *QuerySchema) (*HashMergeTransform, error)

func NewHashMergeStreamTypeTransform added in v1.1.0

func NewHashMergeStreamTypeTransform(inRowDataType, outRowDataType []hybridqp.RowDataType, s *QuerySchema) (*HashMergeTransform, error)

func NewHashMergeTransform added in v1.1.0

func NewHashMergeTransform(inRowDataType, outRowDataType []hybridqp.RowDataType, s *QuerySchema) (*HashMergeTransform, error)

func (*HashMergeTransform) Close added in v1.1.0

func (trans *HashMergeTransform) Close()

func (*HashMergeTransform) Explain added in v1.1.0

func (trans *HashMergeTransform) Explain() []ValuePair

func (*HashMergeTransform) GetInputNumber added in v1.1.0

func (trans *HashMergeTransform) GetInputNumber(_ Port) int

func (*HashMergeTransform) GetInputs added in v1.1.0

func (trans *HashMergeTransform) GetInputs() Ports

func (*HashMergeTransform) GetOutputNumber added in v1.1.0

func (trans *HashMergeTransform) GetOutputNumber(_ Port) int

func (*HashMergeTransform) GetOutputs added in v1.1.0

func (trans *HashMergeTransform) GetOutputs() Ports

func (*HashMergeTransform) Name added in v1.1.0

func (trans *HashMergeTransform) Name() string

func (*HashMergeTransform) Work added in v1.1.0

func (trans *HashMergeTransform) Work(ctx context.Context) error

type HashMergeTransformCreator added in v1.1.0

type HashMergeTransformCreator struct {
}

func (*HashMergeTransformCreator) Create added in v1.1.0

type HashMergeType added in v1.1.0

type HashMergeType uint32
const (
	Hash HashMergeType = iota
	Stream
)

type HeapItem added in v1.3.0

type HeapItem[T util.NumberOnly] struct {
	// contains filtered or unexported fields
}

func NewHeapItem added in v1.3.0

func NewHeapItem[T util.NumberOnly](n int, cmpByValue, cmpByTime func(a, b *PointItem[T]) bool, sortFunc func(item *HeapItem[T])) *HeapItem[T]

func (*HeapItem[T]) Len added in v1.3.0

func (f *HeapItem[T]) Len() int

func (*HeapItem[T]) Less added in v1.3.0

func (f *HeapItem[T]) Less(i, j int) bool

func (*HeapItem[T]) Pop added in v1.3.0

func (f *HeapItem[T]) Pop() interface{}

func (*HeapItem[T]) Push added in v1.3.0

func (f *HeapItem[T]) Push(x interface{})

func (*HeapItem[T]) Reset added in v1.3.0

func (f *HeapItem[T]) Reset()

func (*HeapItem[T]) Swap added in v1.3.0

func (f *HeapItem[T]) Swap(i, j int)

type HeapItems

type HeapItems struct {
	Items []*Item
	// contains filtered or unexported fields
}

func (*HeapItems) GetBreakPoint

func (h *HeapItems) GetBreakPoint() BaseBreakPoint

GetBreakPoint used to get the break point of the records

func (*HeapItems) GetOption added in v1.0.0

func (h *HeapItems) GetOption() *query.ProcessorOptions

func (*HeapItems) Len

func (h *HeapItems) Len() int

func (*HeapItems) Less

func (h *HeapItems) Less(i, j int) bool

func (*HeapItems) Pop

func (h *HeapItems) Pop() interface{}

func (*HeapItems) Push

func (h *HeapItems) Push(x interface{})

func (*HeapItems) Swap

func (h *HeapItems) Swap(i, j int)

type HeuDag

type HeuDag struct {
	// contains filtered or unexported fields
}

func NewHeuDag

func NewHeuDag() *HeuDag

func (*HeuDag) AddEdge

func (dag *HeuDag) AddEdge(from *HeuVertex, to *HeuVertex) bool

func (*HeuDag) AddVerteix

func (dag *HeuDag) AddVerteix(vertex *HeuVertex) bool

func (*HeuDag) AllParents

func (dag *HeuDag) AllParents(vertex *HeuVertex) []*HeuVertex

func (*HeuDag) Contains

func (dag *HeuDag) Contains(vertex *HeuVertex) bool

func (*HeuDag) DigestSize

func (dag *HeuDag) DigestSize() int

func (*HeuDag) EdgeSize

func (dag *HeuDag) EdgeSize() int

func (*HeuDag) GetGraphIterator

func (dag *HeuDag) GetGraphIterator(vertex *HeuVertex, matchOrder HeuMatchOrder) *GraphIterator

func (*HeuDag) GetVertexByDigest

func (dag *HeuDag) GetVertexByDigest(digest string) (*HeuVertex, bool)

func (*HeuDag) RemoveEdge

func (dag *HeuDag) RemoveEdge(from *HeuVertex, to *HeuVertex)

func (*HeuDag) RemoveVertex

func (dag *HeuDag) RemoveVertex(vertex *HeuVertex)

func (*HeuDag) UpdateVerteix

func (dag *HeuDag) UpdateVerteix(vertex *HeuVertex, node hybridqp.QueryNode)

func (*HeuDag) VertexSize

func (dag *HeuDag) VertexSize() int

func (*HeuDag) WalkHeuDag

func (dag *HeuDag) WalkHeuDag(visitor HeuDagVisitor, vertex *HeuVertex)

type HeuDagVisitor

type HeuDagVisitor interface {
	Visit(vertex *HeuVertex) HeuDagVisitor
}

type HeuEdge

type HeuEdge struct {
	// contains filtered or unexported fields
}

func NewHeuEdge

func NewHeuEdge(from *HeuVertex, to *HeuVertex) *HeuEdge

type HeuInstruction

type HeuInstruction interface {
	Initialize(bool)
	Execute(HeuPlanner)
}

type HeuMatchOrder

type HeuMatchOrder uint8
const (
	ARBITRARY HeuMatchOrder = iota
	DEPTH_FIRST
)

type HeuPlanner

type HeuPlanner interface {
	hybridqp.Planner
	ExecuteInstruction(HeuInstruction)
	Vertex(node hybridqp.QueryNode) (*HeuVertex, bool)
}

type HeuPlannerImpl

type HeuPlannerImpl struct {
	// contains filtered or unexported fields
}

func NewHeuPlannerImpl

func NewHeuPlannerImpl(program *HeuProgram) *HeuPlannerImpl

func (*HeuPlannerImpl) AddRule

func (p *HeuPlannerImpl) AddRule(rule OptRule) bool

func (*HeuPlannerImpl) ExecuteInstruction

func (p *HeuPlannerImpl) ExecuteInstruction(instruction HeuInstruction)

func (*HeuPlannerImpl) FindBestExp

func (p *HeuPlannerImpl) FindBestExp() hybridqp.QueryNode

func (*HeuPlannerImpl) SetRoot

func (p *HeuPlannerImpl) SetRoot(root hybridqp.QueryNode)

func (*HeuPlannerImpl) Transformations

func (p *HeuPlannerImpl) Transformations() int

func (*HeuPlannerImpl) Vertex

func (p *HeuPlannerImpl) Vertex(node hybridqp.QueryNode) (*HeuVertex, bool)

func (*HeuPlannerImpl) Visit

type HeuProgram

type HeuProgram struct {
	// contains filtered or unexported fields
}

func NewHeuProgram

func NewHeuProgram(instructions []HeuInstruction) *HeuProgram

func (*HeuProgram) Initialize

func (p *HeuProgram) Initialize(clearCache bool)

type HeuVertex

type HeuVertex struct {
	// contains filtered or unexported fields
}

func NewHeuVertex

func NewHeuVertex(node hybridqp.QueryNode) *HeuVertex

func (*HeuVertex) ApplyTrait

func (v *HeuVertex) ApplyTrait(trait hybridqp.Trait)

func (*HeuVertex) Children

func (v *HeuVertex) Children() []hybridqp.QueryNode

func (*HeuVertex) Clone

func (v *HeuVertex) Clone() hybridqp.QueryNode

func (*HeuVertex) DeriveOperations

func (v *HeuVertex) DeriveOperations()

func (*HeuVertex) Digest

func (v *HeuVertex) Digest() string

func (*HeuVertex) Dummy

func (v *HeuVertex) Dummy() bool

func (*HeuVertex) Equals

func (v *HeuVertex) Equals(rhs *HeuVertex) bool

func (*HeuVertex) GetParentVertex

func (v *HeuVertex) GetParentVertex(vertex *HeuVertex) *HeuVertex

func (*HeuVertex) ID

func (v *HeuVertex) ID() uint64

func (*HeuVertex) New added in v1.1.0

func (v *HeuVertex) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*HeuVertex) Node

func (v *HeuVertex) Node() hybridqp.QueryNode

func (*HeuVertex) ReplaceChild

func (v *HeuVertex) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*HeuVertex) ReplaceChildren

func (v *HeuVertex) ReplaceChildren(children []hybridqp.QueryNode)

func (*HeuVertex) ReplaceNode

func (v *HeuVertex) ReplaceNode(node hybridqp.QueryNode)

func (*HeuVertex) RowDataType

func (v *HeuVertex) RowDataType() hybridqp.RowDataType

func (*HeuVertex) RowExprOptions

func (v *HeuVertex) RowExprOptions() []hybridqp.ExprOptions

func (*HeuVertex) Schema

func (v *HeuVertex) Schema() hybridqp.Catalog

func (*HeuVertex) SetInputs

func (v *HeuVertex) SetInputs(_ []hybridqp.QueryNode)

func (*HeuVertex) SetSchema

func (v *HeuVertex) SetSchema(schema hybridqp.Catalog)

func (*HeuVertex) String

func (v *HeuVertex) String() string

func (*HeuVertex) SubTreeEqual

func (v *HeuVertex) SubTreeEqual(vertex *HeuVertex) bool

func (*HeuVertex) Trait

func (v *HeuVertex) Trait() hybridqp.Trait

func (*HeuVertex) Type

func (v *HeuVertex) Type() string

type HeuVertexInfo

type HeuVertexInfo struct {
	// contains filtered or unexported fields
}

func NewHeuVertexInfo

func NewHeuVertexInfo() *HeuVertexInfo

func (*HeuVertexInfo) AddBackwardEdge

func (info *HeuVertexInfo) AddBackwardEdge(edge *HeuEdge)

func (*HeuVertexInfo) AddDirectEdge

func (info *HeuVertexInfo) AddDirectEdge(edge *HeuEdge)

type HeuVertexWriter

type HeuVertexWriter interface {
	Explain(*HeuVertex)
	Item(string, interface{})
	String() string
}

type HeuVertexs

type HeuVertexs []*HeuVertex

func (HeuVertexs) IndexOf

func (hvs HeuVertexs) IndexOf(vertex *HeuVertex) int

type HistogramQuantileOp added in v1.3.0

type HistogramQuantileOp struct{}

func (*HistogramQuantileOp) CreateRoutine added in v1.3.0

func (c *HistogramQuantileOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type HoltWintersTransform added in v1.0.0

type HoltWintersTransform struct {
	BaseProcessor

	Input  ChunkPort
	Output ChunkPort
	// contains filtered or unexported fields
}

func NewHoltWintersTransform added in v1.0.0

func NewHoltWintersTransform(
	inRowDataType, outRowDataType hybridqp.RowDataType, opt *query.ProcessorOptions, schema hybridqp.Catalog,
) (*HoltWintersTransform, error)

func (*HoltWintersTransform) Close added in v1.0.0

func (trans *HoltWintersTransform) Close()

func (*HoltWintersTransform) Explain added in v1.0.0

func (trans *HoltWintersTransform) Explain() []ValuePair

func (*HoltWintersTransform) GetInputNumber added in v1.0.0

func (trans *HoltWintersTransform) GetInputNumber(port Port) int

func (*HoltWintersTransform) GetInputs added in v1.0.0

func (trans *HoltWintersTransform) GetInputs() Ports

func (*HoltWintersTransform) GetOutputNumber added in v1.0.0

func (trans *HoltWintersTransform) GetOutputNumber(port Port) int

func (*HoltWintersTransform) GetOutputs added in v1.0.0

func (trans *HoltWintersTransform) GetOutputs() Ports

func (*HoltWintersTransform) Name added in v1.0.0

func (trans *HoltWintersTransform) Name() string

func (*HoltWintersTransform) Work added in v1.0.0

func (trans *HoltWintersTransform) Work(ctx context.Context) error

type HoltWintersTransformCreator added in v1.0.0

type HoltWintersTransformCreator struct {
}

func (*HoltWintersTransformCreator) Create added in v1.0.0

type HttpChunkSender

type HttpChunkSender struct {
	RowChunk RowChunk
	// contains filtered or unexported fields
}

func NewHttpChunkSender

func NewHttpChunkSender(opt *query.ProcessorOptions) *HttpChunkSender

func (*HttpChunkSender) GenRows added in v1.3.0

func (w *HttpChunkSender) GenRows(chunk Chunk)

func (*HttpChunkSender) GetRows

func (w *HttpChunkSender) GetRows(chunk Chunk) models.Rows

GetRows transfer Chunk to models.Rows

func (*HttpChunkSender) Release added in v1.1.0

func (w *HttpChunkSender) Release()

func (*HttpChunkSender) SetAbortProcessor added in v1.3.0

func (w *HttpChunkSender) SetAbortProcessor(trans AbortProcessor)

func (*HttpChunkSender) Write

func (w *HttpChunkSender) Write(chunk Chunk, lastChunk bool) bool

type HttpSenderHintTransform

type HttpSenderHintTransform struct {
	BaseProcessor

	Writer ChunkSender
	// contains filtered or unexported fields
}

func NewHttpSenderHintTransform

func NewHttpSenderHintTransform(inRowDataType hybridqp.RowDataType, schema *QuerySchema) *HttpSenderHintTransform

func (*HttpSenderHintTransform) AbortSinkTransform added in v1.3.0

func (trans *HttpSenderHintTransform) AbortSinkTransform()

func (*HttpSenderHintTransform) Close

func (trans *HttpSenderHintTransform) Close()

func (*HttpSenderHintTransform) Explain

func (trans *HttpSenderHintTransform) Explain() []ValuePair

func (*HttpSenderHintTransform) GetInputNumber

func (trans *HttpSenderHintTransform) GetInputNumber(_ Port) int

func (*HttpSenderHintTransform) GetInputs

func (trans *HttpSenderHintTransform) GetInputs() Ports

func (*HttpSenderHintTransform) GetOutputNumber

func (trans *HttpSenderHintTransform) GetOutputNumber(_ Port) int

func (*HttpSenderHintTransform) GetOutputs

func (trans *HttpSenderHintTransform) GetOutputs() Ports

func (*HttpSenderHintTransform) Name

func (trans *HttpSenderHintTransform) Name() string

func (*HttpSenderHintTransform) SetDag added in v1.3.0

func (trans *HttpSenderHintTransform) SetDag(dag *TransformDag)

func (*HttpSenderHintTransform) SetVertex added in v1.3.0

func (trans *HttpSenderHintTransform) SetVertex(vertex *TransformVertex)

func (*HttpSenderHintTransform) Visit added in v1.3.0

func (*HttpSenderHintTransform) Work

func (trans *HttpSenderHintTransform) Work(ctx context.Context) error

type HttpSenderHintTransformCreator

type HttpSenderHintTransformCreator struct {
}

func (*HttpSenderHintTransformCreator) Create

type HttpSenderTransform

type HttpSenderTransform struct {
	BaseProcessor

	Sender *http.ResponseWriter
	Writer ChunkSender
	// contains filtered or unexported fields
}

func NewHttpSenderTransform

func NewHttpSenderTransform(inRowDataType hybridqp.RowDataType, schema *QuerySchema) *HttpSenderTransform

func (*HttpSenderTransform) AbortSinkTransform added in v1.3.0

func (trans *HttpSenderTransform) AbortSinkTransform()

func (*HttpSenderTransform) Close

func (trans *HttpSenderTransform) Close()

func (*HttpSenderTransform) Explain

func (trans *HttpSenderTransform) Explain() []ValuePair

func (*HttpSenderTransform) GetInputNumber

func (trans *HttpSenderTransform) GetInputNumber(_ Port) int

func (*HttpSenderTransform) GetInputs

func (trans *HttpSenderTransform) GetInputs() Ports

func (*HttpSenderTransform) GetOutputNumber

func (trans *HttpSenderTransform) GetOutputNumber(_ Port) int

func (*HttpSenderTransform) GetOutputs

func (trans *HttpSenderTransform) GetOutputs() Ports

func (*HttpSenderTransform) Name

func (trans *HttpSenderTransform) Name() string

func (*HttpSenderTransform) Release added in v1.1.0

func (trans *HttpSenderTransform) Release() error

func (*HttpSenderTransform) SetDag added in v1.3.0

func (trans *HttpSenderTransform) SetDag(dag *TransformDag)

func (*HttpSenderTransform) SetVertex added in v1.3.0

func (trans *HttpSenderTransform) SetVertex(vertex *TransformVertex)

func (*HttpSenderTransform) Visit added in v1.3.0

func (*HttpSenderTransform) Work

func (trans *HttpSenderTransform) Work(ctx context.Context) error

type HttpSenderTransformCreator

type HttpSenderTransformCreator struct {
}

func (*HttpSenderTransformCreator) Create

type IGraph added in v1.5.0

type IGraph interface {
	GetNodeInfo(id string) *GraphNode
	GetEdgeInfo(id string) *GraphEdge
	BatchInsertNodes(graphData GraphData) bool
	BatchInsertEdges(graphData GraphData) (bool, error)
	CreateGraph(jsonGraphData string) (bool, error)
	MultiHopFilter(startNodeId string, hopNum int, nodeCondition influxql.Expr, edgeCondition influxql.Expr) (*Graph, error)
	GraphToRows() models.Rows
}

type InTransform added in v1.5.0

type InTransform struct {
	BaseProcessor

	OuterVarRef *influxql.VarRef

	ExecutorBuilder OuterExecutorBuilder

	BufColumnMap map[interface{}]struct{}
	// contains filtered or unexported fields
}

func NewInTransform added in v1.5.0

func NewInTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, schema hybridqp.Catalog,
	outerStmt *influxql.SelectStatement, outerSchema hybridqp.Catalog, outerQc query.LogicalPlanCreator, outerField influxql.Expr, tagstmt *influxql.ShowTagValuesStatement) (*InTransform, error)

func (*InTransform) AddChunkToBufColumn added in v1.5.0

func (trans *InTransform) AddChunkToBufColumn(c Chunk)

don't add nilMap to buf, skip same val to add

func (*InTransform) AddGraphChunkToBufColumn added in v1.5.0

func (trans *InTransform) AddGraphChunkToBufColumn(c Chunk)

todo: support graph.properties

func (*InTransform) AddTagValsToBufColumn added in v1.5.0

func (trans *InTransform) AddTagValsToBufColumn(s influxql.TablesTagSets)

func (*InTransform) Close added in v1.5.0

func (trans *InTransform) Close()

func (*InTransform) Explain added in v1.5.0

func (trans *InTransform) Explain() []ValuePair

func (*InTransform) GetInputNumber added in v1.5.0

func (trans *InTransform) GetInputNumber(_ Port) int

func (*InTransform) GetInputs added in v1.5.0

func (trans *InTransform) GetInputs() Ports

func (*InTransform) GetOutputNumber added in v1.5.0

func (trans *InTransform) GetOutputNumber(_ Port) int

func (*InTransform) GetOutputs added in v1.5.0

func (trans *InTransform) GetOutputs() Ports

func (*InTransform) Name added in v1.5.0

func (trans *InTransform) Name() string

func (*InTransform) RewriteOuterStmtCondition added in v1.5.0

func (trans *InTransform) RewriteOuterStmtCondition() error

func (*InTransform) RunOuterPlan added in v1.5.0

func (trans *InTransform) RunOuterPlan(ctx context.Context) error

func (*InTransform) SendChunk added in v1.5.0

func (trans *InTransform) SendChunk(c Chunk)

func (*InTransform) ShowTagValRun added in v1.5.0

func (trans *InTransform) ShowTagValRun(stmt *influxql.ShowTagValuesStatement) error

func (*InTransform) WalkDBNodes added in v1.5.0

func (trans *InTransform) WalkDBNodes(database string, fn func(nodeID uint64, pts []uint32) error) error

func (*InTransform) Work added in v1.5.0

func (trans *InTransform) Work(ctx context.Context) error

type InTransformCreator added in v1.5.0

type InTransformCreator struct {
}

func (*InTransformCreator) Create added in v1.5.0

type InTransformRunningState added in v1.5.0

type InTransformRunningState int

type IncAggEntry added in v1.2.0

type IncAggEntry struct {
	// contains filtered or unexported fields
}

func NewIncAggEntry added in v1.2.0

func NewIncAggEntry(queryID string) *IncAggEntry

func (*IncAggEntry) GetKey added in v1.2.0

func (e *IncAggEntry) GetKey() string

func (*IncAggEntry) GetTime added in v1.2.0

func (e *IncAggEntry) GetTime() time.Time

func (*IncAggEntry) GetValue added in v1.2.0

func (e *IncAggEntry) GetValue() interface{}

func (*IncAggEntry) SetTime added in v1.2.0

func (e *IncAggEntry) SetTime(time time.Time)

func (*IncAggEntry) SetValue added in v1.2.0

func (e *IncAggEntry) SetValue(value interface{})

func (*IncAggEntry) Size added in v1.2.0

func (e *IncAggEntry) Size() int64

type IncAggItem added in v1.2.0

type IncAggItem struct {
	// contains filtered or unexported fields
}

func NewIncAggIterm added in v1.2.0

func NewIncAggIterm(iterID int32, value Chunk) *IncAggItem

type IncAggRule added in v1.2.0

type IncAggRule struct {
	OptRuleBase
}

func NewIncAggRule added in v1.2.0

func NewIncAggRule(description string) *IncAggRule

func (*IncAggRule) Category added in v1.5.0

func (r *IncAggRule) Category() OptRuleCategory

func (*IncAggRule) Equals added in v1.2.0

func (r *IncAggRule) Equals(rhs OptRule) bool

func (*IncAggRule) OnMatch added in v1.2.0

func (r *IncAggRule) OnMatch(call *OptRuleCall)

func (*IncAggRule) ToString added in v1.2.0

func (r *IncAggRule) ToString() string

type IncAggTransform added in v1.2.0

type IncAggTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewIncAggTransform added in v1.2.0

func NewIncAggTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions) (*IncAggTransform, error)

func (*IncAggTransform) Close added in v1.2.0

func (trans *IncAggTransform) Close()

func (*IncAggTransform) Explain added in v1.2.0

func (trans *IncAggTransform) Explain() []ValuePair

func (*IncAggTransform) GetIndex added in v1.2.0

func (trans *IncAggTransform) GetIndex(t int64) int64

func (*IncAggTransform) GetInputNumber added in v1.2.0

func (trans *IncAggTransform) GetInputNumber(port Port) int

func (*IncAggTransform) GetInputs added in v1.2.0

func (trans *IncAggTransform) GetInputs() Ports

func (*IncAggTransform) GetOutputNumber added in v1.2.0

func (trans *IncAggTransform) GetOutputNumber(port Port) int

func (*IncAggTransform) GetOutputs added in v1.2.0

func (trans *IncAggTransform) GetOutputs() Ports

func (*IncAggTransform) Name added in v1.2.0

func (trans *IncAggTransform) Name() string

func (*IncAggTransform) Work added in v1.2.0

func (trans *IncAggTransform) Work(ctx context.Context) error

type IncAggTransformCreator added in v1.2.0

type IncAggTransformCreator struct {
}

func (*IncAggTransformCreator) Create added in v1.2.0

type IncHashAggEntry added in v1.2.0

type IncHashAggEntry struct {
	// contains filtered or unexported fields
}

func NewIncHashAggEntry added in v1.2.0

func NewIncHashAggEntry(queryID string) *IncHashAggEntry

func (*IncHashAggEntry) GetKey added in v1.2.0

func (e *IncHashAggEntry) GetKey() string

func (*IncHashAggEntry) GetTime added in v1.2.0

func (e *IncHashAggEntry) GetTime() time.Time

func (*IncHashAggEntry) GetValue added in v1.2.0

func (e *IncHashAggEntry) GetValue() interface{}

func (*IncHashAggEntry) SetTime added in v1.2.0

func (e *IncHashAggEntry) SetTime(time time.Time)

func (*IncHashAggEntry) SetValue added in v1.2.0

func (e *IncHashAggEntry) SetValue(value interface{})

func (*IncHashAggEntry) Size added in v1.2.0

func (e *IncHashAggEntry) Size() int64

type IncHashAggItem added in v1.2.0

type IncHashAggItem struct {
	// contains filtered or unexported fields
}

func GetIncHashAggItem added in v1.2.0

func GetIncHashAggItem(queryID string, iterID int32) (*IncHashAggItem, bool)

func NewIncHashAggItem added in v1.2.0

func NewIncHashAggItem(iterID int32, chunks []Chunk) *IncHashAggItem

func (*IncHashAggItem) AppendGroupIdx added in v1.2.0

func (item *IncHashAggItem) AppendGroupIdx(groupIdx uint32)

func (*IncHashAggItem) GetGroupIds added in v1.2.0

func (item *IncHashAggItem) GetGroupIds(tag []byte) int

func (*IncHashAggItem) Size added in v1.2.0

func (item *IncHashAggItem) Size() int64

func (*IncHashAggItem) UpdateChunkAndIterID added in v1.2.0

func (item *IncHashAggItem) UpdateChunkAndIterID(iterID int32, chunks []Chunk)

type IncHashAggRule added in v1.2.0

type IncHashAggRule struct {
	OptRuleBase
}

func NewIncHashAggRule added in v1.2.0

func NewIncHashAggRule(description string) *IncHashAggRule

func (*IncHashAggRule) Category added in v1.5.0

func (r *IncHashAggRule) Category() OptRuleCategory

func (*IncHashAggRule) Equals added in v1.2.0

func (r *IncHashAggRule) Equals(rhs OptRule) bool

func (*IncHashAggRule) OnMatch added in v1.2.0

func (r *IncHashAggRule) OnMatch(call *OptRuleCall)

func (*IncHashAggRule) ToString added in v1.2.0

func (r *IncHashAggRule) ToString() string

type IncHashAggTransform added in v1.2.0

type IncHashAggTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewIncHashAggTransform added in v1.2.0

func NewIncHashAggTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType,
	ops []hybridqp.ExprOptions, opt *query.ProcessorOptions) (*IncHashAggTransform, error)

func (*IncHashAggTransform) Close added in v1.2.0

func (trans *IncHashAggTransform) Close()

func (*IncHashAggTransform) Explain added in v1.2.0

func (trans *IncHashAggTransform) Explain() []ValuePair

func (*IncHashAggTransform) GetIndex added in v1.2.0

func (trans *IncHashAggTransform) GetIndex(t int64) int64

func (*IncHashAggTransform) GetInputNumber added in v1.2.0

func (trans *IncHashAggTransform) GetInputNumber(port Port) int

func (*IncHashAggTransform) GetInputs added in v1.2.0

func (trans *IncHashAggTransform) GetInputs() Ports

func (*IncHashAggTransform) GetOutputNumber added in v1.2.0

func (trans *IncHashAggTransform) GetOutputNumber(port Port) int

func (*IncHashAggTransform) GetOutputs added in v1.2.0

func (trans *IncHashAggTransform) GetOutputs() Ports

func (*IncHashAggTransform) Name added in v1.2.0

func (trans *IncHashAggTransform) Name() string

func (*IncHashAggTransform) Work added in v1.2.0

func (trans *IncHashAggTransform) Work(ctx context.Context) error

type IncHashAggTransformCreator added in v1.2.0

type IncHashAggTransformCreator struct {
}

func (*IncHashAggTransformCreator) Create added in v1.2.0

type IncQueryFinish added in v1.2.0

type IncQueryFinish struct {
	// contains filtered or unexported fields
}

func (*IncQueryFinish) Instance added in v1.2.0

func (e *IncQueryFinish) Instance() transport.Codec

func (*IncQueryFinish) Marshal added in v1.2.0

func (e *IncQueryFinish) Marshal(buf []byte) ([]byte, error)

func (*IncQueryFinish) Size added in v1.2.0

func (e *IncQueryFinish) Size() int

func (*IncQueryFinish) Unmarshal added in v1.2.0

func (e *IncQueryFinish) Unmarshal(buf []byte) error

type IndexFrags added in v1.2.0

type IndexFrags interface {
	BasePath() string
	FragCount() int64
	IndexCount() int
	Indexes() interface{}
	AppendIndexes(...interface{})
	FragRanges() []fragment.FragmentRanges
	AppendFragRanges(...fragment.FragmentRanges)
	AddFragCount(int64)
	FileMode() FileMode
	SetErr(error)
	GetErr() error
	Size() int
}

type IndexScanExtraInfo

type IndexScanExtraInfo struct {
	Dsc     bool // Used to determines whether to schedule shards in descending.
	ShardID uint64
	Req     *RemoteQuery
	Store   hybridqp.StoreEngine

	PtQuery *PtQuery
	// contains filtered or unexported fields
}

func (*IndexScanExtraInfo) Clone

func (*IndexScanExtraInfo) IsPtQuery added in v1.2.0

func (e *IndexScanExtraInfo) IsPtQuery() bool

func (*IndexScanExtraInfo) Len added in v1.2.0

func (e *IndexScanExtraInfo) Len() int

func (*IndexScanExtraInfo) Next added in v1.2.0

func (e *IndexScanExtraInfo) Next() *ShardInfo

type IndexScanTransform

type IndexScanTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewIndexScanTransform

func NewIndexScanTransform(outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, schema hybridqp.Catalog,
	input hybridqp.QueryNode, info *IndexScanExtraInfo, limiter chan struct{}, limit int, oneShardState bool) *IndexScanTransform

func (*IndexScanTransform) Abort added in v1.0.1

func (trans *IndexScanTransform) Abort()

func (*IndexScanTransform) BuildDownSamplePlan added in v1.0.0

func (trans *IndexScanTransform) BuildDownSamplePlan(s hybridqp.Catalog) (hybridqp.QueryNode, error)

func (*IndexScanTransform) BuildDownSampleSchema added in v1.0.0

func (trans *IndexScanTransform) BuildDownSampleSchema(schema hybridqp.Catalog) *QuerySchema

func (*IndexScanTransform) BuildPlan added in v1.0.0

func (trans *IndexScanTransform) BuildPlan(downSampleLevel int) (*QuerySchema, hybridqp.QueryNode, error)

func (*IndexScanTransform) CanDownSampleRewrite added in v1.0.0

func (trans *IndexScanTransform) CanDownSampleRewrite(downSampleLevel int) bool

func (*IndexScanTransform) Close

func (trans *IndexScanTransform) Close()

func (*IndexScanTransform) CursorsClose added in v1.1.0

func (trans *IndexScanTransform) CursorsClose(plan hybridqp.QueryNode)

func (*IndexScanTransform) Explain

func (trans *IndexScanTransform) Explain() []ValuePair

func (*IndexScanTransform) FreeResFromAllocator added in v1.0.1

func (trans *IndexScanTransform) FreeResFromAllocator()

func (*IndexScanTransform) GetInputNumber

func (trans *IndexScanTransform) GetInputNumber(_ Port) int

func (*IndexScanTransform) GetInputs

func (trans *IndexScanTransform) GetInputs() Ports

func (*IndexScanTransform) GetOutputNumber

func (trans *IndexScanTransform) GetOutputNumber(_ Port) int

func (*IndexScanTransform) GetOutputs

func (trans *IndexScanTransform) GetOutputs() Ports

func (*IndexScanTransform) GetResFromAllocator added in v1.0.1

func (trans *IndexScanTransform) GetResFromAllocator()

func (*IndexScanTransform) Interrupt added in v1.3.0

func (trans *IndexScanTransform) Interrupt()

func (*IndexScanTransform) IsSink added in v1.0.0

func (trans *IndexScanTransform) IsSink() bool

func (*IndexScanTransform) Name

func (trans *IndexScanTransform) Name() string

func (*IndexScanTransform) Release

func (trans *IndexScanTransform) Release() error

func (*IndexScanTransform) RewriteChunk added in v1.0.0

func (trans *IndexScanTransform) RewriteChunk(c Chunk) Chunk

func (*IndexScanTransform) Running added in v1.0.0

func (trans *IndexScanTransform) Running(ctx context.Context) bool

func (*IndexScanTransform) SetDownSampleLevel added in v1.0.0

func (trans *IndexScanTransform) SetDownSampleLevel(l int)

func (*IndexScanTransform) SetIndexScanErr added in v1.1.0

func (trans *IndexScanTransform) SetIndexScanErr(err bool)

func (*IndexScanTransform) SetPipelineExecutor added in v1.0.0

func (trans *IndexScanTransform) SetPipelineExecutor(exec *PipelineExecutor)

func (*IndexScanTransform) Unref

func (trans *IndexScanTransform) Unref()

Unref releases resources associated with the IndexScanTransform instance. It performs the following operations: 1. Releases file handles and resources for all fragments in 'frags'. 2. Releases file handles and resources for all files in 'indexInfo'. 3. Releases resources associated with 'tsIndexInfo'. This method ensures proper cleanup of allocated resources.

func (*IndexScanTransform) Work

func (trans *IndexScanTransform) Work(ctx context.Context) error

func (*IndexScanTransform) WorkHelper added in v1.0.0

func (trans *IndexScanTransform) WorkHelper(ctx context.Context) (error, bool)

type IndexScanTransformCreator

type IndexScanTransformCreator struct {
}

func (*IndexScanTransformCreator) Create

type Int64AppendIterator

type Int64AppendIterator struct {
	// contains filtered or unexported fields
}

func NewInt64AppendIterator

func NewInt64AppendIterator() *Int64AppendIterator

func (*Int64AppendIterator) Next

func (f *Int64AppendIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type Int64LimitIterator

type Int64LimitIterator struct {
	// contains filtered or unexported fields
}

func NewInt64LimitIterator

func NewInt64LimitIterator() *Int64LimitIterator

func (*Int64LimitIterator) Next

func (f *Int64LimitIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type Int64MergeIterator

type Int64MergeIterator struct {
	BaseMergeIterator
	// contains filtered or unexported fields
}

func NewInt64MergeIterator

func NewInt64MergeIterator() *Int64MergeIterator

func (*Int64MergeIterator) Next

func (f *Int64MergeIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type IntegerColFloatRateIterator

type IntegerColFloatRateIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerRateIterator added in v1.3.0

func NewIntegerRateIterator(fn RateMiddleReduceFunc[int64], fv RateFinalReduceFunc[int64],
	fu RateUpdateFunc[int64], fm RateMergeFunc[int64],
	isSingleCall bool, inOrdinal, outOrdinal int, rowDataType hybridqp.RowDataType,
	interval *hybridqp.Interval,
) *IntegerColFloatRateIterator

func (*IntegerColFloatRateIterator) Next

type IntegerColIntegerDistinctIterator

type IntegerColIntegerDistinctIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerColIntegerDistinctIterator

func NewIntegerColIntegerDistinctIterator(
	inOrdinal, outOrdinal int,
) *IntegerColIntegerDistinctIterator

func (*IntegerColIntegerDistinctIterator) Next

type IntegerColIntegerHeapIterator

type IntegerColIntegerHeapIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerColIntegerHeapIterator

func NewIntegerColIntegerHeapIterator(
	inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType, IntegerHeapItem *HeapItem[int64],
) *IntegerColIntegerHeapIterator

func (*IntegerColIntegerHeapIterator) Next

type IntegerColIntegerSampleIterator

type IntegerColIntegerSampleIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerColIntegerSampleIterator

func NewIntegerColIntegerSampleIterator(sampleNum int,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *IntegerColIntegerSampleIterator

func (*IntegerColIntegerSampleIterator) Next

type IntegerColIntegerSliceIterator

type IntegerColIntegerSliceIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerColIntegerSliceIterator

func NewIntegerColIntegerSliceIterator(fn SliceReduce[int64],
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *IntegerColIntegerSliceIterator

func (*IntegerColIntegerSliceIterator) Next

type IntegerCumulativeSumItem

type IntegerCumulativeSumItem struct {
	// contains filtered or unexported fields
}

func NewIntegerCumulativeSumItem

func NewIntegerCumulativeSumItem() *IntegerCumulativeSumItem

func (*IntegerCumulativeSumItem) AppendItem

func (f *IntegerCumulativeSumItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerCumulativeSumItem) AppendItemFastFunc

func (f *IntegerCumulativeSumItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerCumulativeSumItem) AppendItemSlowFunc

func (f *IntegerCumulativeSumItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerCumulativeSumItem) GetBaseTransData

func (f *IntegerCumulativeSumItem) GetBaseTransData() BaseTransData

func (*IntegerCumulativeSumItem) Len

func (f *IntegerCumulativeSumItem) Len() int

func (*IntegerCumulativeSumItem) PrevNil

func (f *IntegerCumulativeSumItem) PrevNil() bool

func (*IntegerCumulativeSumItem) Reset

func (f *IntegerCumulativeSumItem) Reset()

func (*IntegerCumulativeSumItem) ResetPrev

func (f *IntegerCumulativeSumItem) ResetPrev()

type IntegerDerivativeItem

type IntegerDerivativeItem struct {
	// contains filtered or unexported fields
}

func NewIntegerDerivativeItem

func NewIntegerDerivativeItem(isNonNegative, ascending bool, interval hybridqp.Interval) *IntegerDerivativeItem

func (*IntegerDerivativeItem) AppendItem

func (f *IntegerDerivativeItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDerivativeItem) AppendItemFastFunc

func (f *IntegerDerivativeItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDerivativeItem) AppendItemSlowFunc

func (f *IntegerDerivativeItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDerivativeItem) GetBaseTransData

func (f *IntegerDerivativeItem) GetBaseTransData() BaseTransData

func (*IntegerDerivativeItem) Len

func (f *IntegerDerivativeItem) Len() int

func (*IntegerDerivativeItem) PrevNil

func (f *IntegerDerivativeItem) PrevNil() bool

func (*IntegerDerivativeItem) Reset

func (f *IntegerDerivativeItem) Reset()

func (*IntegerDerivativeItem) ResetPrev

func (f *IntegerDerivativeItem) ResetPrev()

type IntegerDifferenceItem

type IntegerDifferenceItem struct {
	// contains filtered or unexported fields
}

func NewIntegerDifferenceItem

func NewIntegerDifferenceItem(isNonNegative bool, diff integerDifference) *IntegerDifferenceItem

func (*IntegerDifferenceItem) AppendItem

func (f *IntegerDifferenceItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDifferenceItem) AppendItemFastFunc

func (f *IntegerDifferenceItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDifferenceItem) AppendItemSlowFunc

func (f *IntegerDifferenceItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerDifferenceItem) GetBaseTransData

func (f *IntegerDifferenceItem) GetBaseTransData() BaseTransData

func (*IntegerDifferenceItem) Len

func (f *IntegerDifferenceItem) Len() int

func (*IntegerDifferenceItem) PrevNil

func (f *IntegerDifferenceItem) PrevNil() bool

func (*IntegerDifferenceItem) Reset

func (f *IntegerDifferenceItem) Reset()

func (*IntegerDifferenceItem) ResetPrev

func (f *IntegerDifferenceItem) ResetPrev()

type IntegerFieldValuer added in v1.0.0

type IntegerFieldValuer struct {
	// contains filtered or unexported fields
}

func (*IntegerFieldValuer) At added in v1.0.0

func (valuer *IntegerFieldValuer) At(col Column, pos int, field *influx.Field) bool

type IntegerIntegralIterator added in v1.3.0

type IntegerIntegralIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerIntegralIterator added in v1.3.0

func NewIntegerIntegralIterator(
	isSingleCall bool, inOrdinal, outOrdinal int, interval hybridqp.Interval,
	opt *query.ProcessorOptions,
) *IntegerIntegralIterator

func (*IntegerIntegralIterator) Next added in v1.3.0

type IntegerIntervalIterator

type IntegerIntervalIterator struct{}

func NewIntegerIntervalIterator

func NewIntegerIntervalIterator() *IntegerIntervalIterator

func (*IntegerIntervalIterator) Next

type IntegerIterator added in v1.3.0

type IntegerIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerIterator added in v1.3.0

func NewIntegerIterator(fn ColReduceFunc[int64], fv ColMergeFunc[int64],
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *IntegerIterator

func (*IntegerIterator) Next added in v1.3.0

type IntegerLinearFillProcessor

type IntegerLinearFillProcessor struct {
	// contains filtered or unexported fields
}

func NewIntegerLinearFillProcessor

func NewIntegerLinearFillProcessor(inOrdinal, outOrdinal int) *IntegerLinearFillProcessor

type IntegerMovingAverageItem

type IntegerMovingAverageItem struct {
	// contains filtered or unexported fields
}

func NewIntegerMovingAverageItem

func NewIntegerMovingAverageItem(n int) *IntegerMovingAverageItem

func (*IntegerMovingAverageItem) AppendItem

func (f *IntegerMovingAverageItem) AppendItem(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerMovingAverageItem) AppendItemFastFunc

func (f *IntegerMovingAverageItem) AppendItemFastFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerMovingAverageItem) AppendItemSlowFunc

func (f *IntegerMovingAverageItem) AppendItemSlowFunc(c Chunk, ordinal int, start, end int, sameInterval bool)

func (*IntegerMovingAverageItem) GetBaseTransData

func (f *IntegerMovingAverageItem) GetBaseTransData() BaseTransData

func (*IntegerMovingAverageItem) Len

func (f *IntegerMovingAverageItem) Len() int

func (*IntegerMovingAverageItem) PrevNil

func (f *IntegerMovingAverageItem) PrevNil() bool

func (*IntegerMovingAverageItem) Reset

func (f *IntegerMovingAverageItem) Reset()

func (*IntegerMovingAverageItem) ResetPrev

func (f *IntegerMovingAverageItem) ResetPrev()

type IntegerNullFillProcessor

type IntegerNullFillProcessor struct {
	// contains filtered or unexported fields
}

func NewIntegerNullFillProcessor

func NewIntegerNullFillProcessor(inOrdinal, outOrdinal int) *IntegerNullFillProcessor

type IntegerNumberFillProcessor

type IntegerNumberFillProcessor struct {
	// contains filtered or unexported fields
}

func NewIntegerNumberFillProcessor

func NewIntegerNumberFillProcessor(inOrdinal, outOrdinal int) *IntegerNumberFillProcessor

type IntegerOGSketchInsertItem added in v1.0.0

type IntegerOGSketchInsertItem struct {
	// contains filtered or unexported fields
}

func NewIntegerOGSketchInsertIem added in v1.0.0

func NewIntegerOGSketchInsertIem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *IntegerOGSketchInsertItem

func (*IntegerOGSketchInsertItem) IsNil added in v1.0.0

func (o *IntegerOGSketchInsertItem) IsNil() bool

func (*IntegerOGSketchInsertItem) Reset added in v1.0.0

func (o *IntegerOGSketchInsertItem) Reset()

func (*IntegerOGSketchInsertItem) UpdateCluster added in v1.0.0

func (o *IntegerOGSketchInsertItem) UpdateCluster(inChunk Chunk, start, end int)

func (*IntegerOGSketchInsertItem) WriteResult added in v1.0.0

func (o *IntegerOGSketchInsertItem) WriteResult(outChunk Chunk, time int64)

type IntegerOGSketchPercentileItem added in v1.0.0

type IntegerOGSketchPercentileItem struct {
	// contains filtered or unexported fields
}

func NewIntegerOGSketchPercentileItem added in v1.0.0

func NewIntegerOGSketchPercentileItem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *IntegerOGSketchPercentileItem

func (*IntegerOGSketchPercentileItem) IsNil added in v1.0.0

func (*IntegerOGSketchPercentileItem) Reset added in v1.0.0

func (o *IntegerOGSketchPercentileItem) Reset()

func (*IntegerOGSketchPercentileItem) UpdateCluster added in v1.0.0

func (o *IntegerOGSketchPercentileItem) UpdateCluster(inChunk Chunk, start, end int)

func (*IntegerOGSketchPercentileItem) WriteResult added in v1.0.0

func (o *IntegerOGSketchPercentileItem) WriteResult(outChunk Chunk, time int64)

type IntegerPercentileApproxItem added in v1.0.0

type IntegerPercentileApproxItem struct {
	// contains filtered or unexported fields
}

func NewIntegerPercentileApproxItem added in v1.0.0

func NewIntegerPercentileApproxItem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *IntegerPercentileApproxItem

func (*IntegerPercentileApproxItem) IsNil added in v1.0.0

func (o *IntegerPercentileApproxItem) IsNil() bool

func (*IntegerPercentileApproxItem) Reset added in v1.0.0

func (o *IntegerPercentileApproxItem) Reset()

func (*IntegerPercentileApproxItem) UpdateCluster added in v1.0.0

func (o *IntegerPercentileApproxItem) UpdateCluster(inChunk Chunk, start, end int)

func (*IntegerPercentileApproxItem) WriteResult added in v1.0.0

func (o *IntegerPercentileApproxItem) WriteResult(outChunk Chunk, time int64)

type IntegerPreviousFillProcessor

type IntegerPreviousFillProcessor struct {
	// contains filtered or unexported fields
}

func NewIntegerPreviousFillProcessor

func NewIntegerPreviousFillProcessor(inOrdinal, outOrdinal int) *IntegerPreviousFillProcessor

type IntegerTimeColIntegerIterator

type IntegerTimeColIntegerIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerTimeColIntegerIterator

func NewIntegerTimeColIntegerIterator(
	fn TimeColReduceFunc[int64], fv ColMergeFunc[int64], inOrdinal, outOrdinal int,
) *IntegerTimeColIntegerIterator

func (*IntegerTimeColIntegerIterator) Next

type IntegerTransIterator added in v1.3.0

type IntegerTransIterator struct {
	// contains filtered or unexported fields
}

func NewIntegerTransIterator added in v1.3.0

func NewIntegerTransIterator(
	isSingleCall bool, inOrdinal, outOrdinal int, transItem TransItem,
) *IntegerTransIterator

func (*IntegerTransIterator) Next added in v1.3.0

type IntegralItem added in v1.3.0

type IntegralItem[T util.NumberOnly] struct {
	// contains filtered or unexported fields
}

func NewIntegralItem added in v1.3.0

func NewIntegralItem[T util.NumberOnly](interval hybridqp.Interval, opt *query.ProcessorOptions) *IntegralItem[T]

func (*IntegralItem[T]) AppendItem added in v1.3.0

func (f *IntegralItem[T]) AppendItem(c Chunk, values []T, ordinal int, start, end int, sameInterval, sameTag bool)

func (*IntegralItem[T]) AppendItemFastFunc added in v1.3.0

func (f *IntegralItem[T]) AppendItemFastFunc(c Chunk, values []T, start, end int, sameInterval bool, sameTag bool)

func (*IntegralItem[T]) AppendItemSlowFunc added in v1.3.0

func (f *IntegralItem[T]) AppendItemSlowFunc(c Chunk, values []T, ordinal int, vs, ve int, sameInterval, sameTag bool)

func (*IntegralItem[T]) CalculateUnit added in v1.3.0

func (f *IntegralItem[T]) CalculateUnit(index int, time int64, value float64)

func (*IntegralItem[T]) Len added in v1.3.0

func (f *IntegralItem[T]) Len() int

func (*IntegralItem[T]) Nil added in v1.3.0

func (f *IntegralItem[T]) Nil() bool

func (*IntegralItem[T]) Reset added in v1.3.0

func (f *IntegralItem[T]) Reset()

func (*IntegralItem[T]) StartNewInterval added in v1.3.0

func (f *IntegralItem[T]) StartNewInterval(time int64)

type IntervalKeysMPool added in v1.1.0

type IntervalKeysMPool struct {
	// contains filtered or unexported fields
}

func NewIntervalKeysMpool added in v1.1.0

func NewIntervalKeysMpool(size int) *IntervalKeysMPool

func (*IntervalKeysMPool) AllocIntervalKeys added in v1.1.0

func (gkp *IntervalKeysMPool) AllocIntervalKeys(size int) []int64

func (*IntervalKeysMPool) AllocValues added in v1.1.0

func (gkp *IntervalKeysMPool) AllocValues(size int) []uint64

func (*IntervalKeysMPool) FreeIntervalKeys added in v1.1.0

func (gkp *IntervalKeysMPool) FreeIntervalKeys(intervalKeys []int64)

func (*IntervalKeysMPool) FreeValues added in v1.1.0

func (gkp *IntervalKeysMPool) FreeValues(values []uint64)

type IntervalToProjectInSubQueryRule

type IntervalToProjectInSubQueryRule struct {
	OptRuleBase
}

func NewIntervalToProjectInSubQueryRule

func NewIntervalToProjectInSubQueryRule(description string) *IntervalToProjectInSubQueryRule

func (*IntervalToProjectInSubQueryRule) Category added in v1.5.0

func (*IntervalToProjectInSubQueryRule) Equals

func (*IntervalToProjectInSubQueryRule) OnMatch

func (r *IntervalToProjectInSubQueryRule) OnMatch(call *OptRuleCall)

func (*IntervalToProjectInSubQueryRule) ToString

type IntervalTransform

type IntervalTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewIntervalTransform

func NewIntervalTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, opt *query.ProcessorOptions) *IntervalTransform

func (*IntervalTransform) Close

func (trans *IntervalTransform) Close()

func (*IntervalTransform) Explain

func (trans *IntervalTransform) Explain() []ValuePair

func (*IntervalTransform) GetInputNumber

func (trans *IntervalTransform) GetInputNumber(port Port) int

func (*IntervalTransform) GetInputs

func (trans *IntervalTransform) GetInputs() Ports

func (*IntervalTransform) GetOutputNumber

func (trans *IntervalTransform) GetOutputNumber(port Port) int

func (*IntervalTransform) GetOutputs

func (trans *IntervalTransform) GetOutputs() Ports

func (*IntervalTransform) Name

func (trans *IntervalTransform) Name() string

func (*IntervalTransform) Work

func (trans *IntervalTransform) Work(ctx context.Context) error

type IntervalTransformCreator

type IntervalTransformCreator struct {
}

func (*IntervalTransformCreator) Create

type Item

type Item struct {
	Input         int
	Index         int
	TagIndex      int
	IntervalIndex int
	ChunkBuf      Chunk
}

func NewItem

func NewItem(in int, c Chunk) *Item

func (*Item) GetTag

func (it *Item) GetTag(i int) (ChunkTags, bool)

func (*Item) GetTimeAndTag

func (it *Item) GetTimeAndTag(i int) (int64, ChunkTags, bool)

func (*Item) IntervalLen

func (it *Item) IntervalLen() int

func (*Item) IsEmpty

func (it *Item) IsEmpty() bool

func (*Item) IsSortedEmpty

func (it *Item) IsSortedEmpty() bool

func (*Item) Len

func (it *Item) Len() int

func (*Item) TagSwitch

func (it *Item) TagSwitch(i int) bool

type Iterator

type Iterator interface {
	Next(*IteratorEndpoint, *IteratorParams)
}

type IteratorEndpoint

type IteratorEndpoint struct {
	InputPoint  EndPointPair
	OutputPoint EndPointPair
}

type IteratorParams

type IteratorParams struct {
	Table ReflectionTable
	// contains filtered or unexported fields
}

func (*IteratorParams) GetErr added in v1.0.0

func (i *IteratorParams) GetErr() error

type JoinTransformCreator added in v1.5.0

type JoinTransformCreator struct {
}

func (*JoinTransformCreator) Create added in v1.5.0

type LLMClassify added in v1.5.0

type LLMClassify struct {
	BaseLLMFunc
}

func (*LLMClassify) GetPrompt added in v1.5.0

func (o *LLMClassify) GetPrompt(prompt string) string

func (*LLMClassify) InitOp added in v1.5.0

func (o *LLMClassify) InitOp(arg string, callLLM func(prompt string, input string) ([]string, error)) LLMFunc

func (*LLMClassify) Name added in v1.5.0

func (o *LLMClassify) Name() string

type LLMFunc added in v1.5.0

type LLMFunc interface {
	Name() string
	InitOp(arg string, f func(prompt string, input string) ([]string, error)) LLMFunc
	Process(dst, src Column) error
}

func GetLLMFunc added in v1.5.0

func GetLLMFunc(name string) LLMFunc

type LLMGenerate added in v1.5.0

type LLMGenerate struct {
	BaseLLMFunc
}

func (*LLMGenerate) GetPrompt added in v1.5.0

func (o *LLMGenerate) GetPrompt(prompt string) string

func (*LLMGenerate) InitOp added in v1.5.0

func (o *LLMGenerate) InitOp(arg string, callLLM func(prompt string, input string) ([]string, error)) LLMFunc

func (*LLMGenerate) Name added in v1.5.0

func (o *LLMGenerate) Name() string

type LLMSemanticTransform added in v1.5.0

type LLMSemanticTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

LLMSemanticTransform is a semantic transformation operator implementing the Processor interface

func NewLLMSemanticTransform added in v1.5.0

func NewLLMSemanticTransform(inRowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, schema hybridqp.Catalog) (*LLMSemanticTransform, error)

NewLLMSemanticTransform creates a new instance of LLMSemanticTransform

func (*LLMSemanticTransform) Close added in v1.5.0

func (st *LLMSemanticTransform) Close()

func (*LLMSemanticTransform) Explain added in v1.5.0

func (st *LLMSemanticTransform) Explain() []ValuePair

func (*LLMSemanticTransform) GetInputNumber added in v1.5.0

func (st *LLMSemanticTransform) GetInputNumber(port Port) int

func (*LLMSemanticTransform) GetInputs added in v1.5.0

func (st *LLMSemanticTransform) GetInputs() Ports

func (*LLMSemanticTransform) GetOutputNumber added in v1.5.0

func (st *LLMSemanticTransform) GetOutputNumber(port Port) int

func (*LLMSemanticTransform) GetOutputs added in v1.5.0

func (st *LLMSemanticTransform) GetOutputs() Ports

func (*LLMSemanticTransform) Name added in v1.5.0

func (st *LLMSemanticTransform) Name() string

func (*LLMSemanticTransform) Work added in v1.5.0

func (st *LLMSemanticTransform) Work(ctx context.Context) error

type LLMSemanticTransformCreator added in v1.5.0

type LLMSemanticTransformCreator struct {
}

func (*LLMSemanticTransformCreator) Create added in v1.5.0

type LLMSummarize added in v1.5.0

type LLMSummarize struct {
	BaseLLMFunc
}

func (*LLMSummarize) GetPrompt added in v1.5.0

func (o *LLMSummarize) GetPrompt(prompt string) string

func (*LLMSummarize) InitOp added in v1.5.0

func (o *LLMSummarize) InitOp(arg string, callLLM func(prompt string, input string) ([]string, error)) LLMFunc

func (*LLMSummarize) Name added in v1.5.0

func (o *LLMSummarize) Name() string

type LabelFunction added in v1.3.0

type LabelFunction interface {
	CallFunc(name string, args []interface{}) (interface{}, bool)
}

type LabelValuer added in v1.3.0

type LabelValuer struct{}

valuer

func (LabelValuer) Call added in v1.3.0

func (v LabelValuer) Call(name string, args []interface{}) (interface{}, bool)

func (LabelValuer) SetValuer added in v1.3.0

func (LabelValuer) SetValuer(_ influxql.Valuer, _ int)

func (LabelValuer) Value added in v1.3.0

func (LabelValuer) Value(_ string) (interface{}, bool)

type LimitPushdownToExchangeRule

type LimitPushdownToExchangeRule struct {
	OptRuleBase
}

func NewLimitPushdownToExchangeRule

func NewLimitPushdownToExchangeRule(description string) *LimitPushdownToExchangeRule

func (*LimitPushdownToExchangeRule) Category added in v1.5.0

func (*LimitPushdownToExchangeRule) Equals

func (r *LimitPushdownToExchangeRule) Equals(rhs OptRule) bool

func (*LimitPushdownToExchangeRule) OnMatch

func (r *LimitPushdownToExchangeRule) OnMatch(call *OptRuleCall)

func (*LimitPushdownToExchangeRule) ToString

func (r *LimitPushdownToExchangeRule) ToString() string

type LimitPushdownToReaderRule

type LimitPushdownToReaderRule struct {
	OptRuleBase
}

func NewLimitPushdownToReaderRule

func NewLimitPushdownToReaderRule(description string) *LimitPushdownToReaderRule

func (*LimitPushdownToReaderRule) Category added in v1.5.0

func (*LimitPushdownToReaderRule) Equals

func (r *LimitPushdownToReaderRule) Equals(rhs OptRule) bool

func (*LimitPushdownToReaderRule) OnMatch

func (r *LimitPushdownToReaderRule) OnMatch(call *OptRuleCall)

func (*LimitPushdownToReaderRule) ToString

func (r *LimitPushdownToReaderRule) ToString() string

type LimitPushdownToSeriesRule

type LimitPushdownToSeriesRule struct {
	OptRuleBase
}

func NewLimitPushdownToSeriesRule

func NewLimitPushdownToSeriesRule(description string) *LimitPushdownToSeriesRule

func (*LimitPushdownToSeriesRule) Category added in v1.5.0

func (*LimitPushdownToSeriesRule) Equals

func (r *LimitPushdownToSeriesRule) Equals(rhs OptRule) bool

func (*LimitPushdownToSeriesRule) OnMatch

func (r *LimitPushdownToSeriesRule) OnMatch(call *OptRuleCall)

func (*LimitPushdownToSeriesRule) ToString

func (r *LimitPushdownToSeriesRule) ToString() string

type LimitTransform

type LimitTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts

	Opt *query.ProcessorOptions

	Count       int
	PreTag      ChunkTags
	CurrItem    Chunk
	NewChunk    Chunk
	CoProcessor CoProcessor

	ChunkInit   bool
	LimitHelper func()

	TagIndex      int
	IntervalIndex int
	// contains filtered or unexported fields
}

func NewLimitTransform

func NewLimitTransform(inRowDataType []hybridqp.RowDataType, outRowDataType []hybridqp.RowDataType, opt *query.ProcessorOptions, para LimitTransformParameters) *LimitTransform

func (*LimitTransform) AppendPoint

func (trans *LimitTransform) AppendPoint(chunk Chunk, in int)

func (*LimitTransform) AppendPoints

func (trans *LimitTransform) AppendPoints(chunk Chunk, in int)

func (*LimitTransform) Close

func (trans *LimitTransform) Close()

func (*LimitTransform) Explain

func (trans *LimitTransform) Explain() []ValuePair

func (*LimitTransform) GetInputNumber

func (trans *LimitTransform) GetInputNumber(port Port) int

func (*LimitTransform) GetInputs

func (trans *LimitTransform) GetInputs() Ports

func (*LimitTransform) GetMultiRowsIndexPara

func (trans *LimitTransform) GetMultiRowsIndexPara() int

func (*LimitTransform) GetOutputNumber

func (trans *LimitTransform) GetOutputNumber(port Port) int

func (*LimitTransform) GetOutputs

func (trans *LimitTransform) GetOutputs() Ports

func (*LimitTransform) GetSingleRowPara

func (trans *LimitTransform) GetSingleRowPara() int

func (*LimitTransform) IntervalIndexGen

func (trans *LimitTransform) IntervalIndexGen()

func (*LimitTransform) MultipleRowsIgnoreTagLimitHelper

func (trans *LimitTransform) MultipleRowsIgnoreTagLimitHelper()

func (*LimitTransform) MultipleRowsLimitHelper

func (trans *LimitTransform) MultipleRowsLimitHelper()

func (*LimitTransform) Name

func (trans *LimitTransform) Name() string

func (*LimitTransform) SameGroup

func (trans *LimitTransform) SameGroup(i int) bool

func (*LimitTransform) SendChunk

func (trans *LimitTransform) SendChunk()

func (*LimitTransform) SetDag

func (trans *LimitTransform) SetDag(dag *TransformDag)

func (*LimitTransform) SetVertex

func (trans *LimitTransform) SetVertex(vertex *TransformVertex)

func (*LimitTransform) SingleRowIgnoreTagLimitHelper

func (trans *LimitTransform) SingleRowIgnoreTagLimitHelper()

func (*LimitTransform) SingleRowLimitHelper

func (trans *LimitTransform) SingleRowLimitHelper()

func (*LimitTransform) Visit

func (*LimitTransform) Work

func (trans *LimitTransform) Work(ctx context.Context) error

type LimitTransformCreator

type LimitTransformCreator struct {
}

func (*LimitTransformCreator) Create

type LimitTransformParameters

type LimitTransformParameters struct {
	Limit     int
	Offset    int
	LimitType hybridqp.LimitType
}

type Loc added in v1.3.0

type Loc struct {
	ChunkLoc int
	GroupLoc int
	RowLoc   int
	// contains filtered or unexported fields
}

type LogicalAggregate

type LogicalAggregate struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewCountDistinctAggregate

func NewCountDistinctAggregate(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalAggregate

func NewLogicalAggregate

func NewLogicalAggregate(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalAggregate

func NewLogicalTagSetAggregate

func NewLogicalTagSetAggregate(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalAggregate

func (*LogicalAggregate) Clone

func (p *LogicalAggregate) Clone() hybridqp.QueryNode

func (*LogicalAggregate) CountToSum

func (p *LogicalAggregate) CountToSum()

func (*LogicalAggregate) DeriveOperations

func (p *LogicalAggregate) DeriveOperations()

func (*LogicalAggregate) Digest

func (p *LogicalAggregate) Digest() string

func (*LogicalAggregate) Explain

func (p *LogicalAggregate) Explain(writer LogicalPlanWriter)

func (*LogicalAggregate) ForwardCallArgs

func (p *LogicalAggregate) ForwardCallArgs()

func (*LogicalAggregate) InferAggLevel added in v1.5.0

func (p *LogicalAggregate) InferAggLevel() AggLevel

func (*LogicalAggregate) InferPromAggLevel added in v1.5.0

func (p *LogicalAggregate) InferPromAggLevel() AggLevel

func (*LogicalAggregate) LogicPlanType added in v1.0.0

func (p *LogicalAggregate) LogicPlanType() internal.LogicPlanType

func (*LogicalAggregate) New added in v1.1.0

func (p *LogicalAggregate) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalAggregate) String

func (p *LogicalAggregate) String() string

func (*LogicalAggregate) Type

func (p *LogicalAggregate) Type() string

type LogicalAlign

type LogicalAlign struct {
	LogicalPlanSingle
}

func NewLogicalAlign

func NewLogicalAlign(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalAlign

func (*LogicalAlign) Clone

func (p *LogicalAlign) Clone() hybridqp.QueryNode

func (*LogicalAlign) DeriveOperations

func (p *LogicalAlign) DeriveOperations()

func (*LogicalAlign) Digest

func (p *LogicalAlign) Digest() string

func (*LogicalAlign) Explain

func (p *LogicalAlign) Explain(writer LogicalPlanWriter)

func (*LogicalAlign) LogicPlanType added in v1.0.0

func (p *LogicalAlign) LogicPlanType() internal.LogicPlanType

func (*LogicalAlign) New added in v1.1.0

func (p *LogicalAlign) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalAlign) String

func (p *LogicalAlign) String() string

func (*LogicalAlign) Type

func (p *LogicalAlign) Type() string

type LogicalBinOp added in v1.3.0

type LogicalBinOp struct {
	Para *influxql.BinOp
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalBinOp added in v1.3.0

func NewLogicalBinOp(left, right hybridqp.QueryNode, lExpr, rExpr influxql.Expr, para *influxql.BinOp, schema hybridqp.Catalog) *LogicalBinOp

func (*LogicalBinOp) Children added in v1.3.0

func (p *LogicalBinOp) Children() []hybridqp.QueryNode

func (*LogicalBinOp) Clone added in v1.3.0

func (p *LogicalBinOp) Clone() hybridqp.QueryNode

func (*LogicalBinOp) DeriveOperations added in v1.3.0

func (p *LogicalBinOp) DeriveOperations()

func (*LogicalBinOp) Digest added in v1.3.0

func (p *LogicalBinOp) Digest() string

func (*LogicalBinOp) Explain added in v1.3.0

func (p *LogicalBinOp) Explain(writer LogicalPlanWriter)

func (*LogicalBinOp) LogicPlanType added in v1.3.0

func (p *LogicalBinOp) LogicPlanType() internal.LogicPlanType

func (*LogicalBinOp) New added in v1.3.0

func (p *LogicalBinOp) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalBinOp) ReplaceChild added in v1.3.0

func (p *LogicalBinOp) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalBinOp) ReplaceChildren added in v1.3.0

func (p *LogicalBinOp) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalBinOp) String added in v1.3.0

func (p *LogicalBinOp) String() string

func (*LogicalBinOp) Type added in v1.3.0

func (p *LogicalBinOp) Type() string

type LogicalCTE added in v1.5.0

type LogicalCTE struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalCTE added in v1.5.0

func NewLogicalCTE(cte *influxql.CTE, schema hybridqp.Catalog, ctePlan hybridqp.QueryNode) *LogicalCTE

func (*LogicalCTE) Clone added in v1.5.0

func (p *LogicalCTE) Clone() hybridqp.QueryNode

func (*LogicalCTE) DeriveOperations added in v1.5.0

func (p *LogicalCTE) DeriveOperations()

func (*LogicalCTE) Digest added in v1.5.0

func (p *LogicalCTE) Digest() string

func (*LogicalCTE) Explain added in v1.5.0

func (p *LogicalCTE) Explain(writer LogicalPlanWriter)

func (*LogicalCTE) GetCTEPlan added in v1.5.0

func (p *LogicalCTE) GetCTEPlan() hybridqp.QueryNode

func (*LogicalCTE) LogicPlanType added in v1.5.0

func (p *LogicalCTE) LogicPlanType() internal.LogicPlanType

func (*LogicalCTE) New added in v1.5.0

func (p *LogicalCTE) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalCTE) ReplaceChild added in v1.5.0

func (p *LogicalCTE) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalCTE) ReplaceChildren added in v1.5.0

func (p *LogicalCTE) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalCTE) String added in v1.5.0

func (p *LogicalCTE) String() string

func (*LogicalCTE) Type added in v1.5.0

func (p *LogicalCTE) Type() string

type LogicalColumnStoreReader added in v1.1.0

type LogicalColumnStoreReader struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalColumnStoreReader added in v1.1.0

func NewLogicalColumnStoreReader(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalColumnStoreReader

func (*LogicalColumnStoreReader) Clone added in v1.1.0

func (*LogicalColumnStoreReader) DeriveOperations added in v1.1.0

func (p *LogicalColumnStoreReader) DeriveOperations()

func (*LogicalColumnStoreReader) Digest added in v1.1.0

func (p *LogicalColumnStoreReader) Digest() string

func (*LogicalColumnStoreReader) Explain added in v1.1.0

func (p *LogicalColumnStoreReader) Explain(writer LogicalPlanWriter)

func (*LogicalColumnStoreReader) ExplainIterms added in v1.1.0

func (p *LogicalColumnStoreReader) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalColumnStoreReader) LogicPlanType added in v1.1.0

func (p *LogicalColumnStoreReader) LogicPlanType() internal.LogicPlanType

func (*LogicalColumnStoreReader) MstName added in v1.1.0

func (p *LogicalColumnStoreReader) MstName() string

func (*LogicalColumnStoreReader) New added in v1.1.0

impl me

func (*LogicalColumnStoreReader) String added in v1.1.0

func (p *LogicalColumnStoreReader) String() string

func (*LogicalColumnStoreReader) Type added in v1.1.0

func (p *LogicalColumnStoreReader) Type() string

type LogicalDistinct added in v1.5.0

type LogicalDistinct struct {
	LogicalPlanSingle
}

func NewLogicalDistinct added in v1.5.0

func NewLogicalDistinct(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalDistinct

func (*LogicalDistinct) Clone added in v1.5.0

func (p *LogicalDistinct) Clone() hybridqp.QueryNode

func (*LogicalDistinct) DeriveOperations added in v1.5.0

func (p *LogicalDistinct) DeriveOperations()

func (*LogicalDistinct) Digest added in v1.5.0

func (p *LogicalDistinct) Digest() string

func (*LogicalDistinct) Explain added in v1.5.0

func (p *LogicalDistinct) Explain(writer LogicalPlanWriter)

func (*LogicalDistinct) LogicPlanType added in v1.5.0

func (p *LogicalDistinct) LogicPlanType() internal.LogicPlanType

func (*LogicalDistinct) New added in v1.5.0

func (p *LogicalDistinct) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalDistinct) String added in v1.5.0

func (p *LogicalDistinct) String() string

func (*LogicalDistinct) Type added in v1.5.0

func (p *LogicalDistinct) Type() string

type LogicalDummyShard

type LogicalDummyShard struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalDummyShard

func NewLogicalDummyShard(info comm.TSIndexInfo) *LogicalDummyShard

func (*LogicalDummyShard) Children

func (p *LogicalDummyShard) Children() []hybridqp.QueryNode

func (*LogicalDummyShard) Clone

func (*LogicalDummyShard) Digest

func (p *LogicalDummyShard) Digest() string

func (*LogicalDummyShard) Explain

func (p *LogicalDummyShard) Explain(writer LogicalPlanWriter)

func (*LogicalDummyShard) ExplainIterms

func (p *LogicalDummyShard) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalDummyShard) GetIndexInfo added in v1.4.0

func (p *LogicalDummyShard) GetIndexInfo() comm.TSIndexInfo

func (*LogicalDummyShard) LogicPlanType added in v1.0.0

func (p *LogicalDummyShard) LogicPlanType() internal.LogicPlanType

func (*LogicalDummyShard) New added in v1.1.0

impl me

func (*LogicalDummyShard) ReplaceChild

func (p *LogicalDummyShard) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalDummyShard) ReplaceChildren

func (p *LogicalDummyShard) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalDummyShard) String

func (p *LogicalDummyShard) String() string

func (*LogicalDummyShard) Type

func (p *LogicalDummyShard) Type() string

type LogicalExchange

type LogicalExchange struct {
	LogicalPlanSingle
	LogicalExchangeBase
}

func NewLogicalExchange

func NewLogicalExchange(input hybridqp.QueryNode, eType ExchangeType, eTraits []hybridqp.Trait, schema hybridqp.Catalog) *LogicalExchange

func (*LogicalExchange) AddTrait

func (p *LogicalExchange) AddTrait(trait interface{})

func (*LogicalExchange) Clone

func (p *LogicalExchange) Clone() hybridqp.QueryNode

func (*LogicalExchange) DeriveOperations

func (p *LogicalExchange) DeriveOperations()

func (*LogicalExchange) Digest

func (p *LogicalExchange) Digest() string

func (*LogicalExchange) Explain

func (p *LogicalExchange) Explain(writer LogicalPlanWriter)

func (*LogicalExchange) ExplainIterms

func (p *LogicalExchange) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalExchange) LogicPlanType added in v1.0.0

func (p *LogicalExchange) LogicPlanType() internal.LogicPlanType

func (*LogicalExchange) New added in v1.1.0

func (p *LogicalExchange) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalExchange) Schema added in v1.2.0

func (p *LogicalExchange) Schema() hybridqp.Catalog

func (*LogicalExchange) String

func (p *LogicalExchange) String() string

func (*LogicalExchange) Type

func (p *LogicalExchange) Type() string

type LogicalExchangeBase added in v1.2.0

type LogicalExchangeBase struct {
	// contains filtered or unexported fields
}

func NewLogicalExchangeBase added in v1.2.0

func NewLogicalExchangeBase(eType ExchangeType, eRole ExchangeRole, eTraits []hybridqp.Trait) *LogicalExchangeBase

func (*LogicalExchangeBase) ERole added in v1.2.0

func (p *LogicalExchangeBase) ERole() ExchangeRole

func (*LogicalExchangeBase) ETraits added in v1.2.0

func (p *LogicalExchangeBase) ETraits() []hybridqp.Trait

func (*LogicalExchangeBase) EType added in v1.2.0

func (p *LogicalExchangeBase) EType() ExchangeType

func (*LogicalExchangeBase) ToProducer added in v1.2.0

func (p *LogicalExchangeBase) ToProducer()

type LogicalFill

type LogicalFill struct {
	LogicalPlanSingle
}

func NewLogicalFill

func NewLogicalFill(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalFill

func (*LogicalFill) Clone

func (p *LogicalFill) Clone() hybridqp.QueryNode

func (*LogicalFill) DeriveOperations

func (p *LogicalFill) DeriveOperations()

func (*LogicalFill) Digest

func (p *LogicalFill) Digest() string

func (*LogicalFill) Explain

func (p *LogicalFill) Explain(writer LogicalPlanWriter)

func (*LogicalFill) LogicPlanType added in v1.0.0

func (p *LogicalFill) LogicPlanType() internal.LogicPlanType

func (*LogicalFill) New added in v1.1.0

func (p *LogicalFill) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalFill) String

func (p *LogicalFill) String() string

func (*LogicalFill) Type

func (p *LogicalFill) Type() string

type LogicalFilter

type LogicalFilter struct {
	LogicalPlanSingle
}

func NewLogicalFilter

func NewLogicalFilter(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalFilter

func (*LogicalFilter) Clone

func (p *LogicalFilter) Clone() hybridqp.QueryNode

func (*LogicalFilter) DeriveOperations

func (p *LogicalFilter) DeriveOperations()

func (*LogicalFilter) Digest

func (p *LogicalFilter) Digest() string

func (*LogicalFilter) Explain

func (p *LogicalFilter) Explain(writer LogicalPlanWriter)

func (*LogicalFilter) LogicPlanType added in v1.0.0

func (p *LogicalFilter) LogicPlanType() internal.LogicPlanType

func (*LogicalFilter) New added in v1.1.0

func (p *LogicalFilter) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalFilter) String

func (p *LogicalFilter) String() string

func (*LogicalFilter) Type

func (p *LogicalFilter) Type() string

type LogicalFilterBlank

type LogicalFilterBlank struct {
	LogicalPlanSingle
}

func NewLogicalFilterBlank

func NewLogicalFilterBlank(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalFilterBlank

func (*LogicalFilterBlank) Clone

func (*LogicalFilterBlank) DeriveOperations

func (p *LogicalFilterBlank) DeriveOperations()

func (*LogicalFilterBlank) Digest

func (p *LogicalFilterBlank) Digest() string

func (*LogicalFilterBlank) Explain

func (p *LogicalFilterBlank) Explain(writer LogicalPlanWriter)

func (*LogicalFilterBlank) LogicPlanType added in v1.0.0

func (p *LogicalFilterBlank) LogicPlanType() internal.LogicPlanType

func (*LogicalFilterBlank) New added in v1.1.0

impl me

func (*LogicalFilterBlank) String

func (p *LogicalFilterBlank) String() string

func (*LogicalFilterBlank) Type

func (p *LogicalFilterBlank) Type() string

type LogicalFullJoin added in v1.0.0

type LogicalFullJoin struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalFullJoin added in v1.0.0

func NewLogicalFullJoin(left hybridqp.QueryNode, right hybridqp.QueryNode, condition influxql.Expr, schema hybridqp.Catalog) *LogicalFullJoin

func (*LogicalFullJoin) Children added in v1.0.0

func (p *LogicalFullJoin) Children() []hybridqp.QueryNode

func (*LogicalFullJoin) Clone added in v1.0.0

func (p *LogicalFullJoin) Clone() hybridqp.QueryNode

func (*LogicalFullJoin) DeriveOperations added in v1.0.0

func (p *LogicalFullJoin) DeriveOperations()

func (*LogicalFullJoin) Digest added in v1.0.0

func (p *LogicalFullJoin) Digest() string

func (*LogicalFullJoin) Explain added in v1.0.0

func (p *LogicalFullJoin) Explain(writer LogicalPlanWriter)

func (*LogicalFullJoin) LogicPlanType added in v1.0.0

func (p *LogicalFullJoin) LogicPlanType() internal.LogicPlanType

func (*LogicalFullJoin) New added in v1.1.0

func (p *LogicalFullJoin) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalFullJoin) ReplaceChild added in v1.0.0

func (p *LogicalFullJoin) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalFullJoin) ReplaceChildren added in v1.0.0

func (p *LogicalFullJoin) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalFullJoin) String added in v1.0.0

func (p *LogicalFullJoin) String() string

func (*LogicalFullJoin) Type added in v1.0.0

func (p *LogicalFullJoin) Type() string

type LogicalGraph added in v1.5.0

type LogicalGraph struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalGraph added in v1.5.0

func (*LogicalGraph) Children added in v1.5.0

func (p *LogicalGraph) Children() []hybridqp.QueryNode

func (*LogicalGraph) Clone added in v1.5.0

func (p *LogicalGraph) Clone() hybridqp.QueryNode

func (*LogicalGraph) DeriveOperations added in v1.5.0

func (p *LogicalGraph) DeriveOperations()

func (*LogicalGraph) Digest added in v1.5.0

func (p *LogicalGraph) Digest() string

func (*LogicalGraph) Explain added in v1.5.0

func (p *LogicalGraph) Explain(writer LogicalPlanWriter)

func (*LogicalGraph) LogicPlanType added in v1.5.0

func (p *LogicalGraph) LogicPlanType() internal.LogicPlanType

func (*LogicalGraph) New added in v1.5.0

func (p *LogicalGraph) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalGraph) ReplaceChild added in v1.5.0

func (p *LogicalGraph) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalGraph) ReplaceChildren added in v1.5.0

func (p *LogicalGraph) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalGraph) String added in v1.5.0

func (p *LogicalGraph) String() string

func (*LogicalGraph) Type added in v1.5.0

func (p *LogicalGraph) Type() string

type LogicalGroupBy

type LogicalGroupBy struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalGroupBy

func NewLogicalGroupBy(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalGroupBy

func (*LogicalGroupBy) Clone

func (p *LogicalGroupBy) Clone() hybridqp.QueryNode

func (*LogicalGroupBy) DeriveOperations

func (p *LogicalGroupBy) DeriveOperations()

func (*LogicalGroupBy) Digest

func (p *LogicalGroupBy) Digest() string

func (*LogicalGroupBy) Explain

func (p *LogicalGroupBy) Explain(writer LogicalPlanWriter)

func (*LogicalGroupBy) ExplainIterms

func (p *LogicalGroupBy) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalGroupBy) LogicPlanType added in v1.0.0

func (p *LogicalGroupBy) LogicPlanType() internal.LogicPlanType

func (*LogicalGroupBy) New added in v1.1.0

func (p *LogicalGroupBy) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalGroupBy) String

func (p *LogicalGroupBy) String() string

func (*LogicalGroupBy) Type

func (p *LogicalGroupBy) Type() string

type LogicalHashAgg added in v1.1.0

type LogicalHashAgg struct {
	LogicalPlanSingle

	LogicalExchangeBase
	// contains filtered or unexported fields
}

func NewLogicalHashAgg added in v1.1.0

func NewLogicalHashAgg(input hybridqp.QueryNode, schema hybridqp.Catalog, eType ExchangeType, eTraits []hybridqp.Trait) *LogicalHashAgg

func (*LogicalHashAgg) AddTrait added in v1.1.0

func (p *LogicalHashAgg) AddTrait(trait interface{})

func (*LogicalHashAgg) Clone added in v1.1.0

func (p *LogicalHashAgg) Clone() hybridqp.QueryNode

func (*LogicalHashAgg) CountToSum added in v1.1.0

func (p *LogicalHashAgg) CountToSum()

func (*LogicalHashAgg) DeriveOperations added in v1.1.0

func (p *LogicalHashAgg) DeriveOperations()

func (*LogicalHashAgg) Digest added in v1.1.0

func (p *LogicalHashAgg) Digest() string

func (*LogicalHashAgg) Explain added in v1.1.0

func (p *LogicalHashAgg) Explain(writer LogicalPlanWriter)

func (*LogicalHashAgg) ExplainIterms added in v1.5.0

func (p *LogicalHashAgg) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalHashAgg) ForwardCallArgs added in v1.1.0

func (p *LogicalHashAgg) ForwardCallArgs()

func (*LogicalHashAgg) LogicPlanType added in v1.1.0

func (p *LogicalHashAgg) LogicPlanType() internal.LogicPlanType

func (*LogicalHashAgg) New added in v1.1.0

func (p *LogicalHashAgg) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalHashAgg) Schema added in v1.2.0

func (p *LogicalHashAgg) Schema() hybridqp.Catalog

func (*LogicalHashAgg) String added in v1.1.0

func (p *LogicalHashAgg) String() string

func (*LogicalHashAgg) Type added in v1.1.0

func (p *LogicalHashAgg) Type() string

type LogicalHashMerge added in v1.1.0

type LogicalHashMerge struct {
	LogicalPlanSingle
	LogicalExchangeBase
}

func NewLogicalHashMerge added in v1.1.0

func NewLogicalHashMerge(input hybridqp.QueryNode, schema hybridqp.Catalog, eType ExchangeType, eTraits []hybridqp.Trait) *LogicalHashMerge

func (*LogicalHashMerge) AddTrait added in v1.1.0

func (p *LogicalHashMerge) AddTrait(trait interface{})

func (*LogicalHashMerge) Clone added in v1.1.0

func (p *LogicalHashMerge) Clone() hybridqp.QueryNode

func (*LogicalHashMerge) DeriveOperations added in v1.1.0

func (p *LogicalHashMerge) DeriveOperations()

func (*LogicalHashMerge) Digest added in v1.1.0

func (p *LogicalHashMerge) Digest() string

func (*LogicalHashMerge) Explain added in v1.1.0

func (p *LogicalHashMerge) Explain(writer LogicalPlanWriter)

func (*LogicalHashMerge) ExplainIterms added in v1.1.0

func (p *LogicalHashMerge) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalHashMerge) LogicPlanType added in v1.1.0

func (p *LogicalHashMerge) LogicPlanType() internal.LogicPlanType

func (*LogicalHashMerge) New added in v1.1.0

func (p *LogicalHashMerge) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalHashMerge) String added in v1.1.0

func (p *LogicalHashMerge) String() string

func (*LogicalHashMerge) Type added in v1.1.0

func (p *LogicalHashMerge) Type() string

type LogicalHoltWinters added in v1.0.0

type LogicalHoltWinters struct {
	LogicalPlanSingle
}

func NewLogicalHoltWinters added in v1.0.0

func NewLogicalHoltWinters(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalHoltWinters

func (*LogicalHoltWinters) Clone added in v1.0.0

func (*LogicalHoltWinters) DeriveOperations added in v1.0.0

func (p *LogicalHoltWinters) DeriveOperations()

func (*LogicalHoltWinters) Digest added in v1.0.0

func (p *LogicalHoltWinters) Digest() string

func (*LogicalHoltWinters) Explain added in v1.0.0

func (p *LogicalHoltWinters) Explain(writer LogicalPlanWriter)

func (*LogicalHoltWinters) LogicPlanType added in v1.0.0

func (p *LogicalHoltWinters) LogicPlanType() internal.LogicPlanType

func (*LogicalHoltWinters) New added in v1.1.0

impl me

func (*LogicalHoltWinters) String added in v1.0.0

func (p *LogicalHoltWinters) String() string

func (*LogicalHoltWinters) Type added in v1.0.0

func (p *LogicalHoltWinters) Type() string

type LogicalHttpSender

type LogicalHttpSender struct {
	LogicalPlanSingle
}

func NewLogicalHttpSender

func NewLogicalHttpSender(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalHttpSender

func (*LogicalHttpSender) Clone

func (*LogicalHttpSender) DeriveOperations

func (p *LogicalHttpSender) DeriveOperations()

func (*LogicalHttpSender) Digest

func (p *LogicalHttpSender) Digest() string

func (*LogicalHttpSender) Explain

func (p *LogicalHttpSender) Explain(writer LogicalPlanWriter)

func (*LogicalHttpSender) LogicPlanType added in v1.0.0

func (p *LogicalHttpSender) LogicPlanType() internal.LogicPlanType

func (*LogicalHttpSender) New added in v1.1.0

func (*LogicalHttpSender) String

func (p *LogicalHttpSender) String() string

func (*LogicalHttpSender) Type

func (p *LogicalHttpSender) Type() string

type LogicalHttpSenderHint

type LogicalHttpSenderHint struct {
	LogicalPlanSingle
}

func NewLogicalHttpSenderHint

func NewLogicalHttpSenderHint(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalHttpSenderHint

func (*LogicalHttpSenderHint) Clone

func (*LogicalHttpSenderHint) DeriveOperations

func (p *LogicalHttpSenderHint) DeriveOperations()

func (*LogicalHttpSenderHint) Digest

func (p *LogicalHttpSenderHint) Digest() string

func (*LogicalHttpSenderHint) Explain

func (p *LogicalHttpSenderHint) Explain(writer LogicalPlanWriter)

func (*LogicalHttpSenderHint) LogicPlanType added in v1.0.0

func (p *LogicalHttpSenderHint) LogicPlanType() internal.LogicPlanType

func (*LogicalHttpSenderHint) New added in v1.1.0

impl me

func (*LogicalHttpSenderHint) String

func (p *LogicalHttpSenderHint) String() string

func (*LogicalHttpSenderHint) Type

func (p *LogicalHttpSenderHint) Type() string

type LogicalIn added in v1.5.0

type LogicalIn struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalIn added in v1.5.0

func NewLogicalIn(innerPlan hybridqp.QueryNode, outerStmt *influxql.SelectStatement, outerSchema hybridqp.Catalog,
	outerField influxql.Expr, outQc query.LogicalPlanCreator, showTagVal *influxql.ShowTagValuesStatement) *LogicalIn

func (*LogicalIn) Children added in v1.5.0

func (p *LogicalIn) Children() []hybridqp.QueryNode

func (*LogicalIn) Clone added in v1.5.0

func (p *LogicalIn) Clone() hybridqp.QueryNode

func (*LogicalIn) DeriveOperations added in v1.5.0

func (p *LogicalIn) DeriveOperations()

func (*LogicalIn) Digest added in v1.5.0

func (p *LogicalIn) Digest() string

func (*LogicalIn) Explain added in v1.5.0

func (p *LogicalIn) Explain(writer LogicalPlanWriter)

func (*LogicalIn) LogicPlanType added in v1.5.0

func (p *LogicalIn) LogicPlanType() internal.LogicPlanType

func (*LogicalIn) New added in v1.5.0

func (p *LogicalIn) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalIn) ReplaceChild added in v1.5.0

func (p *LogicalIn) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalIn) ReplaceChildren added in v1.5.0

func (p *LogicalIn) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalIn) String added in v1.5.0

func (p *LogicalIn) String() string

func (*LogicalIn) Type added in v1.5.0

func (p *LogicalIn) Type() string

type LogicalIncAgg added in v1.2.0

type LogicalIncAgg struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalIncAgg added in v1.2.0

func NewLogicalIncAgg(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalIncAgg

func (*LogicalIncAgg) Clone added in v1.2.0

func (p *LogicalIncAgg) Clone() hybridqp.QueryNode

func (*LogicalIncAgg) CountToSum added in v1.2.0

func (p *LogicalIncAgg) CountToSum()

func (*LogicalIncAgg) DeriveOperations added in v1.2.0

func (p *LogicalIncAgg) DeriveOperations()

func (*LogicalIncAgg) Digest added in v1.2.0

func (p *LogicalIncAgg) Digest() string

func (*LogicalIncAgg) Explain added in v1.2.0

func (p *LogicalIncAgg) Explain(writer LogicalPlanWriter)

func (*LogicalIncAgg) ForwardCallArgs added in v1.2.0

func (p *LogicalIncAgg) ForwardCallArgs()

func (*LogicalIncAgg) New added in v1.2.0

func (p *LogicalIncAgg) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalIncAgg) String added in v1.2.0

func (p *LogicalIncAgg) String() string

func (*LogicalIncAgg) Type added in v1.2.0

func (p *LogicalIncAgg) Type() string

type LogicalIncHashAgg added in v1.2.0

type LogicalIncHashAgg struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalIncHashAgg added in v1.2.0

func NewLogicalIncHashAgg(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalIncHashAgg

func (*LogicalIncHashAgg) Clone added in v1.2.0

func (*LogicalIncHashAgg) CountToSum added in v1.2.0

func (p *LogicalIncHashAgg) CountToSum()

func (*LogicalIncHashAgg) DeriveOperations added in v1.2.0

func (p *LogicalIncHashAgg) DeriveOperations()

func (*LogicalIncHashAgg) Digest added in v1.2.0

func (p *LogicalIncHashAgg) Digest() string

func (*LogicalIncHashAgg) Explain added in v1.2.0

func (p *LogicalIncHashAgg) Explain(writer LogicalPlanWriter)

func (*LogicalIncHashAgg) ForwardCallArgs added in v1.2.0

func (p *LogicalIncHashAgg) ForwardCallArgs()

func (*LogicalIncHashAgg) New added in v1.2.0

func (*LogicalIncHashAgg) String added in v1.2.0

func (p *LogicalIncHashAgg) String() string

func (*LogicalIncHashAgg) Type added in v1.2.0

func (p *LogicalIncHashAgg) Type() string

type LogicalIndexScan

type LogicalIndexScan struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalIndexScan

func NewLogicalIndexScan(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalIndexScan

func (*LogicalIndexScan) Clone

func (p *LogicalIndexScan) Clone() hybridqp.QueryNode

func (*LogicalIndexScan) DeriveOperations

func (p *LogicalIndexScan) DeriveOperations()

func (*LogicalIndexScan) Digest

func (p *LogicalIndexScan) Digest() string

func (*LogicalIndexScan) Explain

func (p *LogicalIndexScan) Explain(writer LogicalPlanWriter)

func (*LogicalIndexScan) GetOneShardState added in v1.2.0

func (p *LogicalIndexScan) GetOneShardState() bool

func (*LogicalIndexScan) LogicPlanType added in v1.0.0

func (p *LogicalIndexScan) LogicPlanType() internal.LogicPlanType

func (*LogicalIndexScan) New added in v1.1.0

func (p *LogicalIndexScan) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalIndexScan) SetOneShardState added in v1.2.0

func (p *LogicalIndexScan) SetOneShardState(enable bool)

func (*LogicalIndexScan) String

func (p *LogicalIndexScan) String() string

func (*LogicalIndexScan) Type

func (p *LogicalIndexScan) Type() string

type LogicalInterval

type LogicalInterval struct {
	LogicalPlanSingle
}

func NewLogicalInterval

func NewLogicalInterval(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalInterval

func (*LogicalInterval) Clone

func (p *LogicalInterval) Clone() hybridqp.QueryNode

func (*LogicalInterval) DeriveOperations

func (p *LogicalInterval) DeriveOperations()

func (*LogicalInterval) Digest

func (p *LogicalInterval) Digest() string

func (*LogicalInterval) Explain

func (p *LogicalInterval) Explain(writer LogicalPlanWriter)

func (*LogicalInterval) LogicPlanType added in v1.0.0

func (p *LogicalInterval) LogicPlanType() internal.LogicPlanType

func (*LogicalInterval) New added in v1.1.0

func (p *LogicalInterval) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalInterval) String

func (p *LogicalInterval) String() string

func (*LogicalInterval) Type

func (p *LogicalInterval) Type() string

type LogicalJoin added in v1.1.0

type LogicalJoin struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalJoin added in v1.1.0

func NewLogicalJoin(left hybridqp.QueryNode, right hybridqp.QueryNode, condition influxql.Expr, joinType influxql.JoinType, schema hybridqp.Catalog) *LogicalJoin

func (*LogicalJoin) Children added in v1.5.0

func (p *LogicalJoin) Children() []hybridqp.QueryNode

func (*LogicalJoin) Clone added in v1.1.0

func (p *LogicalJoin) Clone() hybridqp.QueryNode

func (*LogicalJoin) DeriveOperations added in v1.1.0

func (p *LogicalJoin) DeriveOperations()

func (*LogicalJoin) Digest added in v1.1.0

func (p *LogicalJoin) Digest() string

func (*LogicalJoin) Explain added in v1.1.0

func (p *LogicalJoin) Explain(writer LogicalPlanWriter)

func (*LogicalJoin) LogicPlanType added in v1.1.0

func (p *LogicalJoin) LogicPlanType() internal.LogicPlanType

func (*LogicalJoin) New added in v1.1.0

func (p *LogicalJoin) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalJoin) ReplaceChild added in v1.5.0

func (p *LogicalJoin) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalJoin) ReplaceChildren added in v1.5.0

func (p *LogicalJoin) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalJoin) String added in v1.1.0

func (p *LogicalJoin) String() string

func (*LogicalJoin) Type added in v1.1.0

func (p *LogicalJoin) Type() string

type LogicalLLMSemantic added in v1.5.0

type LogicalLLMSemantic struct {
	LogicalPlanSingle
}

func NewLogicalLLMSemantic added in v1.5.0

func NewLogicalLLMSemantic(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalLLMSemantic

func (*LogicalLLMSemantic) Clone added in v1.5.0

func (*LogicalLLMSemantic) DeriveOperations added in v1.5.0

func (p *LogicalLLMSemantic) DeriveOperations()

func (*LogicalLLMSemantic) Digest added in v1.5.0

func (p *LogicalLLMSemantic) Digest() string

func (*LogicalLLMSemantic) Explain added in v1.5.0

func (p *LogicalLLMSemantic) Explain(writer LogicalPlanWriter)

func (*LogicalLLMSemantic) LogicPlanType added in v1.5.0

func (p *LogicalLLMSemantic) LogicPlanType() internal.LogicPlanType

func (*LogicalLLMSemantic) New added in v1.5.0

impl me

func (*LogicalLLMSemantic) String added in v1.5.0

func (p *LogicalLLMSemantic) String() string

func (*LogicalLLMSemantic) Type added in v1.5.0

func (p *LogicalLLMSemantic) Type() string

type LogicalLimit

type LogicalLimit struct {
	LimitPara LimitTransformParameters
	LogicalPlanSingle
}

func NewLogicalLimit

func NewLogicalLimit(input hybridqp.QueryNode, schema hybridqp.Catalog, parameters LimitTransformParameters) *LogicalLimit

func (*LogicalLimit) Clone

func (p *LogicalLimit) Clone() hybridqp.QueryNode

func (*LogicalLimit) DeriveOperations

func (p *LogicalLimit) DeriveOperations()

func (*LogicalLimit) Digest

func (p *LogicalLimit) Digest() string

func (*LogicalLimit) Explain

func (p *LogicalLimit) Explain(writer LogicalPlanWriter)

func (*LogicalLimit) ExplainIterms added in v1.2.0

func (p *LogicalLimit) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalLimit) LogicPlanType added in v1.0.0

func (p *LogicalLimit) LogicPlanType() internal.LogicPlanType

func (*LogicalLimit) New added in v1.1.0

func (p *LogicalLimit) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalLimit) String

func (p *LogicalLimit) String() string

func (*LogicalLimit) Type

func (p *LogicalLimit) Type() string

type LogicalMerge

type LogicalMerge struct {
	LogicalPlanMulti
}

func NewLogicalMerge

func NewLogicalMerge(inputs []hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalMerge

func (*LogicalMerge) Clone

func (p *LogicalMerge) Clone() hybridqp.QueryNode

func (*LogicalMerge) DeriveOperations

func (p *LogicalMerge) DeriveOperations()

func (*LogicalMerge) Digest

func (p *LogicalMerge) Digest() string

func (*LogicalMerge) Explain

func (p *LogicalMerge) Explain(writer LogicalPlanWriter)

func (*LogicalMerge) LogicPlanType added in v1.0.0

func (p *LogicalMerge) LogicPlanType() internal.LogicPlanType

func (*LogicalMerge) New added in v1.1.0

func (p *LogicalMerge) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalMerge) String

func (p *LogicalMerge) String() string

func (*LogicalMerge) Type

func (p *LogicalMerge) Type() string

type LogicalMst

type LogicalMst struct {
	LogicalPlanBase
}

func NewLogicalMst

func NewLogicalMst(rt hybridqp.RowDataType) *LogicalMst

func (*LogicalMst) Children

func (p *LogicalMst) Children() []hybridqp.QueryNode

func (*LogicalMst) Clone

func (p *LogicalMst) Clone() hybridqp.QueryNode

func (*LogicalMst) Digest

func (p *LogicalMst) Digest() string

func (*LogicalMst) Dummy

func (p *LogicalMst) Dummy() bool

func (*LogicalMst) Explain

func (p *LogicalMst) Explain(writer LogicalPlanWriter)

func (*LogicalMst) LogicPlanType added in v1.0.0

func (p *LogicalMst) LogicPlanType() internal.LogicPlanType

func (*LogicalMst) New added in v1.1.0

func (p *LogicalMst) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalMst) ReplaceChild

func (p *LogicalMst) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalMst) ReplaceChildren

func (p *LogicalMst) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalMst) String

func (p *LogicalMst) String() string

func (*LogicalMst) Type

func (p *LogicalMst) Type() string

type LogicalOrderBy

type LogicalOrderBy struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalOrderBy

func NewLogicalOrderBy(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalOrderBy

func (*LogicalOrderBy) Clone

func (p *LogicalOrderBy) Clone() hybridqp.QueryNode

func (*LogicalOrderBy) DeriveOperations

func (p *LogicalOrderBy) DeriveOperations()

func (*LogicalOrderBy) Digest

func (p *LogicalOrderBy) Digest() string

func (*LogicalOrderBy) Explain

func (p *LogicalOrderBy) Explain(writer LogicalPlanWriter)

func (*LogicalOrderBy) ExplainIterms

func (p *LogicalOrderBy) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalOrderBy) LogicPlanType added in v1.0.0

func (p *LogicalOrderBy) LogicPlanType() internal.LogicPlanType

func (*LogicalOrderBy) New added in v1.1.0

func (p *LogicalOrderBy) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalOrderBy) String

func (p *LogicalOrderBy) String() string

func (*LogicalOrderBy) Type

func (p *LogicalOrderBy) Type() string

type LogicalPlan

type LogicalPlan interface {
	hybridqp.QueryNode
	Explain(LogicalPlanWriter)
	Schema() hybridqp.Catalog
}

type LogicalPlanBase

type LogicalPlanBase struct {
	// contains filtered or unexported fields
}

func NewLogicalPlanBase

func NewLogicalPlanBase(schema hybridqp.Catalog, rt hybridqp.RowDataType, ops []hybridqp.ExprOptions) *LogicalPlanBase

func (*LogicalPlanBase) ApplyTrait

func (p *LogicalPlanBase) ApplyTrait(trait hybridqp.Trait)

func (*LogicalPlanBase) Children added in v1.0.0

func (p *LogicalPlanBase) Children() []hybridqp.QueryNode

func (*LogicalPlanBase) DeriveOperations

func (p *LogicalPlanBase) DeriveOperations()

func (*LogicalPlanBase) Dummy

func (p *LogicalPlanBase) Dummy() bool

func (*LogicalPlanBase) ExplainIterms

func (p *LogicalPlanBase) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalPlanBase) ForwardInit

func (p *LogicalPlanBase) ForwardInit(input hybridqp.QueryNode)

func (*LogicalPlanBase) ID

func (p *LogicalPlanBase) ID() uint64

func (*LogicalPlanBase) InitRef added in v1.1.0

func (p *LogicalPlanBase) InitRef(input hybridqp.QueryNode)

func (*LogicalPlanBase) PreAggInit added in v1.0.0

func (p *LogicalPlanBase) PreAggInit()

func (*LogicalPlanBase) RowDataType

func (p *LogicalPlanBase) RowDataType() hybridqp.RowDataType

func (*LogicalPlanBase) RowExprOptions

func (p *LogicalPlanBase) RowExprOptions() []hybridqp.ExprOptions

func (*LogicalPlanBase) Schema

func (p *LogicalPlanBase) Schema() hybridqp.Catalog

func (*LogicalPlanBase) SetInputs

func (p *LogicalPlanBase) SetInputs(inputs []hybridqp.QueryNode)

func (*LogicalPlanBase) SetSchema

func (p *LogicalPlanBase) SetSchema(schema hybridqp.Catalog)

func (*LogicalPlanBase) Trait

func (p *LogicalPlanBase) Trait() hybridqp.Trait

type LogicalPlanBuilder

type LogicalPlanBuilder interface {
	Build() (hybridqp.QueryNode, error)
	Push(hybridqp.QueryNode) LogicalPlanBuilder
	Aggregate() LogicalPlanBuilder
	SlidingWindow() LogicalPlanBuilder
	CountDistinct() LogicalPlanBuilder
	Limit(parameters LimitTransformParameters) LogicalPlanBuilder
	Filter() LogicalPlanBuilder
	Merge() LogicalPlanBuilder
	Distinct() LogicalPlanBuilder
	Interval() LogicalPlanBuilder
	IndexScan() LogicalPlanBuilder
	FilterBlank() LogicalPlanBuilder
	Fill() LogicalPlanBuilder
	Reader(config.EngineType) LogicalPlanBuilder
	GroupBy() LogicalPlanBuilder
	OrderBy() LogicalPlanBuilder
	SubQuery() LogicalPlanBuilder
	TagSubset() LogicalPlanBuilder
	Project() LogicalPlanBuilder
	HttpSender() LogicalPlanBuilder
	SplitGroup() LogicalPlanBuilder
	Exchange(eType ExchangeType, eTraits []hybridqp.Trait) LogicalPlanBuilder
	CreateSeriesPlan() (hybridqp.QueryNode, error)
	CreateSegmentPlan(schema hybridqp.Catalog) (hybridqp.QueryNode, error)
	CreateMeasurementPlan(hybridqp.QueryNode) (hybridqp.QueryNode, error)
	CreateShardPlan(hybridqp.QueryNode) (hybridqp.QueryNode, error)
	CreateNodePlan(hybridqp.QueryNode, []hybridqp.Trait) (hybridqp.QueryNode, error)
	CreateLimit(hybridqp.QueryNode) (hybridqp.QueryNode, error)
	CreateAggregate(hybridqp.QueryNode) (hybridqp.QueryNode, error)
	CreateDistinct(hybridqp.QueryNode) (hybridqp.QueryNode, error)
}

type LogicalPlanBuilderImpl

type LogicalPlanBuilderImpl struct {
	// contains filtered or unexported fields
}

func NewLogicalPlanBuilderImpl

func NewLogicalPlanBuilderImpl(schema hybridqp.Catalog) *LogicalPlanBuilderImpl

func (*LogicalPlanBuilderImpl) Aggregate

func (*LogicalPlanBuilderImpl) Align added in v1.5.0

func (*LogicalPlanBuilderImpl) Build

func (*LogicalPlanBuilderImpl) CountDistinct

func (b *LogicalPlanBuilderImpl) CountDistinct() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) CreateAggregate

func (b *LogicalPlanBuilderImpl) CreateAggregate(input hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateColStoreCursorPlan added in v1.2.0

func (b *LogicalPlanBuilderImpl) CreateColStoreCursorPlan() (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateColStoreReaderPlan added in v1.2.0

func (b *LogicalPlanBuilderImpl) CreateColStoreReaderPlan(seriesPlan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateDistinct added in v1.5.0

func (b *LogicalPlanBuilderImpl) CreateDistinct(input hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateInterval

func (b *LogicalPlanBuilderImpl) CreateInterval(input hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateLimit

func (*LogicalPlanBuilderImpl) CreateMeasurementPlan

func (b *LogicalPlanBuilderImpl) CreateMeasurementPlan(seriesPlan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateNodePlan

func (b *LogicalPlanBuilderImpl) CreateNodePlan(shardPlan hybridqp.QueryNode, eTraits []hybridqp.Trait) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreatePartitionPlan added in v1.2.0

func (b *LogicalPlanBuilderImpl) CreatePartitionPlan(scanPlan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateProject

func (b *LogicalPlanBuilderImpl) CreateProject(input hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateScanPlan

func (b *LogicalPlanBuilderImpl) CreateScanPlan(mstPlan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateSegmentPlan added in v1.1.0

func (b *LogicalPlanBuilderImpl) CreateSegmentPlan(schema hybridqp.Catalog) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateSeriesPlan

func (b *LogicalPlanBuilderImpl) CreateSeriesPlan() (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateShardPlan

func (b *LogicalPlanBuilderImpl) CreateShardPlan(scanPlan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateSlideWindow

func (b *LogicalPlanBuilderImpl) CreateSlideWindow(input hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) CreateSparseIndexScanPlan added in v1.1.0

func (b *LogicalPlanBuilderImpl) CreateSparseIndexScanPlan(plan hybridqp.QueryNode) (hybridqp.QueryNode, error)

func (*LogicalPlanBuilderImpl) Distinct added in v1.5.0

func (*LogicalPlanBuilderImpl) Exchange

func (*LogicalPlanBuilderImpl) Fill

func (*LogicalPlanBuilderImpl) Filter

func (*LogicalPlanBuilderImpl) FilterBlank

func (b *LogicalPlanBuilderImpl) FilterBlank() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) GroupBy

func (*LogicalPlanBuilderImpl) HashAgg added in v1.3.0

only use for prom for build hashAggNode

func (*LogicalPlanBuilderImpl) HashMerge added in v1.1.0

func (b *LogicalPlanBuilderImpl) HashMerge(eType ExchangeType, eTraits []hybridqp.Trait) LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) HoltWinters added in v1.0.0

func (b *LogicalPlanBuilderImpl) HoltWinters() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) HttpSender

func (*LogicalPlanBuilderImpl) HttpSenderHint

func (b *LogicalPlanBuilderImpl) HttpSenderHint() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) IncAgg added in v1.2.0

func (*LogicalPlanBuilderImpl) IncHashAgg added in v1.2.0

func (*LogicalPlanBuilderImpl) IndexScan

func (*LogicalPlanBuilderImpl) Interval

func (*LogicalPlanBuilderImpl) LLMSemantic added in v1.5.0

func (b *LogicalPlanBuilderImpl) LLMSemantic() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) Limit

func (*LogicalPlanBuilderImpl) Merge

func (*LogicalPlanBuilderImpl) OrderBy

func (*LogicalPlanBuilderImpl) Project

func (*LogicalPlanBuilderImpl) PromSort added in v1.4.0

func (*LogicalPlanBuilderImpl) PromSubquery added in v1.3.0

func (*LogicalPlanBuilderImpl) Push

func (*LogicalPlanBuilderImpl) Reader

func (*LogicalPlanBuilderImpl) Rewrite

func (*LogicalPlanBuilderImpl) Series

func (*LogicalPlanBuilderImpl) SlidingWindow

func (b *LogicalPlanBuilderImpl) SlidingWindow() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) Sort added in v1.1.0

func (*LogicalPlanBuilderImpl) SparseIndexScan added in v1.1.0

func (b *LogicalPlanBuilderImpl) SparseIndexScan() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) SplitGroup added in v0.2.0

func (*LogicalPlanBuilderImpl) SubQuery

func (*LogicalPlanBuilderImpl) TagSetAggregate

func (b *LogicalPlanBuilderImpl) TagSetAggregate() LogicalPlanBuilder

func (*LogicalPlanBuilderImpl) TagSubset

func (*LogicalPlanBuilderImpl) Target added in v1.0.0

type LogicalPlanMulti added in v1.0.0

type LogicalPlanMulti struct {
	LogicalPlanBase
}

func (*LogicalPlanMulti) ReplaceChild added in v1.0.0

func (p *LogicalPlanMulti) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalPlanMulti) ReplaceChildren added in v1.0.0

func (p *LogicalPlanMulti) ReplaceChildren(children []hybridqp.QueryNode)

type LogicalPlanRewriter

type LogicalPlanRewriter interface {
	// contains filtered or unexported methods
}

type LogicalPlanRewriterImpl

type LogicalPlanRewriterImpl struct {
}

type LogicalPlanSingle added in v1.0.0

type LogicalPlanSingle struct {
	LogicalPlanBase
}

func NewLogicalPlanSingle added in v1.0.0

func NewLogicalPlanSingle(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalPlanSingle

func (*LogicalPlanSingle) ReplaceChild added in v1.0.0

func (p *LogicalPlanSingle) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalPlanSingle) ReplaceChildren added in v1.0.0

func (p *LogicalPlanSingle) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalPlanSingle) SetHoltWintersType added in v1.1.0

func (p *LogicalPlanSingle) SetHoltWintersType(setOps bool, fields influxql.Fields)

type LogicalPlanVisitor

type LogicalPlanVisitor interface {
	Visit(hybridqp.QueryNode) LogicalPlanVisitor
}

type LogicalPlanWriter

type LogicalPlanWriter interface {
	Explain(LogicalPlan)
	Item(string, interface{})
	String() string
}

type LogicalPlanWriterImpl

type LogicalPlanWriterImpl struct {
	Builder *strings.Builder
	Values  *list.List
	Spacer  *Spacer
}

func NewLogicalPlanWriterImpl

func NewLogicalPlanWriterImpl(builder *strings.Builder) *LogicalPlanWriterImpl

func (*LogicalPlanWriterImpl) Explain

func (w *LogicalPlanWriterImpl) Explain(node LogicalPlan)

func (*LogicalPlanWriterImpl) ExplainChildren

func (w *LogicalPlanWriterImpl) ExplainChildren(children []hybridqp.QueryNode)

func (*LogicalPlanWriterImpl) Item

func (w *LogicalPlanWriterImpl) Item(term string, value interface{})

func (*LogicalPlanWriterImpl) String

func (w *LogicalPlanWriterImpl) String() string

type LogicalPlaner added in v1.0.0

type LogicalPlaner interface {
	RowDataType() hybridqp.RowDataType
	RowExprOptions() []hybridqp.ExprOptions
	SetInputs(inputs []hybridqp.QueryNode)
	LogicPlanType() internal.LogicPlanType
}

type LogicalProject

type LogicalProject struct {
	LogicalPlanSingle
}

func NewLogicalProject

func NewLogicalProject(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalProject

func (*LogicalProject) Clone

func (p *LogicalProject) Clone() hybridqp.QueryNode

func (*LogicalProject) DeriveOperations

func (p *LogicalProject) DeriveOperations()

func (*LogicalProject) Digest

func (p *LogicalProject) Digest() string

func (*LogicalProject) Explain

func (p *LogicalProject) Explain(writer LogicalPlanWriter)

func (*LogicalProject) LogicPlanType added in v1.0.0

func (p *LogicalProject) LogicPlanType() internal.LogicPlanType

func (*LogicalProject) New added in v1.1.0

func (p *LogicalProject) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalProject) String

func (p *LogicalProject) String() string

func (*LogicalProject) Type

func (p *LogicalProject) Type() string

type LogicalPromSort added in v1.4.0

type LogicalPromSort struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalPromSort added in v1.4.0

func NewLogicalPromSort(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalPromSort

func (*LogicalPromSort) Clone added in v1.4.0

func (p *LogicalPromSort) Clone() hybridqp.QueryNode

func (*LogicalPromSort) DeriveOperations added in v1.4.0

func (p *LogicalPromSort) DeriveOperations()

func (*LogicalPromSort) Digest added in v1.4.0

func (p *LogicalPromSort) Digest() string

func (*LogicalPromSort) Explain added in v1.4.0

func (p *LogicalPromSort) Explain(writer LogicalPlanWriter)

func (*LogicalPromSort) ExplainIterms added in v1.4.0

func (p *LogicalPromSort) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalPromSort) New added in v1.4.0

func (p *LogicalPromSort) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalPromSort) String added in v1.4.0

func (p *LogicalPromSort) String() string

func (*LogicalPromSort) Type added in v1.4.0

func (p *LogicalPromSort) Type() string

type LogicalPromSubquery added in v1.3.0

type LogicalPromSubquery struct {
	Call *influxql.PromSubCall
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalPromSubquery added in v1.3.0

func NewLogicalPromSubquery(input hybridqp.QueryNode, schema hybridqp.Catalog, Call *influxql.PromSubCall) *LogicalPromSubquery

func (*LogicalPromSubquery) Children added in v1.3.0

func (p *LogicalPromSubquery) Children() []hybridqp.QueryNode

func (*LogicalPromSubquery) Clone added in v1.3.0

func (*LogicalPromSubquery) DeriveOperations added in v1.3.0

func (p *LogicalPromSubquery) DeriveOperations()

func (*LogicalPromSubquery) Digest added in v1.3.0

func (p *LogicalPromSubquery) Digest() string

func (*LogicalPromSubquery) Explain added in v1.3.0

func (p *LogicalPromSubquery) Explain(writer LogicalPlanWriter)

func (*LogicalPromSubquery) LogicPlanType added in v1.3.0

func (p *LogicalPromSubquery) LogicPlanType() internal.LogicPlanType

func (*LogicalPromSubquery) New added in v1.3.0

impl me

func (*LogicalPromSubquery) ReplaceChild added in v1.3.0

func (p *LogicalPromSubquery) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalPromSubquery) ReplaceChildren added in v1.3.0

func (p *LogicalPromSubquery) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalPromSubquery) String added in v1.3.0

func (p *LogicalPromSubquery) String() string

func (*LogicalPromSubquery) Type added in v1.3.0

func (p *LogicalPromSubquery) Type() string

type LogicalReader

type LogicalReader struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalReader

func NewLogicalReader(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalReader

func (*LogicalReader) Clone

func (p *LogicalReader) Clone() hybridqp.QueryNode

func (*LogicalReader) Cursors

func (p *LogicalReader) Cursors() []interface{}

func (*LogicalReader) DeriveOperations

func (p *LogicalReader) DeriveOperations()

func (*LogicalReader) Digest

func (p *LogicalReader) Digest() string

func (*LogicalReader) Explain

func (p *LogicalReader) Explain(writer LogicalPlanWriter)

func (*LogicalReader) ExplainIterms

func (p *LogicalReader) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalReader) GetOneReaderState added in v1.1.0

func (p *LogicalReader) GetOneReaderState() bool

func (*LogicalReader) HasPreAgg

func (p *LogicalReader) HasPreAgg() bool

func (*LogicalReader) LogicPlanType added in v1.0.0

func (p *LogicalReader) LogicPlanType() internal.LogicPlanType

func (*LogicalReader) MstName

func (p *LogicalReader) MstName() string

func (*LogicalReader) New added in v1.1.0

func (p *LogicalReader) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalReader) SetCursor

func (p *LogicalReader) SetCursor(cursor []interface{})

func (*LogicalReader) SetOneReaderState added in v1.1.0

func (p *LogicalReader) SetOneReaderState(state bool)

func (*LogicalReader) String

func (p *LogicalReader) String() string

func (*LogicalReader) Type

func (p *LogicalReader) Type() string

type LogicalSequenceAggregate added in v1.0.0

type LogicalSequenceAggregate struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalSequenceAggregate added in v1.0.0

func NewLogicalSequenceAggregate(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSequenceAggregate

func (*LogicalSequenceAggregate) Clone added in v1.0.0

func (*LogicalSequenceAggregate) DeriveOperations added in v1.0.0

func (p *LogicalSequenceAggregate) DeriveOperations()

func (*LogicalSequenceAggregate) Digest added in v1.0.0

func (p *LogicalSequenceAggregate) Digest() string

func (*LogicalSequenceAggregate) Explain added in v1.0.0

func (p *LogicalSequenceAggregate) Explain(writer LogicalPlanWriter)

func (*LogicalSequenceAggregate) LogicPlanType added in v1.0.0

func (p *LogicalSequenceAggregate) LogicPlanType() internal.LogicPlanType

func (*LogicalSequenceAggregate) New added in v1.1.0

impl me

func (*LogicalSequenceAggregate) String added in v1.0.0

func (p *LogicalSequenceAggregate) String() string

func (*LogicalSequenceAggregate) Type added in v1.0.0

func (p *LogicalSequenceAggregate) Type() string

type LogicalSeries

type LogicalSeries struct {
	LogicalPlanBase
	// contains filtered or unexported fields
}

func NewLogicalSeries

func NewLogicalSeries(schema hybridqp.Catalog) *LogicalSeries

func (*LogicalSeries) Children

func (p *LogicalSeries) Children() []hybridqp.QueryNode

func (*LogicalSeries) Clone

func (p *LogicalSeries) Clone() hybridqp.QueryNode

func (*LogicalSeries) DeriveOperations

func (p *LogicalSeries) DeriveOperations()

func (*LogicalSeries) Digest

func (p *LogicalSeries) Digest() string

func (*LogicalSeries) Explain

func (p *LogicalSeries) Explain(writer LogicalPlanWriter)

func (*LogicalSeries) ExplainIterms

func (p *LogicalSeries) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalSeries) LogicPlanType added in v1.0.0

func (p *LogicalSeries) LogicPlanType() internal.LogicPlanType

func (*LogicalSeries) New added in v1.1.0

func (p *LogicalSeries) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

func (*LogicalSeries) ReplaceChild

func (p *LogicalSeries) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalSeries) ReplaceChildren

func (p *LogicalSeries) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalSeries) String

func (p *LogicalSeries) String() string

func (*LogicalSeries) Type

func (p *LogicalSeries) Type() string

type LogicalSlidingWindow

type LogicalSlidingWindow struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalSlidingWindow

func NewLogicalSlidingWindow(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSlidingWindow

func (*LogicalSlidingWindow) Clone

func (*LogicalSlidingWindow) CountToSum

func (p *LogicalSlidingWindow) CountToSum()

func (*LogicalSlidingWindow) DeriveOperations

func (p *LogicalSlidingWindow) DeriveOperations()

func (*LogicalSlidingWindow) Digest

func (p *LogicalSlidingWindow) Digest() string

func (*LogicalSlidingWindow) Explain

func (p *LogicalSlidingWindow) Explain(writer LogicalPlanWriter)

func (*LogicalSlidingWindow) ForwardCallArgs

func (p *LogicalSlidingWindow) ForwardCallArgs()

func (*LogicalSlidingWindow) LogicPlanType added in v1.0.0

func (p *LogicalSlidingWindow) LogicPlanType() internal.LogicPlanType

func (*LogicalSlidingWindow) New added in v1.1.0

impl me

func (*LogicalSlidingWindow) String

func (p *LogicalSlidingWindow) String() string

func (*LogicalSlidingWindow) Type

func (p *LogicalSlidingWindow) Type() string

type LogicalSort added in v1.1.0

type LogicalSort struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalSort added in v1.1.0

func NewLogicalSort(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSort

func (*LogicalSort) Clone added in v1.1.0

func (p *LogicalSort) Clone() hybridqp.QueryNode

func (*LogicalSort) DeriveOperations added in v1.1.0

func (p *LogicalSort) DeriveOperations()

func (*LogicalSort) Digest added in v1.1.0

func (p *LogicalSort) Digest() string

func (*LogicalSort) Explain added in v1.1.0

func (p *LogicalSort) Explain(writer LogicalPlanWriter)

func (*LogicalSort) ExplainIterms added in v1.1.0

func (p *LogicalSort) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalSort) LogicPlanType added in v1.1.0

func (p *LogicalSort) LogicPlanType() internal.LogicPlanType

func (*LogicalSort) New added in v1.1.0

func (p *LogicalSort) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalSort) String added in v1.1.0

func (p *LogicalSort) String() string

func (*LogicalSort) Type added in v1.1.0

func (p *LogicalSort) Type() string

type LogicalSortAppend

type LogicalSortAppend struct {
	LogicalPlanMulti
}

func NewLogicalSortAppend

func NewLogicalSortAppend(inputs []hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSortAppend

func (*LogicalSortAppend) Clone

func (*LogicalSortAppend) DeriveOperations

func (p *LogicalSortAppend) DeriveOperations()

func (*LogicalSortAppend) Digest

func (p *LogicalSortAppend) Digest() string

func (*LogicalSortAppend) Explain

func (p *LogicalSortAppend) Explain(writer LogicalPlanWriter)

func (*LogicalSortAppend) LogicPlanType added in v1.0.0

func (p *LogicalSortAppend) LogicPlanType() internal.LogicPlanType

func (*LogicalSortAppend) New added in v1.1.0

impl me

func (*LogicalSortAppend) String

func (p *LogicalSortAppend) String() string

func (*LogicalSortAppend) Type

func (p *LogicalSortAppend) Type() string

type LogicalSortMerge

type LogicalSortMerge struct {
	LogicalPlanMulti
}

func NewLogicalSortMerge

func NewLogicalSortMerge(inputs []hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSortMerge

func (*LogicalSortMerge) Clone

func (p *LogicalSortMerge) Clone() hybridqp.QueryNode

func (*LogicalSortMerge) DeriveOperations

func (p *LogicalSortMerge) DeriveOperations()

func (*LogicalSortMerge) Digest

func (p *LogicalSortMerge) Digest() string

func (*LogicalSortMerge) Explain

func (p *LogicalSortMerge) Explain(writer LogicalPlanWriter)

func (*LogicalSortMerge) LogicPlanType added in v1.0.0

func (p *LogicalSortMerge) LogicPlanType() internal.LogicPlanType

func (*LogicalSortMerge) New added in v1.1.0

func (p *LogicalSortMerge) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalSortMerge) String

func (p *LogicalSortMerge) String() string

func (*LogicalSortMerge) Type

func (p *LogicalSortMerge) Type() string

type LogicalSparseIndexScan added in v1.1.0

type LogicalSparseIndexScan struct {
	LogicalPlanSingle
}

func NewLogicalSparseIndexScan added in v1.1.0

func NewLogicalSparseIndexScan(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSparseIndexScan

func (*LogicalSparseIndexScan) Clone added in v1.1.0

func (*LogicalSparseIndexScan) DeriveOperations added in v1.1.0

func (p *LogicalSparseIndexScan) DeriveOperations()

func (*LogicalSparseIndexScan) Digest added in v1.1.0

func (p *LogicalSparseIndexScan) Digest() string

func (*LogicalSparseIndexScan) Explain added in v1.1.0

func (p *LogicalSparseIndexScan) Explain(writer LogicalPlanWriter)

func (*LogicalSparseIndexScan) LogicPlanType added in v1.1.0

func (p *LogicalSparseIndexScan) LogicPlanType() internal.LogicPlanType

func (*LogicalSparseIndexScan) New added in v1.1.0

impl me

func (*LogicalSparseIndexScan) String added in v1.1.0

func (p *LogicalSparseIndexScan) String() string

func (*LogicalSparseIndexScan) Type added in v1.1.0

func (p *LogicalSparseIndexScan) Type() string

type LogicalSplitGroup added in v0.2.0

type LogicalSplitGroup struct {
	LogicalPlanSingle
}

func NewLogicalSplitGroup added in v0.2.0

func NewLogicalSplitGroup(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSplitGroup

func (*LogicalSplitGroup) Clone added in v0.2.0

func (*LogicalSplitGroup) DeriveOperations added in v0.2.0

func (p *LogicalSplitGroup) DeriveOperations()

func (*LogicalSplitGroup) Digest added in v0.2.0

func (p *LogicalSplitGroup) Digest() string

func (*LogicalSplitGroup) Explain added in v0.2.0

func (p *LogicalSplitGroup) Explain(writer LogicalPlanWriter)

func (*LogicalSplitGroup) LogicPlanType added in v1.0.0

func (p *LogicalSplitGroup) LogicPlanType() internal.LogicPlanType

func (*LogicalSplitGroup) New added in v1.1.0

impl me

func (*LogicalSplitGroup) String added in v0.2.0

func (p *LogicalSplitGroup) String() string

func (*LogicalSplitGroup) Type added in v0.2.0

func (p *LogicalSplitGroup) Type() string

type LogicalSubQuery

type LogicalSubQuery struct {
	LogicalPlanSingle
}

func NewLogicalSubQuery

func NewLogicalSubQuery(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalSubQuery

func (*LogicalSubQuery) Clone

func (p *LogicalSubQuery) Clone() hybridqp.QueryNode

func (*LogicalSubQuery) DeriveOperations

func (p *LogicalSubQuery) DeriveOperations()

func (*LogicalSubQuery) Digest

func (p *LogicalSubQuery) Digest() string

func (*LogicalSubQuery) Explain

func (p *LogicalSubQuery) Explain(writer LogicalPlanWriter)

func (*LogicalSubQuery) LogicPlanType added in v1.0.0

func (p *LogicalSubQuery) LogicPlanType() internal.LogicPlanType

func (*LogicalSubQuery) New added in v1.1.0

func (p *LogicalSubQuery) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalSubQuery) String

func (p *LogicalSubQuery) String() string

func (*LogicalSubQuery) Type

func (p *LogicalSubQuery) Type() string

type LogicalTSSPScan added in v1.0.0

type LogicalTSSPScan struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalTSSPScan added in v1.0.0

func NewLogicalTSSPScan(schema hybridqp.Catalog) *LogicalTSSPScan

func (*LogicalTSSPScan) Children added in v1.0.0

func (p *LogicalTSSPScan) Children() []hybridqp.QueryNode

func (*LogicalTSSPScan) Clone added in v1.0.0

func (p *LogicalTSSPScan) Clone() hybridqp.QueryNode

func (*LogicalTSSPScan) DeriveOperations added in v1.0.0

func (p *LogicalTSSPScan) DeriveOperations()

func (*LogicalTSSPScan) Digest added in v1.0.0

func (p *LogicalTSSPScan) Digest() string

func (*LogicalTSSPScan) Explain added in v1.0.0

func (p *LogicalTSSPScan) Explain(writer LogicalPlanWriter)

func (*LogicalTSSPScan) ExplainIterms added in v1.0.0

func (p *LogicalTSSPScan) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalTSSPScan) GetFiles added in v1.0.0

func (p *LogicalTSSPScan) GetFiles() *immutable.TSSPFiles

func (*LogicalTSSPScan) GetNewSeqs added in v1.0.0

func (p *LogicalTSSPScan) GetNewSeqs() []uint64

func (*LogicalTSSPScan) LogicPlanType added in v1.0.0

func (p *LogicalTSSPScan) LogicPlanType() internal.LogicPlanType

func (*LogicalTSSPScan) MstName added in v1.0.0

func (p *LogicalTSSPScan) MstName() string

func (*LogicalTSSPScan) New added in v1.1.0

func (p *LogicalTSSPScan) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalTSSPScan) ReplaceChild added in v1.0.0

func (p *LogicalTSSPScan) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalTSSPScan) ReplaceChildren added in v1.0.0

func (p *LogicalTSSPScan) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalTSSPScan) SetFiles added in v1.0.0

func (p *LogicalTSSPScan) SetFiles(files *immutable.TSSPFiles)

func (*LogicalTSSPScan) SetNewSeqs added in v1.0.0

func (p *LogicalTSSPScan) SetNewSeqs(seqs []uint64)

func (*LogicalTSSPScan) String added in v1.0.0

func (p *LogicalTSSPScan) String() string

func (*LogicalTSSPScan) Type added in v1.0.0

func (p *LogicalTSSPScan) Type() string

type LogicalTableFunction added in v1.5.0

type LogicalTableFunction struct {
	LogicalPlanMulti
	TableFunctionName string
	// contains filtered or unexported fields
}

func NewLogicalTableFunction added in v1.5.0

func NewLogicalTableFunction(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, tableFunctionName string, tableFunctionParam string) *LogicalTableFunction

func (*LogicalTableFunction) Clone added in v1.5.0

func (*LogicalTableFunction) DeriveOperations added in v1.5.0

func (p *LogicalTableFunction) DeriveOperations()

func (*LogicalTableFunction) Digest added in v1.5.0

func (p *LogicalTableFunction) Digest() string

func (*LogicalTableFunction) Explain added in v1.5.0

func (p *LogicalTableFunction) Explain(writer LogicalPlanWriter)

func (*LogicalTableFunction) LogicPlanType added in v1.5.0

func (p *LogicalTableFunction) LogicPlanType() internal.LogicPlanType

func (*LogicalTableFunction) New added in v1.5.0

func (*LogicalTableFunction) String added in v1.5.0

func (p *LogicalTableFunction) String() string

func (*LogicalTableFunction) Type added in v1.5.0

func (p *LogicalTableFunction) Type() string

type LogicalTagSubset

type LogicalTagSubset struct {
	LogicalPlanSingle
}

func NewLogicalTagSubset

func NewLogicalTagSubset(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalTagSubset

NewLogicalTagSubset unused

func (*LogicalTagSubset) Clone

func (p *LogicalTagSubset) Clone() hybridqp.QueryNode

func (*LogicalTagSubset) DeriveOperations added in v1.0.0

func (p *LogicalTagSubset) DeriveOperations()

func (*LogicalTagSubset) Digest

func (p *LogicalTagSubset) Digest() string

func (*LogicalTagSubset) Explain

func (p *LogicalTagSubset) Explain(writer LogicalPlanWriter)

func (*LogicalTagSubset) LogicPlanType added in v1.0.0

func (p *LogicalTagSubset) LogicPlanType() internal.LogicPlanType

func (*LogicalTagSubset) New added in v1.1.0

func (p *LogicalTagSubset) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalTagSubset) String

func (p *LogicalTagSubset) String() string

func (*LogicalTagSubset) Type

func (p *LogicalTagSubset) Type() string

type LogicalTarget added in v1.0.0

type LogicalTarget struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalTarget added in v1.0.0

func NewLogicalTarget(input hybridqp.QueryNode, schema hybridqp.Catalog, mst *influxql.Measurement) *LogicalTarget

func (*LogicalTarget) Clone added in v1.0.0

func (p *LogicalTarget) Clone() hybridqp.QueryNode

func (*LogicalTarget) DeriveOperations added in v1.0.0

func (p *LogicalTarget) DeriveOperations()

func (*LogicalTarget) Digest added in v1.0.0

func (p *LogicalTarget) Digest() string

func (*LogicalTarget) Explain added in v1.0.0

func (p *LogicalTarget) Explain(writer LogicalPlanWriter)

func (*LogicalTarget) LogicPlanType added in v1.0.0

func (p *LogicalTarget) LogicPlanType() internal.LogicPlanType

func (*LogicalTarget) New added in v1.1.0

func (p *LogicalTarget) New(inputs []hybridqp.QueryNode, schema hybridqp.Catalog, eTrait []hybridqp.Trait) hybridqp.QueryNode

impl me

func (*LogicalTarget) String added in v1.0.0

func (p *LogicalTarget) String() string

func (*LogicalTarget) TargetMeasurement added in v1.0.0

func (p *LogicalTarget) TargetMeasurement() *influxql.Measurement

func (*LogicalTarget) Type added in v1.0.0

func (p *LogicalTarget) Type() string

type LogicalWriteIntoStorage added in v1.0.0

type LogicalWriteIntoStorage struct {
	LogicalPlanSingle
	// contains filtered or unexported fields
}

func NewLogicalWriteIntoStorage added in v1.0.0

func NewLogicalWriteIntoStorage(input hybridqp.QueryNode, schema hybridqp.Catalog) *LogicalWriteIntoStorage

func (*LogicalWriteIntoStorage) Clone added in v1.0.0

func (*LogicalWriteIntoStorage) DeriveOperations added in v1.0.0

func (p *LogicalWriteIntoStorage) DeriveOperations()

func (*LogicalWriteIntoStorage) Digest added in v1.0.0

func (p *LogicalWriteIntoStorage) Digest() string

func (*LogicalWriteIntoStorage) Explain added in v1.0.0

func (p *LogicalWriteIntoStorage) Explain(writer LogicalPlanWriter)

func (*LogicalWriteIntoStorage) ExplainIterms added in v1.0.0

func (p *LogicalWriteIntoStorage) ExplainIterms(writer LogicalPlanWriter)

func (*LogicalWriteIntoStorage) GetMmsTables added in v1.0.0

func (p *LogicalWriteIntoStorage) GetMmsTables() *immutable.MmsTables

func (*LogicalWriteIntoStorage) LogicPlanType added in v1.0.0

func (p *LogicalWriteIntoStorage) LogicPlanType() internal.LogicPlanType

func (*LogicalWriteIntoStorage) MstName added in v1.0.0

func (p *LogicalWriteIntoStorage) MstName() string

func (*LogicalWriteIntoStorage) New added in v1.1.0

impl me

func (*LogicalWriteIntoStorage) ReplaceChild added in v1.0.0

func (p *LogicalWriteIntoStorage) ReplaceChild(ordinal int, child hybridqp.QueryNode)

func (*LogicalWriteIntoStorage) ReplaceChildren added in v1.0.0

func (p *LogicalWriteIntoStorage) ReplaceChildren(children []hybridqp.QueryNode)

func (*LogicalWriteIntoStorage) SetMmsTables added in v1.0.0

func (p *LogicalWriteIntoStorage) SetMmsTables(m *immutable.MmsTables)

func (*LogicalWriteIntoStorage) String added in v1.0.0

func (p *LogicalWriteIntoStorage) String() string

func (*LogicalWriteIntoStorage) Type added in v1.0.0

func (p *LogicalWriteIntoStorage) Type() string

type MaterializeTransform

type MaterializeTransform struct {
	BaseProcessor

	ResetTime bool

	HasBinaryExpr bool

	ColumnMap [][]int
	// contains filtered or unexported fields
}

func NewMaterializeTransform

func NewMaterializeTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions, writer ChunkWriter, schema *QuerySchema) *MaterializeTransform

func (*MaterializeTransform) Close

func (trans *MaterializeTransform) Close()

func (*MaterializeTransform) ColumnMapInit

func (trans *MaterializeTransform) ColumnMapInit()

func (*MaterializeTransform) Explain

func (trans *MaterializeTransform) Explain() []ValuePair

func (*MaterializeTransform) GetInputNumber

func (trans *MaterializeTransform) GetInputNumber(_ Port) int

func (*MaterializeTransform) GetInputs

func (trans *MaterializeTransform) GetInputs() Ports

func (*MaterializeTransform) GetOutputNumber

func (trans *MaterializeTransform) GetOutputNumber(_ Port) int

func (*MaterializeTransform) GetOutputs

func (trans *MaterializeTransform) GetOutputs() Ports

func (*MaterializeTransform) Name

func (trans *MaterializeTransform) Name() string

func (*MaterializeTransform) ResetTransparents added in v1.2.0

func (trans *MaterializeTransform) ResetTransparents()

func (*MaterializeTransform) Work

func (trans *MaterializeTransform) Work(ctx context.Context) error

type MaterializeTransformCreator

type MaterializeTransformCreator struct {
}

func (*MaterializeTransformCreator) Create

type MaxOp added in v1.3.0

type MaxOp struct{}

func (*MaxOp) CreateRoutine added in v1.3.0

func (c *MaxOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type MaxPromOp added in v1.3.0

type MaxPromOp struct {
	BasePromOp
}

func (*MaxPromOp) CreateRoutine added in v1.3.0

func (c *MaxPromOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type MergeTransf added in v1.0.0

type MergeTransf struct {
}

func (*MergeTransf) CostName added in v1.0.0

func (t *MergeTransf) CostName() string

func (*MergeTransf) GetType added in v1.0.0

func (t *MergeTransf) GetType() MergeTransformType

func (*MergeTransf) InitColumnsIteratorHelper added in v1.0.0

func (t *MergeTransf) InitColumnsIteratorHelper(rt hybridqp.RowDataType) CoProcessor

func (*MergeTransf) InitHeapItems added in v1.0.0

func (t *MergeTransf) InitHeapItems(inRowDataLen int, _ hybridqp.RowDataType, schema *QuerySchema) BaseHeapItems

func (*MergeTransf) Name added in v1.0.0

func (t *MergeTransf) Name() string

type MergeTransform

type MergeTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts

	BreakPoint BaseBreakPoint

	HeapItems BaseHeapItems

	NewChunk    Chunk
	CoProcessor CoProcessor

	WaitMerge chan Semaphore
	NextChunk []chan Semaphore

	ReflectionTables
	// contains filtered or unexported fields
}

func NewBaseMergeTransform added in v1.0.0

func NewBaseMergeTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, schema *QuerySchema,
	children []hybridqp.QueryNode, mergeType MergeType) *MergeTransform

func NewMergeTransform

func NewMergeTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, _ []hybridqp.ExprOptions, schema *QuerySchema) *MergeTransform

func NewSortAppendTransform

func NewSortAppendTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, schema *QuerySchema, children []hybridqp.QueryNode) *MergeTransform

func NewSortedMergeTransform

func NewSortedMergeTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType, _ []hybridqp.ExprOptions, schema *QuerySchema) *MergeTransform

func (*MergeTransform) AddTagAndIndexes

func (trans *MergeTransform) AddTagAndIndexes(tag ChunkTags, iLen int, i int, flag bool)

func (*MergeTransform) AppendIntervalIndex added in v1.5.0

func (trans *MergeTransform) AppendIntervalIndex(chunk Chunk, i int, opt *query.ProcessorOptions, flag bool) bool

func (*MergeTransform) AppendToHeap

func (trans *MergeTransform) AppendToHeap(in int, c Chunk)

func (*MergeTransform) Close

func (trans *MergeTransform) Close()

func (*MergeTransform) CostName added in v1.0.0

func (trans *MergeTransform) CostName() string

func (*MergeTransform) Explain

func (trans *MergeTransform) Explain() []ValuePair

func (*MergeTransform) GetInputNumber

func (trans *MergeTransform) GetInputNumber(port Port) int

func (*MergeTransform) GetInputs

func (trans *MergeTransform) GetInputs() Ports

func (*MergeTransform) GetMstName added in v1.0.0

func (trans *MergeTransform) GetMstName() string

func (*MergeTransform) GetOutputNumber

func (trans *MergeTransform) GetOutputNumber(port Port) int

func (*MergeTransform) GetOutputs

func (trans *MergeTransform) GetOutputs() Ports

func (*MergeTransform) InitHeapItems added in v1.0.0

func (trans *MergeTransform) InitHeapItems(inRowDataLen int, rt hybridqp.RowDataType, schema *QuerySchema)

func (*MergeTransform) IsNewMstName added in v1.0.0

func (trans *MergeTransform) IsNewMstName() bool

func (*MergeTransform) Merge

func (trans *MergeTransform) Merge(ctx context.Context, errs *errno.Errs)

Merge used to merge chunks to a sorted chunks.

func (*MergeTransform) Name

func (trans *MergeTransform) Name() string

func (*MergeTransform) SendChunk

func (trans *MergeTransform) SendChunk()

func (*MergeTransform) UpdateWithSingleChunk

func (trans *MergeTransform) UpdateWithSingleChunk()

func (*MergeTransform) Work

func (trans *MergeTransform) Work(ctx context.Context) error

type MergeTransformCreator

type MergeTransformCreator struct {
}

func (*MergeTransformCreator) Create

type MergeTransformType added in v1.0.0

type MergeTransformType int
const (
	MergeTrans MergeTransformType = iota
	SortMergeTrans
	SortAppendTrans
)

type MergeType added in v1.0.0

type MergeType interface {
	Name() string
	CostName() string
	GetType() MergeTransformType
	InitHeapItems(inRowDataLen int, rt hybridqp.RowDataType, schema *QuerySchema) BaseHeapItems
	InitColumnsIteratorHelper(rt hybridqp.RowDataType) CoProcessor
	// contains filtered or unexported methods
}

type MetaData added in v1.5.0

type MetaData struct {
	Region    string   `json:"region"`
	Timestamp string   `json:"timestamp"`
	Topokeys  []string `json:"topokeys"`
}

type MinOp added in v1.3.0

type MinOp struct{}

func (*MinOp) CreateRoutine added in v1.3.0

func (c *MinOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type MinPromOp added in v1.3.0

type MinPromOp struct {
	BasePromOp
}

func (*MinPromOp) CreateRoutine added in v1.3.0

func (c *MinPromOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type MultiMstInfo added in v1.5.0

type MultiMstInfo struct {
	ShardIds []uint64
	Opt      query.ProcessorOptions
}

type MultiMstReqs added in v1.2.0

type MultiMstReqs struct {
	// contains filtered or unexported fields
}

func NewMultiMstReqs added in v1.2.0

func NewMultiMstReqs() *MultiMstReqs

func (*MultiMstReqs) SetReqs added in v1.4.0

func (m *MultiMstReqs) SetReqs(reqs []*RemoteQuery)

type NewAggOperator added in v1.1.0

type NewAggOperator func() aggOperator

type NilSink

type NilSink struct {
	BaseProcessor

	Input *ChunkPort
}

func NewNilSink

func NewNilSink(rowDataType hybridqp.RowDataType) *NilSink

func (*NilSink) Close

func (sink *NilSink) Close()

func (*NilSink) Explain

func (sink *NilSink) Explain() []ValuePair

func (*NilSink) GetInputNumber

func (sink *NilSink) GetInputNumber(_ Port) int

func (*NilSink) GetInputs

func (sink *NilSink) GetInputs() Ports

func (*NilSink) GetOutputNumber

func (sink *NilSink) GetOutputNumber(_ Port) int

func (*NilSink) GetOutputs

func (sink *NilSink) GetOutputs() Ports

func (*NilSink) Name

func (sink *NilSink) Name() string

func (*NilSink) Work

func (sink *NilSink) Work(ctx context.Context) error

type NilTransform

type NilTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
}

func NewNilTransform

func NewNilTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataTypes []hybridqp.RowDataType) *NilTransform

func (*NilTransform) Close

func (trans *NilTransform) Close()

func (*NilTransform) Explain

func (trans *NilTransform) Explain() []ValuePair

func (*NilTransform) GetInputNumber

func (trans *NilTransform) GetInputNumber(port Port) int

func (*NilTransform) GetInputs

func (trans *NilTransform) GetInputs() Ports

func (*NilTransform) GetOutputNumber

func (trans *NilTransform) GetOutputNumber(port Port) int

func (*NilTransform) GetOutputs

func (trans *NilTransform) GetOutputs() Ports

func (*NilTransform) Name

func (trans *NilTransform) Name() string

func (*NilTransform) Work

func (trans *NilTransform) Work(ctx context.Context) error

type NodeMetaData added in v1.5.0

type NodeMetaData struct {
	Kind   string            `json:"kind"`
	Region string            `json:"region"`
	Tags   map[string]string `json:"tags"`
}

type NodeTraitKey added in v1.5.0

type NodeTraitKey struct {
	Database string
	PtID     uint32 // for tsstore
	NodeID   uint64
}

type OGSketch added in v1.0.0

type OGSketch interface {
	InsertPoints(...float64)
	InsertClusters(...floatTuple)
	DeletePoints(...float64)
	DeleteClusters(...floatTuple)
	Merge(*OGSketchImpl)
	Percentile(float64) float64
	Rank(float64) int64
	Clusters() ClusterSet
	Reset()
	Len() int
	EquiHeightHistogram(int, float64, float64) []float64
	DemarcationHistogram(float64, float64, int, int) []int64
}

type OGSketchImpl added in v1.0.0

type OGSketchImpl struct {
	// contains filtered or unexported fields
}

func NewOGSketchImpl added in v1.0.0

func NewOGSketchImpl(c float64) *OGSketchImpl

func (*OGSketchImpl) Clusters added in v1.0.0

func (s *OGSketchImpl) Clusters() ClusterSet

func (*OGSketchImpl) DeleteClusters added in v1.0.0

func (s *OGSketchImpl) DeleteClusters(cluster ...floatTuple)

func (*OGSketchImpl) DeletePoints added in v1.0.0

func (s *OGSketchImpl) DeletePoints(m ...float64)

func (*OGSketchImpl) DemarcationHistogram added in v1.0.0

func (s *OGSketchImpl) DemarcationHistogram(begin, width float64, binsNum, binsType int) []int64

func (*OGSketchImpl) EquiHeightHistogram added in v1.0.0

func (s *OGSketchImpl) EquiHeightHistogram(binNum int, begin float64, end float64) []float64

func (*OGSketchImpl) InsertClusters added in v1.0.0

func (s *OGSketchImpl) InsertClusters(cluster ...floatTuple)

func (*OGSketchImpl) InsertPoints added in v1.0.0

func (s *OGSketchImpl) InsertPoints(m ...float64)

func (*OGSketchImpl) Len added in v1.0.0

func (s *OGSketchImpl) Len() int

func (*OGSketchImpl) Merge added in v1.0.0

func (s *OGSketchImpl) Merge(s1 *OGSketchImpl)

func (*OGSketchImpl) Percentile added in v1.0.0

func (s *OGSketchImpl) Percentile(q float64) float64

func (*OGSketchImpl) Rank added in v1.0.0

func (s *OGSketchImpl) Rank(Q float64) int64

func (*OGSketchImpl) Reset added in v1.0.0

func (s *OGSketchImpl) Reset()

type OGSketchItem added in v1.0.0

type OGSketchItem interface {
	UpdateCluster(inChunk Chunk, start, end int)
	WriteResult(outChunk Chunk, time int64)
	IsNil() bool
	Reset()
}

type OGSketchIterator added in v1.0.0

type OGSketchIterator struct {
	// contains filtered or unexported fields
}

func NewOGSketchIterator added in v1.0.0

func NewOGSketchIterator(
	isSingleCall bool, inOrdinal, outOrdinal int, clusterNum int, opt *query.ProcessorOptions, sketch OGSketchItem,
) *OGSketchIterator

func (*OGSketchIterator) Next added in v1.0.0

type OGSketchMergeItem added in v1.0.0

type OGSketchMergeItem struct {
	// contains filtered or unexported fields
}

func NewOGSketchMergeItem added in v1.0.0

func NewOGSketchMergeItem(isSingleCall bool, inOrdinal, outOrdinal, clusterNum int, percentile float64) *OGSketchMergeItem

func (*OGSketchMergeItem) IsNil added in v1.0.0

func (o *OGSketchMergeItem) IsNil() bool

func (*OGSketchMergeItem) Reset added in v1.0.0

func (o *OGSketchMergeItem) Reset()

func (*OGSketchMergeItem) UpdateCluster added in v1.0.0

func (o *OGSketchMergeItem) UpdateCluster(inChunk Chunk, start, end int)

func (*OGSketchMergeItem) WriteResult added in v1.0.0

func (o *OGSketchMergeItem) WriteResult(outChunk Chunk, time int64)

type OptRule

type OptRule interface {
	Initialize(rule OptRule, operand OptRuleOperand, description string)
	ToString() string
	Description() string
	Category() OptRuleCategory
	Equals(OptRule) bool
	GetOperand() OptRuleOperand
	Matches(call *OptRuleCall) bool
	OnMatch(call *OptRuleCall)
}

type OptRuleBase

type OptRuleBase struct {
	// contains filtered or unexported fields
}

func (*OptRuleBase) Description

func (r *OptRuleBase) Description() string

func (*OptRuleBase) Equals

func (r *OptRuleBase) Equals(rhs *OptRuleBase) bool

func (*OptRuleBase) GetOperand

func (r *OptRuleBase) GetOperand() OptRuleOperand

func (*OptRuleBase) Initialize

func (r *OptRuleBase) Initialize(rule OptRule, operand OptRuleOperand, description string)

func (*OptRuleBase) Matches

func (r *OptRuleBase) Matches(_ *OptRuleCall) bool

type OptRuleCall

type OptRuleCall struct {
	// contains filtered or unexported fields
}

func NewOptRuleCall

func NewOptRuleCall(planner HeuPlanner,
	operand OptRuleOperand,
	nodes []hybridqp.QueryNode,
) *OptRuleCall

func (*OptRuleCall) GetResult

func (c *OptRuleCall) GetResult() []hybridqp.QueryNode

func (*OptRuleCall) Node

func (c *OptRuleCall) Node(ordinal int) hybridqp.QueryNode

func (*OptRuleCall) Rule

func (c *OptRuleCall) Rule() OptRule

func (*OptRuleCall) TransformTo

func (c *OptRuleCall) TransformTo(to hybridqp.QueryNode) []hybridqp.QueryNode

type OptRuleCategory added in v1.5.0

type OptRuleCategory int
const (
	RULE_TEST OptRuleCategory = iota
	RULE_PUSHDOWN_LIMIT
	RULE_PUSHDOWN_AGG
	RULE_SPREAD_AGG
	RULE_SUBQUERY
	RULE_HEIMADLL_PUSHDOWN
	RULE_PUSHDOWN_DISTINCT
)

type OptRuleOperand

type OptRuleOperand interface {
	Equals(rhs OptRuleOperand) bool
	SetParent(parent OptRuleOperand)
	Parent() OptRuleOperand
	Matches(node hybridqp.QueryNode) bool
	Policy() OptRuleOperandChildPolicy
	SetRule(rule OptRule)
	Rule() OptRule
	SetOrdinalInParent(int)
	SetOrdinalInRule(int)
	Children() []OptRuleOperand
}

type OptRuleOperandBase

type OptRuleOperandBase struct {
	// contains filtered or unexported fields
}

func NewOptRuleOperandBase

func NewOptRuleOperandBase(planType string, policy OptRuleOperandChildPolicy, children []OptRuleOperand) *OptRuleOperandBase

func (*OptRuleOperandBase) Children

func (ob *OptRuleOperandBase) Children() []OptRuleOperand

func (*OptRuleOperandBase) Equals

func (ob *OptRuleOperandBase) Equals(rhs OptRuleOperand) bool

func (*OptRuleOperandBase) Matches

func (ob *OptRuleOperandBase) Matches(node hybridqp.QueryNode) bool

func (*OptRuleOperandBase) Parent

func (ob *OptRuleOperandBase) Parent() OptRuleOperand

func (*OptRuleOperandBase) Policy

func (*OptRuleOperandBase) Rule

func (ob *OptRuleOperandBase) Rule() OptRule

func (*OptRuleOperandBase) SetOrdinalInParent

func (ob *OptRuleOperandBase) SetOrdinalInParent(ordinal int)

func (*OptRuleOperandBase) SetOrdinalInRule

func (ob *OptRuleOperandBase) SetOrdinalInRule(ordinal int)

func (*OptRuleOperandBase) SetParent

func (ob *OptRuleOperandBase) SetParent(parent OptRuleOperand)

func (*OptRuleOperandBase) SetRule

func (ob *OptRuleOperandBase) SetRule(rule OptRule)

type OptRuleOperandBuilder

type OptRuleOperandBuilder interface {
	NoInput(planType string)
	AnyInput(planType string)
	OneInput(planType string, input OptRuleOperand)
	WildCardInput(planType string)
	Inputs(planType string, inputs ...OptRuleOperand)
	UnorderedInputs(planType string, inputs ...OptRuleOperand)
	Operands() []OptRuleOperand
	Operand() OptRuleOperand
}

type OptRuleOperandBuilderBase

type OptRuleOperandBuilderBase struct {
	// contains filtered or unexported fields
}

func NewOptRuleOperandBuilderBase

func NewOptRuleOperandBuilderBase() *OptRuleOperandBuilderBase

func (*OptRuleOperandBuilderBase) AfterInput

func (builder *OptRuleOperandBuilderBase) AfterInput(planType string)

func (*OptRuleOperandBuilderBase) AnyInput

func (builder *OptRuleOperandBuilderBase) AnyInput(planType string)

func (*OptRuleOperandBuilderBase) Inputs

func (builder *OptRuleOperandBuilderBase) Inputs(planType string, inputs ...OptRuleOperand)

func (*OptRuleOperandBuilderBase) NoInput

func (builder *OptRuleOperandBuilderBase) NoInput(planType string)

func (*OptRuleOperandBuilderBase) OneInput

func (builder *OptRuleOperandBuilderBase) OneInput(planType string, input OptRuleOperand)

func (*OptRuleOperandBuilderBase) Operand

func (builder *OptRuleOperandBuilderBase) Operand() OptRuleOperand

func (*OptRuleOperandBuilderBase) Operands

func (builder *OptRuleOperandBuilderBase) Operands() []OptRuleOperand

func (*OptRuleOperandBuilderBase) UnorderedInputs

func (builder *OptRuleOperandBuilderBase) UnorderedInputs(planType string, inputs ...OptRuleOperand)

func (*OptRuleOperandBuilderBase) WildCardInput

func (builder *OptRuleOperandBuilderBase) WildCardInput(planType string, input OptRuleOperand)

type OptRuleOperandChildPolicy

type OptRuleOperandChildPolicy int
const (
	ANY OptRuleOperandChildPolicy = iota
	LEAF
	SOME
	UNORDERED
	WILDCARD
	AFTER
)

type OptRuleOperands

type OptRuleOperands []OptRuleOperand

func (OptRuleOperands) Equals

func (operands OptRuleOperands) Equals(rhs OptRuleOperands) bool

type OrderByTransform

type OrderByTransform struct {
	BaseProcessor

	ResultChunkPool *CircularChunkPool

	CoProcessor CoProcessor
	// contains filtered or unexported fields
}

func NewOrderByTransform

func NewOrderByTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions, dimensions []string) *OrderByTransform

func (*OrderByTransform) Close

func (trans *OrderByTransform) Close()

func (*OrderByTransform) Explain

func (trans *OrderByTransform) Explain() []ValuePair

func (*OrderByTransform) GetCurrTags added in v1.3.0

func (trans *OrderByTransform) GetCurrTags(i int) string

func (*OrderByTransform) GetInputNumber

func (trans *OrderByTransform) GetInputNumber(_ Port) int

func (*OrderByTransform) GetInputs

func (trans *OrderByTransform) GetInputs() Ports

func (*OrderByTransform) GetOutputNumber

func (trans *OrderByTransform) GetOutputNumber(_ Port) int

func (*OrderByTransform) GetOutputs

func (trans *OrderByTransform) GetOutputs() Ports

func (*OrderByTransform) GetTagAndIndexes

func (trans *OrderByTransform) GetTagAndIndexes(chunk Chunk)

func (*OrderByTransform) GetTagsResetTagIndexes

func (trans *OrderByTransform) GetTagsResetTagIndexes(chunk Chunk)

func (*OrderByTransform) IntervalIndexReGen

func (trans *OrderByTransform) IntervalIndexReGen()

func (*OrderByTransform) Name

func (trans *OrderByTransform) Name() string

func (*OrderByTransform) OrderTime

func (trans *OrderByTransform) OrderTime()

func (*OrderByTransform) RebuildChunk

func (trans *OrderByTransform) RebuildChunk()

func (*OrderByTransform) Release

func (trans *OrderByTransform) Release() error

func (*OrderByTransform) SendChunk

func (trans *OrderByTransform) SendChunk()

func (*OrderByTransform) TagAndTagIndexHandler

func (trans *OrderByTransform) TagAndTagIndexHandler()

func (*OrderByTransform) Work

func (trans *OrderByTransform) Work(ctx context.Context) error

type OrderByTransformCreator

type OrderByTransformCreator struct {
}

func (*OrderByTransformCreator) Create

type OuterExecutorBuilder added in v1.5.0

type OuterExecutorBuilder interface {
	Analyze(span *tracing.Span)
	Build(node hybridqp.QueryNode) (hybridqp.Executor, error)
}

type Para added in v1.0.0

type Para struct {
	Expr   influxql.Expr
	VarRef *influxql.VarRef
}

type ParaPointers added in v1.0.0

type ParaPointers []*Para

func (ParaPointers) Len added in v1.0.0

func (a ParaPointers) Len() int

Len implements sort.Interface.

func (ParaPointers) Less added in v1.0.0

func (a ParaPointers) Less(i, j int) bool

Less implements sort.Interface.

func (ParaPointers) Swap added in v1.0.0

func (a ParaPointers) Swap(i, j int)

Swap implements sort.Interface.

type PercentileApproxOp added in v1.3.0

type PercentileApproxOp struct{}

func (*PercentileApproxOp) CreateRoutine added in v1.3.0

func (c *PercentileApproxOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PipelineExecutor

type PipelineExecutor struct {
	Query string

	RunTimeStats *statistics.StatisticTimer
	// contains filtered or unexported fields
}

func NewPipelineExecutor

func NewPipelineExecutor(processors Processors) *PipelineExecutor

func NewPipelineExecutorFromDag

func NewPipelineExecutorFromDag(dag *TransformDag, root *TransformVertex) *PipelineExecutor

func (*PipelineExecutor) Abort

func (exec *PipelineExecutor) Abort()

func (*PipelineExecutor) Aborted

func (exec *PipelineExecutor) Aborted() bool

func (*PipelineExecutor) Crash

func (exec *PipelineExecutor) Crash()

func (*PipelineExecutor) Crashed

func (exec *PipelineExecutor) Crashed() bool

func (*PipelineExecutor) Execute

func (exec *PipelineExecutor) Execute(ctx context.Context) error

func (*PipelineExecutor) ExecuteExecutor

func (exec *PipelineExecutor) ExecuteExecutor(ctx context.Context) error

func (*PipelineExecutor) GetProcessors added in v1.0.0

func (exec *PipelineExecutor) GetProcessors() Processors

func (*PipelineExecutor) GetRoot added in v1.0.0

func (exec *PipelineExecutor) GetRoot() *TransformVertex

func (*PipelineExecutor) InitContext added in v1.3.0

func (exec *PipelineExecutor) InitContext(ctx context.Context) error

func (*PipelineExecutor) NoMarkCrash added in v1.4.0

func (exec *PipelineExecutor) NoMarkCrash()

func (*PipelineExecutor) Release

func (exec *PipelineExecutor) Release()

func (*PipelineExecutor) SetDag added in v1.0.0

func (exec *PipelineExecutor) SetDag(d *TransformDag)

func (*PipelineExecutor) SetProcessors added in v1.0.0

func (exec *PipelineExecutor) SetProcessors(pro Processors)

func (*PipelineExecutor) SetRoot added in v1.0.0

func (exec *PipelineExecutor) SetRoot(r *TransformVertex)

func (*PipelineExecutor) Visit

type PlanFrame

type PlanFrame struct {
	// contains filtered or unexported fields
}

func NewPlanFrame

func NewPlanFrame(plan hybridqp.QueryNode) *PlanFrame

type PlanTemplate added in v1.1.0

type PlanTemplate struct {
	// contains filtered or unexported fields
}

func NewOneShardStorePlanTemplate added in v1.1.0

func NewOneShardStorePlanTemplate(t PlanType) *PlanTemplate

func NewSqlPlanTemplate added in v1.1.0

func NewSqlPlanTemplate(t PlanType) *PlanTemplate

func NewStorePlanTemplate added in v1.1.0

func NewStorePlanTemplate(t PlanType) *PlanTemplate

func (*PlanTemplate) GetLocalStorePlan added in v1.2.0

func (pp *PlanTemplate) GetLocalStorePlan() []hybridqp.QueryNode

func (*PlanTemplate) GetPlan added in v1.1.0

func (pp *PlanTemplate) GetPlan() []hybridqp.QueryNode

func (*PlanTemplate) NewLocalStoreSqlPlanTemplate added in v1.2.0

func (pp *PlanTemplate) NewLocalStoreSqlPlanTemplate()

type PlanType added in v1.1.0

type PlanType uint32
const (
	AGG_INTERVAL PlanType = iota
	AGG_INTERVAL_LIMIT
	NO_AGG_NO_GROUP
	AGG_GROUP
	NO_AGG_NO_GROUP_LIMIT
	AGG_INTERVAL_FILLNONE
	UNKNOWN
)

func MatchAggGroup added in v1.1.0

func MatchAggGroup(schema hybridqp.Catalog) PlanType

func MatchAggInterval added in v1.1.0

func MatchAggInterval(schema hybridqp.Catalog) PlanType

func MatchAggIntervalLimit added in v1.1.0

func MatchAggIntervalLimit(schema hybridqp.Catalog) PlanType

func MatchNoAggNoGroup added in v1.1.0

func MatchNoAggNoGroup(schema hybridqp.Catalog) PlanType

func MatchNoAggNoGroupLimit added in v1.2.0

func MatchNoAggNoGroupLimit(schema hybridqp.Catalog) PlanType

func NilGetPlanType added in v1.1.0

func NilGetPlanType(schema hybridqp.Catalog, stmt *influxql.SelectStatement) PlanType

func NormalGetPlanType added in v1.1.0

func NormalGetPlanType(schema hybridqp.Catalog, stmt *influxql.SelectStatement) PlanType

type PlanTypeInitShard added in v1.1.0

type PlanTypeInitShard struct {
	// contains filtered or unexported fields
}

func NewPlanTypeInitShardGroup added in v1.1.0

func NewPlanTypeInitShardGroup() *PlanTypeInitShard

func (*PlanTypeInitShard) CheckDatabaseExists added in v1.5.0

func (pts *PlanTypeInitShard) CheckDatabaseExists(name string) error

func (*PlanTypeInitShard) Close added in v1.1.0

func (pts *PlanTypeInitShard) Close() error

func (*PlanTypeInitShard) CreateLogicalPlan added in v1.1.0

func (pts *PlanTypeInitShard) CreateLogicalPlan(ctx context.Context, sources influxql.Sources, schema hybridqp.Catalog) (hybridqp.QueryNode, error)

func (*PlanTypeInitShard) FieldDimensions added in v1.1.0

func (pts *PlanTypeInitShard) FieldDimensions(
	m *influxql.Measurement) (map[string]influxql.DataType, map[string]struct{}, *influxql.Schema, error)

func (*PlanTypeInitShard) GetETraits added in v1.1.0

func (pts *PlanTypeInitShard) GetETraits(ctx context.Context, sources influxql.Sources, schema hybridqp.Catalog) ([]hybridqp.Trait, error)

func (*PlanTypeInitShard) GetResource added in v1.5.0

func (pts *PlanTypeInitShard) GetResource(resource string) map[string]string

func (*PlanTypeInitShard) GetSeriesKey added in v1.1.0

func (pts *PlanTypeInitShard) GetSeriesKey() []byte

func (*PlanTypeInitShard) GetSources added in v1.1.0

func (pts *PlanTypeInitShard) GetSources(sources influxql.Sources) influxql.Sources

func (*PlanTypeInitShard) GetTagKeys added in v1.5.0

func (pts *PlanTypeInitShard) GetTagKeys(stmt *influxql.ShowTagValuesStatement) (map[string]map[string]struct{}, error)

func (*PlanTypeInitShard) GetTagVals added in v1.5.0

func (pts *PlanTypeInitShard) GetTagVals(nodeID uint64, stmt *influxql.ShowTagValuesStatement, pts2 []uint32, tagKeys map[string]map[string]struct{}, exact bool) (influxql.TablesTagSets, error)

func (*PlanTypeInitShard) LogicalPlanCost added in v1.1.0

func (*PlanTypeInitShard) MapType added in v1.1.0

func (*PlanTypeInitShard) MapTypeBatch added in v1.1.0

func (pts *PlanTypeInitShard) MapTypeBatch(m *influxql.Measurement, fields map[string]*influxql.FieldNameSpace, schema *influxql.Schema) error

func (*PlanTypeInitShard) QueryNodePtsMap added in v1.5.0

func (pts *PlanTypeInitShard) QueryNodePtsMap(database string) (map[uint64][]uint32, error)

type PlanTypeInitShardMapper added in v1.1.0

type PlanTypeInitShardMapper struct {
}

func NewPlanTypeInitShardMapper added in v1.1.0

func NewPlanTypeInitShardMapper() *PlanTypeInitShardMapper

func (*PlanTypeInitShardMapper) Close added in v1.1.0

func (pts *PlanTypeInitShardMapper) Close() error

func (*PlanTypeInitShardMapper) MapShards added in v1.1.0

type Point added in v1.3.0

type Point[T util.ExceptString] struct {
	// contains filtered or unexported fields
}

func (*Point[T]) Assign added in v1.3.0

func (p *Point[T]) Assign(c *Point[T])

func (*Point[T]) Reset added in v1.3.0

func (p *Point[T]) Reset()

func (*Point[T]) Set added in v1.3.0

func (p *Point[T]) Set(index int, time int64, value T)

type PointItem added in v1.3.0

type PointItem[T util.BasicType] struct {
	// contains filtered or unexported fields
}

func NewPointItem added in v1.3.0

func NewPointItem[T util.BasicType](time int64, value T) *PointItem[T]

type PointMerge added in v1.3.0

type PointMerge[T util.ExceptString] func(prevPoint, currPoint *Point[T])

type PointRowIterator added in v1.0.0

type PointRowIterator interface {
	GetNext(row *influx.Row, tuple *TargetTuple)
	HasMore() bool
}

type Port

type Port interface {
	Equal(to Port) bool
	Connect(to Port)
	Redirect(to Port)
	ConnectionId() uintptr
	Close()
	Release()
}

type Ports

type Ports []Port

func (Ports) Close

func (ports Ports) Close()

func (Ports) Release

func (ports Ports) Release()

type PreAggregateCallMapping

type PreAggregateCallMapping struct {
	// contains filtered or unexported fields
}

func GetPreAggregateCallMapping

func GetPreAggregateCallMapping() *PreAggregateCallMapping

func NewPreAggregateCallMapping

func NewPreAggregateCallMapping() *PreAggregateCallMapping

func (*PreAggregateCallMapping) Contains

func (mapping *PreAggregateCallMapping) Contains(name string) bool

type PrePoint added in v1.4.0

type PrePoint struct {
	// contains filtered or unexported fields
}

func (*PrePoint) Clear added in v1.4.0

func (p *PrePoint) Clear()

type PrepareStmtBuilderCreator

type PrepareStmtBuilderCreator struct {
}

func (*PrepareStmtBuilderCreator) Create

func (p *PrepareStmtBuilderCreator) Create(stmt *influxql.SelectStatement, opt hybridqp.Options,
	shards interface {
		query.LogicalPlanCreator
		io.Closer
	}, columns []string, MaxPointN int, now time.Time) query.PreparedStatement

type Processor

type Processor interface {
	Work(ctx context.Context) error
	Close()
	Abort()
	Release() error
	Name() string
	GetOutputs() Ports
	GetInputs() Ports
	GetOutputNumber(port Port) int
	GetInputNumber(port Port) int
	IsSink() bool
	Explain() []ValuePair
	Analyze(span *tracing.Span)
	StartSpan(name string, withPP bool) *tracing.Span
	FinishSpan()
	Interrupt()
	InterruptWithoutMark()
}

func CreateBaseMergeTransform added in v1.0.0

func CreateBaseMergeTransform(plan LogicalPlan, mergeType MergeType) (Processor, error)

func NewHashAggTransform added in v1.1.0

func NewHashAggTransform(
	inRowDataType, outRowDataType []hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, s *QuerySchema, t HashAggType) (Processor, error)

func NewTopNTransform added in v1.4.0

func NewTopNTransform(
	inRowDataType, outRowDataType []hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, s *QuerySchema, t HashAggType, name string) (Processor, error)

type Processors

type Processors []Processor

func (Processors) Close

func (ps Processors) Close()

func (Processors) Empty

func (ps Processors) Empty() bool

func (Processors) Interrupt added in v1.3.0

func (ps Processors) Interrupt()

func (Processors) InterruptWithoutMark added in v1.4.0

func (ps Processors) InterruptWithoutMark()

func (Processors) Peek

func (ps Processors) Peek() Processor

func (*Processors) Pop

func (ps *Processors) Pop() Processor

func (*Processors) Push

func (ps *Processors) Push(p Processor)

func (Processors) Size

func (ps Processors) Size() int

type PromAbsentOp added in v1.4.0

type PromAbsentOp struct{}

func (*PromAbsentOp) CreateRoutine added in v1.4.0

func (c *PromAbsentOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PromGroupOp added in v1.3.0

type PromGroupOp struct{}

func (*PromGroupOp) CreateRoutine added in v1.3.0

func (c *PromGroupOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PromInstantVectorTransform added in v1.4.0

type PromInstantVectorTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewPromInstantVectorTransform added in v1.4.0

func NewPromInstantVectorTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, schema *QuerySchema, call *influxql.PromSubCall) (*PromInstantVectorTransform, error)

func (*PromInstantVectorTransform) Close added in v1.4.0

func (trans *PromInstantVectorTransform) Close()

func (*PromInstantVectorTransform) Explain added in v1.4.0

func (trans *PromInstantVectorTransform) Explain() []ValuePair

func (*PromInstantVectorTransform) GetGroupRange added in v1.4.0

func (trans *PromInstantVectorTransform) GetGroupRange(i int, chunk Chunk) (int, int)

func (*PromInstantVectorTransform) GetInputNumber added in v1.4.0

func (trans *PromInstantVectorTransform) GetInputNumber(_ Port) int

func (*PromInstantVectorTransform) GetInputs added in v1.4.0

func (trans *PromInstantVectorTransform) GetInputs() Ports

func (*PromInstantVectorTransform) GetOutputNumber added in v1.4.0

func (trans *PromInstantVectorTransform) GetOutputNumber(_ Port) int

func (*PromInstantVectorTransform) GetOutputs added in v1.4.0

func (trans *PromInstantVectorTransform) GetOutputs() Ports

func (*PromInstantVectorTransform) GroupCall added in v1.4.0

func (trans *PromInstantVectorTransform) GroupCall(startIdx, endIdx int)

func (*PromInstantVectorTransform) Name added in v1.4.0

func (trans *PromInstantVectorTransform) Name() string

func (*PromInstantVectorTransform) RangeCall added in v1.4.0

func (trans *PromInstantVectorTransform) RangeCall(startCurrLoc, endCurrLoc int, start, rangeMaxt int64)

func (*PromInstantVectorTransform) UpdateLoc added in v1.4.0

func (trans *PromInstantVectorTransform) UpdateLoc(startCurrLoc, endCurrLoc int)

func (*PromInstantVectorTransform) Work added in v1.4.0

func (*PromInstantVectorTransform) WorkHelper added in v1.4.0

func (trans *PromInstantVectorTransform) WorkHelper(ctx context.Context, errs *errno.Errs)

type PromInstantVectorTransformCreator added in v1.4.0

type PromInstantVectorTransformCreator struct {
}

func (*PromInstantVectorTransformCreator) Create added in v1.4.0

type PromQuantileOp added in v1.3.0

type PromQuantileOp struct{}

func (*PromQuantileOp) CreateRoutine added in v1.3.0

func (c *PromQuantileOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PromRangeVectorTransform added in v1.3.0

type PromRangeVectorTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewPromRangeVectorTransform added in v1.3.0

func NewPromRangeVectorTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, schema *QuerySchema, call *influxql.PromSubCall) (*PromRangeVectorTransform, error)

func (*PromRangeVectorTransform) Close added in v1.3.0

func (trans *PromRangeVectorTransform) Close()

func (*PromRangeVectorTransform) Explain added in v1.3.0

func (trans *PromRangeVectorTransform) Explain() []ValuePair

func (*PromRangeVectorTransform) GetGroupRange added in v1.3.0

func (trans *PromRangeVectorTransform) GetGroupRange(i int, chunk Chunk) (int, int)

func (*PromRangeVectorTransform) GetInputNumber added in v1.3.0

func (trans *PromRangeVectorTransform) GetInputNumber(_ Port) int

func (*PromRangeVectorTransform) GetInputs added in v1.3.0

func (trans *PromRangeVectorTransform) GetInputs() Ports

func (*PromRangeVectorTransform) GetOutputNumber added in v1.3.0

func (trans *PromRangeVectorTransform) GetOutputNumber(_ Port) int

func (*PromRangeVectorTransform) GetOutputs added in v1.3.0

func (trans *PromRangeVectorTransform) GetOutputs() Ports

func (*PromRangeVectorTransform) GroupCall added in v1.3.0

func (trans *PromRangeVectorTransform) GroupCall(startIdx, endIdx int)

func (*PromRangeVectorTransform) Name added in v1.3.0

func (trans *PromRangeVectorTransform) Name() string

func (*PromRangeVectorTransform) NewPromSubqueryCallFunc added in v1.3.0

func (trans *PromRangeVectorTransform) NewPromSubqueryCallFunc(call *influxql.PromSubCall) (CallFn, error)

func (*PromRangeVectorTransform) RangeCall added in v1.3.0

func (trans *PromRangeVectorTransform) RangeCall(startPreLoc, endPreLoc, startCurrLoc, endCurrLoc int, start, rangeMaxt int64)

func (*PromRangeVectorTransform) UpdateLoc added in v1.3.0

func (trans *PromRangeVectorTransform) UpdateLoc(startPreLoc, endPreLoc, startCurrLoc, endCurrLoc int)

func (*PromRangeVectorTransform) Work added in v1.3.0

func (trans *PromRangeVectorTransform) Work(ctx context.Context) error

func (*PromRangeVectorTransform) WorkHelper added in v1.3.0

func (trans *PromRangeVectorTransform) WorkHelper(ctx context.Context, errs *errno.Errs)

type PromRangeVectorTransformCreator added in v1.3.0

type PromRangeVectorTransformCreator struct {
}

func (*PromRangeVectorTransformCreator) Create added in v1.3.0

type PromScalarOp added in v1.3.0

type PromScalarOp struct{}

func (*PromScalarOp) CreateRoutine added in v1.3.0

func (c *PromScalarOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PromSortGroups added in v1.4.0

type PromSortGroups struct {
	// contains filtered or unexported fields
}

func NewPromSortGroups added in v1.4.0

func NewPromSortGroups() PromSortGroups

func (*PromSortGroups) AppendMany added in v1.4.0

func (rows *PromSortGroups) AppendMany(tagKey, tagValue []string, value []float64, timestamp []int64)

type PromSortSeries added in v1.4.0

type PromSortSeries struct {
	// contains filtered or unexported fields
}

func NewPromSortSeries added in v1.4.0

func NewPromSortSeries(tagKey, tagValue []string, value []float64, timestamp []int64) *PromSortSeries

func (*PromSortSeries) Value added in v1.4.0

func (r *PromSortSeries) Value() float64

As range query results always have a fixed output ordering, we don't care how values are comparing in such a situation. So we simply take the first sample value for each series.

type PromSortTransform added in v1.4.0

type PromSortTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewPromSortTransform added in v1.4.0

func NewPromSortTransform(inRowDataType, outRowDataType hybridqp.RowDataType, s *QuerySchema, sortFields influxql.SortFields) (*PromSortTransform, error)

func (*PromSortTransform) Close added in v1.4.0

func (trans *PromSortTransform) Close()

func (*PromSortTransform) Explain added in v1.4.0

func (trans *PromSortTransform) Explain() []ValuePair

func (*PromSortTransform) GetInputNumber added in v1.4.0

func (trans *PromSortTransform) GetInputNumber(_ Port) int

func (*PromSortTransform) GetInputs added in v1.4.0

func (trans *PromSortTransform) GetInputs() Ports

func (*PromSortTransform) GetOutputNumber added in v1.4.0

func (trans *PromSortTransform) GetOutputNumber(_ Port) int

func (*PromSortTransform) GetOutputs added in v1.4.0

func (trans *PromSortTransform) GetOutputs() Ports

func (*PromSortTransform) Name added in v1.4.0

func (trans *PromSortTransform) Name() string

func (*PromSortTransform) Work added in v1.4.0

func (trans *PromSortTransform) Work(ctx context.Context) error

func (*PromSortTransform) WorkHelper added in v1.4.0

func (trans *PromSortTransform) WorkHelper(ctx context.Context, errs *errno.Errs)

type PromSortTransformCreator added in v1.4.0

type PromSortTransformCreator struct {
}

func (*PromSortTransformCreator) Create added in v1.4.0

type PromStdOp added in v1.3.0

type PromStdOp struct {
	// contains filtered or unexported fields
}

func (*PromStdOp) CreateRoutine added in v1.3.0

func (c *PromStdOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type PromStepInvariantType added in v1.4.0

type PromStepInvariantType int
const (
	SkipLastGroup PromStepInvariantType = iota
	OnlyLastGroup
)

type PromTimeFunction added in v1.3.0

type PromTimeFunction interface {
	CallFunc(name string, args []interface{}) (interface{}, bool)
}

type PromTimeValuer added in v1.3.0

type PromTimeValuer struct{}

valuer

func (PromTimeValuer) Call added in v1.3.0

func (v PromTimeValuer) Call(name string, args []interface{}) (interface{}, bool)

func (PromTimeValuer) SetValuer added in v1.3.0

func (PromTimeValuer) SetValuer(_ influxql.Valuer, _ int)

func (PromTimeValuer) Value added in v1.3.0

func (PromTimeValuer) Value(_ string) (interface{}, bool)

type Property added in v1.5.0

type Property struct {
	Key   string `json:"key"`
	Value string `json:"value"`
}

type PtQuery added in v1.2.0

type PtQuery struct {
	PtID       uint32
	ShardInfos []ShardInfo
}

func UnmarshalPtQuerys added in v1.2.0

func UnmarshalPtQuerys(ptQuerys []*proto2.PtQuery) []PtQuery

type QueryNodeStack

type QueryNodeStack []hybridqp.QueryNode

func (*QueryNodeStack) Empty

func (s *QueryNodeStack) Empty() bool

func (*QueryNodeStack) Peek

func (s *QueryNodeStack) Peek() hybridqp.QueryNode

func (*QueryNodeStack) Pop

func (*QueryNodeStack) Push

func (s *QueryNodeStack) Push(node hybridqp.QueryNode)

func (*QueryNodeStack) Size

func (s *QueryNodeStack) Size() int

type QuerySchema

type QuerySchema struct {
	PromSubCalls []*influxql.PromSubCall

	InConditons []*influxql.InCondition
	// contains filtered or unexported fields
}

func NewQuerySchema

func NewQuerySchema(fields influxql.Fields, columnNames []string, opt hybridqp.Options, sortFields influxql.SortFields) *QuerySchema

func NewQuerySchemaWithJoinCase added in v1.0.0

func NewQuerySchemaWithJoinCase(fields influxql.Fields, sources influxql.Sources, columnNames []string, opt hybridqp.Options,
	joinCases []*influxql.Join, unionCases []*influxql.Union, unnest []*influxql.Unnest, sortFields influxql.SortFields) *QuerySchema

func NewQuerySchemaWithOpt

func NewQuerySchemaWithOpt(opt hybridqp.Options) *QuerySchema

func NewQuerySchemaWithSources

func NewQuerySchemaWithSources(fields influxql.Fields, sources influxql.Sources, columnNames []string, opt hybridqp.Options, sortFields influxql.SortFields) *QuerySchema

func (*QuerySchema) AddHoltWinters added in v1.0.0

func (qs *QuerySchema) AddHoltWinters(call *influxql.Call, alias string)

func (*QuerySchema) AddLLMFunc added in v1.5.0

func (qs *QuerySchema) AddLLMFunc(call *influxql.Call, alias string)

func (*QuerySchema) AddLabelCalls added in v1.3.0

func (qs *QuerySchema) AddLabelCalls(key string, labelCalls *influxql.Call)

func (*QuerySchema) AddMath

func (qs *QuerySchema) AddMath(key string, math *influxql.Call)

func (*QuerySchema) AddPromTimeCalls added in v1.3.0

func (qs *QuerySchema) AddPromTimeCalls(key string, promTimeCalls *influxql.Call)

func (*QuerySchema) AddSlidingWindow

func (qs *QuerySchema) AddSlidingWindow(key string, str *influxql.Call)

func (*QuerySchema) AddString

func (qs *QuerySchema) AddString(key string, str *influxql.Call)

func (*QuerySchema) AddTable

func (qs *QuerySchema) AddTable(m *influxql.Measurement, refs []influxql.VarRef)

func (*QuerySchema) Binarys

func (qs *QuerySchema) Binarys() map[string]*influxql.BinaryExpr

func (*QuerySchema) BuildDownSampleSchema added in v1.0.0

func (qs *QuerySchema) BuildDownSampleSchema(addPrefix bool) record.Schemas

func (*QuerySchema) CalculateQueryTagFromSymbol

func (qs *QuerySchema) CalculateQueryTagFromSymbol(symbolVal string) (string, error)

func (*QuerySchema) Calls

func (qs *QuerySchema) Calls() map[string]*influxql.Call

func (*QuerySchema) CanAggPushDown

func (qs *QuerySchema) CanAggPushDown() bool

func (*QuerySchema) CanAggTagSet added in v1.3.0

func (qs *QuerySchema) CanAggTagSet() bool

CanAggTagSet indicates that aggregation is performed among multiple TagSets. File traversal and SeqAgg optimization are used.

func (*QuerySchema) CanCallsPushdown

func (qs *QuerySchema) CanCallsPushdown() bool

func (*QuerySchema) CanLimitCut

func (qs *QuerySchema) CanLimitCut() bool

func (*QuerySchema) CanSeqAggPushDown added in v1.2.0

func (qs *QuerySchema) CanSeqAggPushDown() bool

CanSeqAggPushDown determines whether the csstore engine performs seqAgg optimization.

func (*QuerySchema) ClearInConditions added in v1.5.0

func (qs *QuerySchema) ClearInConditions()

func (*QuerySchema) CloneField

func (qs *QuerySchema) CloneField(f *influxql.Field) *influxql.Field

func (*QuerySchema) CompositeCall added in v1.0.0

func (qs *QuerySchema) CompositeCall() map[string]*hybridqp.OGSketchCompositeOperator

func (*QuerySchema) ContainSeriesIgnoreCall

func (qs *QuerySchema) ContainSeriesIgnoreCall() bool

func (*QuerySchema) CountDistinct

func (qs *QuerySchema) CountDistinct() *influxql.Call

func (*QuerySchema) CountField

func (qs *QuerySchema) CountField() map[int]bool

func (*QuerySchema) DerivedOrigCall

func (qs *QuerySchema) DerivedOrigCall(call *influxql.Call) influxql.VarRef

func (*QuerySchema) DerivedRef

func (qs *QuerySchema) DerivedRef(ref *influxql.VarRef) influxql.VarRef

func (*QuerySchema) Fields

func (qs *QuerySchema) Fields() influxql.Fields

func (*QuerySchema) FieldsRef

func (qs *QuerySchema) FieldsRef() influxql.VarRefs

func (*QuerySchema) GetColumnNames

func (qs *QuerySchema) GetColumnNames() []string

func (*QuerySchema) GetFieldType added in v1.0.0

func (qs *QuerySchema) GetFieldType(i int) (int64, error)

func (*QuerySchema) GetJoinCaseCount added in v1.0.0

func (qs *QuerySchema) GetJoinCaseCount() int

func (*QuerySchema) GetJoinCases added in v1.0.0

func (qs *QuerySchema) GetJoinCases() []*influxql.Join

func (*QuerySchema) GetOptions

func (qs *QuerySchema) GetOptions() hybridqp.Options

func (*QuerySchema) GetPlanType added in v1.1.0

func (qs *QuerySchema) GetPlanType() PlanType

func (*QuerySchema) GetPromCalls added in v1.3.0

func (qs *QuerySchema) GetPromCalls() []*influxql.PromSubCall

func (*QuerySchema) GetQueryFields

func (qs *QuerySchema) GetQueryFields() influxql.Fields

func (*QuerySchema) GetResource added in v1.5.0

func (qs *QuerySchema) GetResource(name string) map[string]string

func (*QuerySchema) GetSortFields added in v1.1.0

func (qs *QuerySchema) GetSortFields() influxql.SortFields

func (*QuerySchema) GetSourcesNames added in v1.0.0

func (qs *QuerySchema) GetSourcesNames() []string

func (*QuerySchema) GetTimeRangeByTC added in v1.2.0

func (qs *QuerySchema) GetTimeRangeByTC() util.TimeRange

func (*QuerySchema) GetUnionCases added in v1.5.0

func (qs *QuerySchema) GetUnionCases() []*influxql.Union

func (*QuerySchema) GetUnnests added in v1.2.0

func (qs *QuerySchema) GetUnnests() influxql.Unnests

func (*QuerySchema) HasAuxTag added in v1.0.0

func (qs *QuerySchema) HasAuxTag() bool

func (*QuerySchema) HasBlankRowCall

func (qs *QuerySchema) HasBlankRowCall() bool

func (*QuerySchema) HasCall

func (qs *QuerySchema) HasCall() bool

func (*QuerySchema) HasCastorCall added in v0.2.0

func (qs *QuerySchema) HasCastorCall() bool

func (*QuerySchema) HasDistinct added in v1.5.0

func (qs *QuerySchema) HasDistinct() bool

func (*QuerySchema) HasExcatLimit added in v1.0.0

func (qs *QuerySchema) HasExcatLimit() bool

func (*QuerySchema) HasFieldCondition

func (qs *QuerySchema) HasFieldCondition() bool

func (*QuerySchema) HasGroupBy

func (qs *QuerySchema) HasGroupBy() bool

func (*QuerySchema) HasHoltWintersCall added in v1.0.0

func (qs *QuerySchema) HasHoltWintersCall() bool

func (*QuerySchema) HasInCondition added in v1.5.0

func (qs *QuerySchema) HasInCondition() bool

func (*QuerySchema) HasInSeriesAgg

func (qs *QuerySchema) HasInSeriesAgg() bool

func (*QuerySchema) HasInterval

func (qs *QuerySchema) HasInterval() bool

func (*QuerySchema) HasLLMFunc added in v1.5.0

func (qs *QuerySchema) HasLLMFunc() bool

func (*QuerySchema) HasLabelCalls added in v1.3.0

func (qs *QuerySchema) HasLabelCalls() bool

func (*QuerySchema) HasLimit

func (qs *QuerySchema) HasLimit() bool

func (*QuerySchema) HasMath

func (qs *QuerySchema) HasMath() bool

func (*QuerySchema) HasMeanCall

func (qs *QuerySchema) HasMeanCall() bool

func (*QuerySchema) HasNonPreCall

func (qs *QuerySchema) HasNonPreCall() bool

func (*QuerySchema) HasNotInSeriesAgg

func (qs *QuerySchema) HasNotInSeriesAgg() bool

func (*QuerySchema) HasOptimizeAgg added in v1.0.0

func (qs *QuerySchema) HasOptimizeAgg() bool

func (*QuerySchema) HasOptimizeCall added in v1.1.0

func (qs *QuerySchema) HasOptimizeCall() bool

func (*QuerySchema) HasPercentileOGSketch added in v1.0.0

func (qs *QuerySchema) HasPercentileOGSketch() bool

func (*QuerySchema) HasPromAbsentCall added in v1.4.0

func (qs *QuerySchema) HasPromAbsentCall() bool

func (*QuerySchema) HasPromNestedCall added in v1.3.0

func (qs *QuerySchema) HasPromNestedCall() bool

func (*QuerySchema) HasPromTimeCalls added in v1.3.0

func (qs *QuerySchema) HasPromTimeCalls() bool

func (*QuerySchema) HasRowCount added in v1.1.0

func (qs *QuerySchema) HasRowCount() bool

HasRowCount check whether all data is queried to use mst-level pre-aggregation.

func (*QuerySchema) HasSlidingWindowCall

func (qs *QuerySchema) HasSlidingWindowCall() bool

func (*QuerySchema) HasSort added in v1.1.0

func (qs *QuerySchema) HasSort() bool

func (*QuerySchema) HasStreamCall

func (qs *QuerySchema) HasStreamCall() bool

func (*QuerySchema) HasString

func (qs *QuerySchema) HasString() bool

func (*QuerySchema) HasSubQuery

func (qs *QuerySchema) HasSubQuery() bool

func (*QuerySchema) HasTopN added in v1.5.0

func (qs *QuerySchema) HasTopN() (string, bool)

func (*QuerySchema) HasUnnests added in v1.2.0

func (qs *QuerySchema) HasUnnests() bool

func (*QuerySchema) HoltWinters added in v1.0.0

func (qs *QuerySchema) HoltWinters() []*influxql.Field

func (*QuerySchema) InitFieldCondition added in v1.0.0

func (qs *QuerySchema) InitFieldCondition()

func (*QuerySchema) IsColumnStoreCount added in v1.5.0

func (qs *QuerySchema) IsColumnStoreCount() bool

func (*QuerySchema) IsCompareCall added in v1.5.0

func (qs *QuerySchema) IsCompareCall() bool

func (*QuerySchema) IsHoltWinters added in v1.1.0

func (qs *QuerySchema) IsHoltWinters(val string) bool

func (*QuerySchema) IsInSubquerySchema added in v1.5.0

func (qs *QuerySchema) IsInSubquerySchema() bool

func (*QuerySchema) IsLastFieldQuery added in v1.5.0

func (qs *QuerySchema) IsLastFieldQuery() bool

func (*QuerySchema) IsLastRowQuery added in v1.5.0

func (qs *QuerySchema) IsLastRowQuery() bool

func (*QuerySchema) IsMultiMeasurements

func (qs *QuerySchema) IsMultiMeasurements() bool

func (*QuerySchema) IsOnlyCSPreAgg

func (qs *QuerySchema) IsOnlyCSPreAgg() bool

func (*QuerySchema) IsPromAbsentCall added in v1.4.0

func (qs *QuerySchema) IsPromAbsentCall() bool

func (*QuerySchema) IsPromNestedCall added in v1.3.0

func (qs *QuerySchema) IsPromNestedCall(call *influxql.Call) bool

func (*QuerySchema) IsPromNestedCountCall added in v1.4.0

func (qs *QuerySchema) IsPromNestedCountCall() bool

func (*QuerySchema) IsRefInQueryFields

func (qs *QuerySchema) IsRefInQueryFields(ref *influxql.VarRef) bool

func (*QuerySchema) IsRefInSymbolFields

func (qs *QuerySchema) IsRefInSymbolFields(ref *influxql.VarRef) bool

func (*QuerySchema) IsSelectDistinct

func (qs *QuerySchema) IsSelectDistinct() bool

func (*QuerySchema) IsTimeZero

func (qs *QuerySchema) IsTimeZero() bool

func (*QuerySchema) LLMFunc added in v1.5.0

func (qs *QuerySchema) LLMFunc() []*influxql.Field

func (*QuerySchema) LimitAndOffset

func (qs *QuerySchema) LimitAndOffset() (int, int)

func (*QuerySchema) LimitType

func (qs *QuerySchema) LimitType() hybridqp.LimitType

func (*QuerySchema) MakeRefs

func (qs *QuerySchema) MakeRefs() []influxql.VarRef

func (*QuerySchema) Mapping

func (qs *QuerySchema) Mapping() map[influxql.Expr]influxql.VarRef

func (*QuerySchema) MatchPreAgg

func (qs *QuerySchema) MatchPreAgg() bool

func (*QuerySchema) OnlyOneCallRef

func (qs *QuerySchema) OnlyOneCallRef() *influxql.VarRef

func (*QuerySchema) Options

func (qs *QuerySchema) Options() hybridqp.Options

func (*QuerySchema) OrigCalls

func (qs *QuerySchema) OrigCalls() map[string]*influxql.Call

func (*QuerySchema) PromNestedCall added in v1.3.0

func (qs *QuerySchema) PromNestedCall() map[string]*hybridqp.PromNestedCall

func (*QuerySchema) PromResetTime added in v1.3.0

func (qs *QuerySchema) PromResetTime() bool

PromResetTime is used to determine whether to set the time of result to the end time of the query, according to the semantics of the prom instant query,

func (*QuerySchema) RecordExpr added in v1.5.0

func (qs *QuerySchema) RecordExpr(expr influxql.Node)

func (*QuerySchema) Refs

func (qs *QuerySchema) Refs() map[string]*influxql.VarRef

func (*QuerySchema) SetCompareCall added in v1.5.0

func (qs *QuerySchema) SetCompareCall()

func (*QuerySchema) SetDistinct added in v1.5.0

func (qs *QuerySchema) SetDistinct(isDistinct bool)

func (*QuerySchema) SetFill added in v1.1.0

func (qs *QuerySchema) SetFill(fill influxql.FillOption)

func (*QuerySchema) SetHoltWinters added in v1.0.0

func (qs *QuerySchema) SetHoltWinters(calls []*influxql.Call)

func (*QuerySchema) SetIsInSubquerySchema added in v1.5.0

func (qs *QuerySchema) SetIsInSubquerySchema(isInSubquerySchema bool)

func (*QuerySchema) SetJoinCases added in v1.5.0

func (qs *QuerySchema) SetJoinCases(joinCase []*influxql.Join)

func (*QuerySchema) SetOpt

func (qs *QuerySchema) SetOpt(opt hybridqp.Options)

func (*QuerySchema) SetPlanType added in v1.1.0

func (qs *QuerySchema) SetPlanType(planType PlanType)

func (*QuerySchema) SetPromCalls added in v1.3.0

func (qs *QuerySchema) SetPromCalls(calls []*influxql.PromSubCall)

func (*QuerySchema) SetResource added in v1.5.0

func (qs *QuerySchema) SetResource(name string, resource map[string]string)

func (*QuerySchema) SetSelectDistinct

func (qs *QuerySchema) SetSelectDistinct(isSelectDistinct bool)

func (*QuerySchema) SetSimpleTagset added in v1.3.0

func (qs *QuerySchema) SetSimpleTagset()

func (*QuerySchema) SetSources added in v1.5.0

func (qs *QuerySchema) SetSources(sources influxql.Sources)

func (*QuerySchema) SetUnionCases added in v1.5.0

func (qs *QuerySchema) SetUnionCases(unionCase []*influxql.Union)

func (*QuerySchema) SetUnnests added in v1.2.0

func (qs *QuerySchema) SetUnnests(unnests []*influxql.Unnest)

func (*QuerySchema) SlidingWindow

func (qs *QuerySchema) SlidingWindow() map[string]*influxql.Call

func (*QuerySchema) Sources

func (qs *QuerySchema) Sources() influxql.Sources

func (*QuerySchema) Symbols

func (qs *QuerySchema) Symbols() map[string]influxql.VarRef

func (*QuerySchema) Table

func (qs *QuerySchema) Table(name string) *QueryTable

type QuerySchemaCreator

type QuerySchemaCreator struct {
}

func (*QuerySchemaCreator) Create

func (c *QuerySchemaCreator) Create(fields influxql.Fields, columnNames []string, opt hybridqp.Options) hybridqp.Catalog

type QuerySchemaExpressionVisitor added in v1.5.0

type QuerySchemaExpressionVisitor struct {
	QuerySchema *QuerySchema
	ExprString  *influxql.NestedNodeStringRepresentation
}

func (*QuerySchemaExpressionVisitor) Visit added in v1.5.0

type QueryTable

type QueryTable struct {
	// contains filtered or unexported fields
}

func NewQueryTable

func NewQueryTable(m *influxql.Measurement, refs []influxql.VarRef) *QueryTable

func (*QueryTable) Name

func (b *QueryTable) Name() string

func (*QueryTable) RowDataType

func (b *QueryTable) RowDataType() hybridqp.RowDataType

type RCAOp added in v1.5.0

type RCAOp struct{}

func (*RCAOp) CreateRoutine added in v1.5.0

func (c *RCAOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

func (*RCAOp) Run added in v1.5.0

func (c *RCAOp) Run(params *TableFunctionParams) ([]Chunk, error)

type RPCClient

type RPCClient struct {
	// contains filtered or unexported fields
}

func NewRPCClient

func NewRPCClient(query *RemoteQuery) *RPCClient

func (*RPCClient) Abort

func (c *RPCClient) Abort(noMarkCrash bool)

func (*RPCClient) AddHandler

func (c *RPCClient) AddHandler(msgType byte, handle Handler)

func (*RPCClient) FinishAnalyze

func (c *RPCClient) FinishAnalyze()

func (*RPCClient) GetCodec

func (c *RPCClient) GetCodec() transport.Codec

func (*RPCClient) Handle

func (c *RPCClient) Handle(data interface{}) error

func (*RPCClient) Init added in v1.1.0

func (c *RPCClient) Init(ctx context.Context, queryNode []byte)

func (*RPCClient) Interrupt added in v1.3.0

func (c *RPCClient) Interrupt(noMarkCrash bool)

func (*RPCClient) Run

func (c *RPCClient) Run() error

func (*RPCClient) StartAnalyze

func (c *RPCClient) StartAnalyze(span *tracing.Span)

type RPCReaderTransform

type RPCReaderTransform struct {
	BaseProcessor

	Output *ChunkPort

	NoMarkCrash bool
	// contains filtered or unexported fields
}

func NewRPCReaderTransform

func NewRPCReaderTransform(outRowDataType hybridqp.RowDataType, queryId uint64, rq *RemoteQuery) *RPCReaderTransform

func (*RPCReaderTransform) Abort

func (t *RPCReaderTransform) Abort()

func (*RPCReaderTransform) Close

func (t *RPCReaderTransform) Close()

func (*RPCReaderTransform) Distribute

func (t *RPCReaderTransform) Distribute(node hybridqp.QueryNode)

func (*RPCReaderTransform) Explain

func (t *RPCReaderTransform) Explain() []ValuePair

func (*RPCReaderTransform) GetInputNumber

func (t *RPCReaderTransform) GetInputNumber(_ Port) int

func (*RPCReaderTransform) GetInputs

func (t *RPCReaderTransform) GetInputs() Ports

func (*RPCReaderTransform) GetOutputNumber

func (t *RPCReaderTransform) GetOutputNumber(_ Port) int

func (*RPCReaderTransform) GetOutputs

func (t *RPCReaderTransform) GetOutputs() Ports

func (*RPCReaderTransform) Interrupt added in v1.3.0

func (t *RPCReaderTransform) Interrupt()

func (*RPCReaderTransform) InterruptWithoutMark added in v1.4.0

func (t *RPCReaderTransform) InterruptWithoutMark()

func (*RPCReaderTransform) IsSink

func (t *RPCReaderTransform) IsSink() bool

func (*RPCReaderTransform) Name

func (t *RPCReaderTransform) Name() string

func (*RPCReaderTransform) Release

func (t *RPCReaderTransform) Release() error

func (*RPCReaderTransform) Work

func (t *RPCReaderTransform) Work(ctx context.Context) error

type RPCSenderTransform

type RPCSenderTransform struct {
	BaseProcessor

	Input *ChunkPort
	// contains filtered or unexported fields
}

func (*RPCSenderTransform) Close

func (t *RPCSenderTransform) Close()

func (*RPCSenderTransform) Explain

func (t *RPCSenderTransform) Explain() []ValuePair

func (*RPCSenderTransform) GetInputNumber

func (t *RPCSenderTransform) GetInputNumber(_ Port) int

func (*RPCSenderTransform) GetInputs

func (t *RPCSenderTransform) GetInputs() Ports

func (*RPCSenderTransform) GetOutputNumber

func (t *RPCSenderTransform) GetOutputNumber(_ Port) int

func (*RPCSenderTransform) GetOutputs

func (t *RPCSenderTransform) GetOutputs() Ports

func (*RPCSenderTransform) Name

func (t *RPCSenderTransform) Name() string

func (*RPCSenderTransform) Work

func (t *RPCSenderTransform) Work(ctx context.Context) error

type RateFinalReduceFunc added in v1.3.0

type RateFinalReduceFunc[T util.NumberOnly] func(firstTime int64, lastTime int64, firstValue T, lastValue T, interval *hybridqp.Interval) (float64, bool)

type RateMergeFunc added in v1.3.0

type RateMergeFunc[T util.NumberOnly] func(prevPoints [2]*Point[T], interval *hybridqp.Interval) (float64, bool)

type RateMiddleReduceFunc added in v1.3.0

type RateMiddleReduceFunc[T util.NumberOnly] func(c Chunk, values []T, ordinal, start, end int) (firstIndex, lastIndex int, firstValue, lastValue T, isNil bool)

type RateUpdateFunc added in v1.3.0

type RateUpdateFunc[T util.NumberOnly] func(prevPoints, currPoints [2]*Point[T])

type ReaderCreator added in v1.1.0

type ReaderCreator interface {
	CreateReader(plan hybridqp.QueryNode, frags interface{}) (Processor, error)
}

type ReaderCreatorFactory added in v1.1.0

type ReaderCreatorFactory struct {
	// contains filtered or unexported fields
}

func GetReaderFactoryInstance added in v1.1.0

func GetReaderFactoryInstance() *ReaderCreatorFactory

func NewReaderCreatorFactory added in v1.1.0

func NewReaderCreatorFactory() *ReaderCreatorFactory

func (*ReaderCreatorFactory) Add added in v1.1.0

func (r *ReaderCreatorFactory) Add(name string, creator ReaderCreator)

func (*ReaderCreatorFactory) Find added in v1.1.0

func (r *ReaderCreatorFactory) Find(name string) (ReaderCreator, bool)

type ReaderUpdateInSubQueryRule

type ReaderUpdateInSubQueryRule struct {
	OptRuleBase
}

func NewReaderUpdateInSubQueryRule

func NewReaderUpdateInSubQueryRule(description string) *ReaderUpdateInSubQueryRule

func (*ReaderUpdateInSubQueryRule) Category added in v1.5.0

func (*ReaderUpdateInSubQueryRule) Equals

func (r *ReaderUpdateInSubQueryRule) Equals(rhs OptRule) bool

func (*ReaderUpdateInSubQueryRule) OnMatch

func (r *ReaderUpdateInSubQueryRule) OnMatch(call *OptRuleCall)

func (*ReaderUpdateInSubQueryRule) ToString

func (r *ReaderUpdateInSubQueryRule) ToString() string

type RecordsGenerator added in v1.5.0

type RecordsGenerator struct {
	// contains filtered or unexported fields
}

func NewRecordsGenerator added in v1.5.0

func NewRecordsGenerator(chunkedSize int, opt hybridqp.Options) *RecordsGenerator

func (*RecordsGenerator) Generate added in v1.5.0

func (g *RecordsGenerator) Generate(chunk Chunk, lastChunk bool, records []*models.RecordContainer) []*models.RecordContainer

func (*RecordsGenerator) Release added in v1.5.0

func (g *RecordsGenerator) Release()

type ReflectionTable

type ReflectionTable []int

type ReflectionTables

type ReflectionTables []ReflectionTable

type RegrSlopeOp added in v1.5.0

type RegrSlopeOp struct{}

func (*RegrSlopeOp) CreateRoutine added in v1.5.0

func (c *RegrSlopeOp) CreateRoutine(params *AggCallFuncParams) (Routine, error)

type RemoteQuery

type RemoteQuery struct {
	Database string
	PtID     uint32 // for tsstore
	NodeID   uint64
	ShardIDs []uint64  // for tsstore
	PtQuerys []PtQuery // for csstore
	Opt      query.ProcessorOptions
	Analyze  bool
	Node     []byte
	MstInfos []*MultiMstInfo
}

func (*RemoteQuery) BuildMstTraits added in v1.5.0

func (c *RemoteQuery) BuildMstTraits() []*MultiMstReqs

func (*RemoteQuery) Clone added in v1.5.0

func (c *RemoteQuery) Clone() *RemoteQuery

func (*RemoteQuery) Empty added in v1.2.0

func (c *RemoteQuery) Empty() bool

func (*RemoteQuery) HaveLocalMst added in v1.2.0

func (c *RemoteQuery) HaveLocalMst() bool

func (*RemoteQuery) Instance

func (c *RemoteQuery) Instance() transport.Codec

func (*RemoteQuery) Len added in v1.2.0

func (c *RemoteQuery) Len() int

func (*RemoteQuery) Marshal

func (c *RemoteQuery) Marshal(buf []byte) ([]byte, error)

func (*RemoteQuery) MarshalMstInfos added in v1.5.0

func (c *RemoteQuery) MarshalMstInfos(rq *proto2.RemoteQuery) error

func (*RemoteQuery) Size

func (c *RemoteQuery) Size() int

func (*RemoteQuery) Unmarshal

func (c *RemoteQuery) Unmarshal(buf []byte) error

func (*RemoteQuery) UnmarshalMstInfos added in v1.5.0

func (c *RemoteQuery) UnmarshalMstInfos(pb *proto2.RemoteQuery) error

type Response added in v1.5.0

type Response struct {
	Data GraphData `json:"data"`
}

type ResultEval added in v1.2.0

type ResultEval struct {
	// contains filtered or unexported fields
}

func (*ResultEval) IsNil added in v1.2.0

func (re *ResultEval) IsNil(i int) bool

type ResultEvalPool added in v1.2.0

type ResultEvalPool struct {
	// contains filtered or unexported fields
}

func NewResultEvalPool added in v1.2.0

func NewResultEvalPool(len int, isPromQuery bool) *ResultEvalPool

type RetryErrTransform added in v1.4.0

type RetryErrTransform struct {
	BaseProcessor
}

func NewRetryErrTransform added in v1.4.0

func NewRetryErrTransform() *RetryErrTransform

func (*RetryErrTransform) Close added in v1.4.0

func (source *RetryErrTransform) Close()

func (*RetryErrTransform) Explain added in v1.4.0

func (source *RetryErrTransform) Explain() []ValuePair

func (*RetryErrTransform) GetInputNumber added in v1.4.0

func (source *RetryErrTransform) GetInputNumber(_ Port) int

func (*RetryErrTransform) GetInputs added in v1.4.0

func (source *RetryErrTransform) GetInputs() Ports

func (*RetryErrTransform) GetOutputNumber added in v1.4.0

func (source *RetryErrTransform) GetOutputNumber(_ Port) int

func (*RetryErrTransform) GetOutputs added in v1.4.0

func (source *RetryErrTransform) GetOutputs() Ports

func (*RetryErrTransform) Name added in v1.4.0

func (source *RetryErrTransform) Name() string

func (*RetryErrTransform) Work added in v1.4.0

func (source *RetryErrTransform) Work(ctx context.Context) error

type Routine

type Routine interface {
	WorkOnChunk(Chunk, Chunk, *IteratorParams)
}

func NewAbsentRoutineImpl

func NewAbsentRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool) (Routine, error)

func NewBottomRoutineImpl

func NewBottomRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, auxProcessor []*AuxProcessor, ops *query.ProcessorOptions) (Routine, error)

func NewCountRoutineImpl

func NewCountRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool) (Routine, error)

func NewCumulativeSumRoutineImpl

func NewCumulativeSumRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall bool,
) (Routine, error)

func NewDerivativeRoutineImpl

func NewDerivativeRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall, isNonNegative, ascending bool, interval hybridqp.Interval,
) (Routine, error)

func NewDifferenceRoutineImpl

func NewDifferenceRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall, isNonNegative bool,
) (Routine, error)

func NewDistinctRoutineImpl

func NewDistinctRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions) (Routine, error)

func NewElapsedRoutineImpl

func NewElapsedRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall bool,
) (Routine, error)

func NewFirstRoutineImpl

func NewFirstRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool, auxProcessor []*AuxProcessor) (Routine, error)

func NewIntegralRoutineImpl

func NewIntegralRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	opts *query.ProcessorOptions, isSingleCall bool,
) (Routine, error)

func NewLastRoutineImpl

func NewLastRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool, auxProcessor []*AuxProcessor) (Routine, error)

func NewMedianRoutineImpl

func NewMedianRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool) (Routine, error)

func NewModeRoutineImpl

func NewModeRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool) (Routine, error)

func NewMovingAverageRoutineImpl

func NewMovingAverageRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall bool) (Routine, error)

func NewPercentileApproxRoutineImpl added in v1.0.0

func NewPercentileApproxRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, exprOpt hybridqp.ExprOptions, isSingleCall bool, opt *query.ProcessorOptions, name string, clusterNum int, percentile float64) (Routine, error)

func NewPercentileRoutineImpl

func NewPercentileRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool, auxProcessor []*AuxProcessor) (Routine, error)

func NewRateRoutineImpl

func NewRateRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions,
	isSingleCall, isRate bool, interval hybridqp.Interval,
) (Routine, error)

func NewSampleRoutineImpl

func NewSampleRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool, auxProcessor []*AuxProcessor) (Routine, error)

func NewSlidingWindowCountRoutineImpl

func NewSlidingWindowCountRoutineImpl(
	inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, slidingNum int,
) Routine

func NewSlidingWindowMaxRoutineImpl

func NewSlidingWindowMaxRoutineImpl(
	inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, slidingNum int,
) Routine

func NewSlidingWindowMinRoutineImpl

func NewSlidingWindowMinRoutineImpl(
	inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, slidingNum int,
) Routine

func NewSlidingWindowSumRoutineImpl

func NewSlidingWindowSumRoutineImpl(
	inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, slidingNum int,
) Routine

func NewStddevRoutineImpl

func NewStddevRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool,
) (Routine, error)

func NewSumRoutineImpl

func NewSumRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, isSingleCall bool) (Routine, error)

func NewTopRoutineImpl

func NewTopRoutineImpl(inRowDataType, outRowDataType hybridqp.RowDataType, opt hybridqp.ExprOptions, auxProcessor []*AuxProcessor, ops *query.ProcessorOptions) (Routine, error)

type RoutineImpl

type RoutineImpl struct {
	// contains filtered or unexported fields
}

func NewRoutineImpl

func NewRoutineImpl(iterator Iterator, inOrdinal int, outOrdinal int) *RoutineImpl

func (*RoutineImpl) WorkOnChunk

func (r *RoutineImpl) WorkOnChunk(in Chunk, out Chunk, params *IteratorParams)

type Row

type Row struct {
	// contains filtered or unexported fields
}

type RowChunk

type RowChunk struct {
	Name string
	Tags []ChunkTags

	Series     []*Row
	ColumnName []string
}

func (*RowChunk) RowsGen

func (r *RowChunk) RowsGen(c Chunk) []*Row

func (*RowChunk) RowsInit

func (r *RowChunk) RowsInit(c Chunk)

type Rows

type Rows struct {
	Time   []int64
	Values [][]interface{}
	Series Series
	// contains filtered or unexported fields
}

type RowsGenerator added in v1.1.0

type RowsGenerator struct {
	// contains filtered or unexported fields
}

func NewRowsGenerator added in v1.1.0

func NewRowsGenerator() *RowsGenerator

func (*RowsGenerator) Generate added in v1.1.0

func (g *RowsGenerator) Generate(chunk Chunk, loc *time.Location) models.Rows

func (*RowsGenerator) GenerateFromRecord

func (g *RowsGenerator) GenerateFromRecord(rec *record.Record, loc *time.Location, name string, schema record.Schemas) models.Rows

func (*RowsGenerator) GetColValue added in v1.1.0

func (g *RowsGenerator) GetColValue(col2 Column, idx int) interface{}

func (*RowsGenerator) GetColValueFromRec

func (g *RowsGenerator) GetColValueFromRec(col record.ColVal, idx int, ref record.Field) (interface{}, bool)

func (*RowsGenerator) Release added in v1.1.0

func (g *RowsGenerator) Release()

func (*RowsGenerator) Reset added in v1.1.0

func (g *RowsGenerator) Reset()

type RuleInstruction

type RuleInstruction struct {
	// contains filtered or unexported fields
}

func NewRuleInstruction

func NewRuleInstruction(ruleCatatory OptRuleCategory) *RuleInstruction

func (*RuleInstruction) Execute

func (g *RuleInstruction) Execute(planner HeuPlanner)

func (*RuleInstruction) Initialize

func (g *RuleInstruction) Initialize(clearCache bool)

func (*RuleInstruction) RuleCategory added in v1.5.0

func (g *RuleInstruction) RuleCategory() OptRuleCategory

func (*RuleInstruction) SkippingGroup added in v1.1.0

func (g *RuleInstruction) SkippingGroup(schema hybridqp.Catalog) bool

type RuleSet

type RuleSet map[OptRule]struct{}

func (RuleSet) Add

func (rs RuleSet) Add(rule OptRule)

func (RuleSet) AddAll

func (rs RuleSet) AddAll(ruleSet RuleSet)

type SampleItem added in v1.3.0

type SampleItem[T util.BasicType] struct {
	// contains filtered or unexported fields
}

func NewSampleItem added in v1.3.0

func NewSampleItem[T util.BasicType](items []PointItem[T]) *SampleItem[T]

func (*SampleItem[T]) Len added in v1.3.0

func (f *SampleItem[T]) Len() int

func (*SampleItem[T]) Less added in v1.3.0

func (f *SampleItem[T]) Less(i, j int) bool

func (*SampleItem[T]) Reset added in v1.3.0

func (f *SampleItem[T]) Reset()

func (*SampleItem[T]) Swap added in v1.3.0

func (f *SampleItem[T]) Swap(i, j int)

type ScalarBuf added in v1.3.0

type ScalarBuf struct {
	// contains filtered or unexported fields
}

type ScalarIterator added in v1.3.0

type ScalarIterator struct {
	// contains filtered or unexported fields
}

func NewScalarIterator added in v1.3.0

func NewScalarIterator(inOrdinal, outOrdinal int) *ScalarIterator

func (*ScalarIterator) Next added in v1.3.0

type Schema added in v1.5.0

type Schema interface {
	Mapping() map[influxql.Expr]influxql.VarRef
}

type Semaphore

type Semaphore int

type Series

type Series struct {
	// Name is the measurement name.
	Name string

	// Tags for the series.
	Tags ChunkTags
	// contains filtered or unexported fields
}

type SeriesRecord added in v1.0.0

type SeriesRecord struct {
	// contains filtered or unexported fields
}

func NewSeriesRecord added in v1.0.0

func NewSeriesRecord(rec *record.Record, sid uint64, file immutable.TSSPFile, seq uint64, tr *util.TimeRange, err error) *SeriesRecord

func (*SeriesRecord) GetErr added in v1.0.0

func (r *SeriesRecord) GetErr() error

func (*SeriesRecord) GetRec added in v1.0.0

func (r *SeriesRecord) GetRec() *record.Record

func (*SeriesRecord) GetSeq added in v1.0.0

func (r *SeriesRecord) GetSeq() uint64

func (*SeriesRecord) GetSid added in v1.0.0

func (r *SeriesRecord) GetSid() uint64

func (*SeriesRecord) GetTr added in v1.0.1

func (r *SeriesRecord) GetTr() *util.TimeRange

func (*SeriesRecord) GetTsspFile added in v1.0.0

func (r *SeriesRecord) GetTsspFile() immutable.TSSPFile

func (*SeriesRecord) SetRec added in v1.0.1

func (r *SeriesRecord) SetRec(re *record.Record)

type SeriesRecordPort added in v1.0.0

type SeriesRecordPort struct {
	RowDataType hybridqp.RowDataType
	State       chan *SeriesRecord
	OrigiState  chan *SeriesRecord
	Redirected  bool
	// contains filtered or unexported fields
}

func NewSeriesRecordPort added in v1.0.0

func NewSeriesRecordPort(rowDataType hybridqp.RowDataType) *SeriesRecordPort

func (*SeriesRecordPort) Close added in v1.0.0

func (p *SeriesRecordPort) Close()

func (*SeriesRecordPort) Connect added in v1.0.0

func (p *SeriesRecordPort) Connect(to Port)

func (*SeriesRecordPort) ConnectWithoutCache added in v1.0.0

func (p *SeriesRecordPort) ConnectWithoutCache(to Port)

func (*SeriesRecordPort) ConnectionId added in v1.0.0

func (p *SeriesRecordPort) ConnectionId() uintptr

func (*SeriesRecordPort) Equal added in v1.0.0

func (p *SeriesRecordPort) Equal(to Port) bool

func (*SeriesRecordPort) Redirect added in v1.0.0

func (p *SeriesRecordPort) Redirect(to Port)

func (*SeriesRecordPort) Release added in v1.0.0

func (p *SeriesRecordPort) Release()

type SetSchemaVisitor

type SetSchemaVisitor struct {
	// contains filtered or unexported fields
}

func (*SetSchemaVisitor) Visit

type ShardFileFragment added in v1.1.0

type ShardFileFragment struct {
	// contains filtered or unexported fields
}

func NewShardFileFragment added in v1.1.0

func NewShardFileFragment(shardId uint64, item FileFragment) *ShardFileFragment

type ShardInfo added in v1.2.0

type ShardInfo struct {
	ID      uint64
	Path    string // used for remote storage
	Version uint32 // identify data from different layouts
}

func UnmarshalShardInfos added in v1.2.0

func UnmarshalShardInfos(shardInfos []*proto2.ShardInfo) []ShardInfo

type ShardsFragments added in v1.1.0

type ShardsFragments map[uint64]*FileFragments

func NewShardsFragments added in v1.1.0

func NewShardsFragments() ShardsFragments

func (*ShardsFragments) String added in v1.1.0

func (sfs *ShardsFragments) String() string

type ShardsFragmentsGroup added in v1.1.0

type ShardsFragmentsGroup struct {
	// contains filtered or unexported fields
}

func NewShardsFragmentsGroup added in v1.1.0

func NewShardsFragmentsGroup(frags ShardsFragments, fragmentCount int64) *ShardsFragmentsGroup

func (*ShardsFragmentsGroup) GetFragmentCount added in v1.1.0

func (sfg *ShardsFragmentsGroup) GetFragmentCount() int64

func (*ShardsFragmentsGroup) GetFrags added in v1.1.0

func (sfg *ShardsFragmentsGroup) GetFrags() ShardsFragments

func (*ShardsFragmentsGroup) MoveTo added in v1.1.0

func (sfg *ShardsFragmentsGroup) MoveTo(des *ShardsFragmentsGroup, id uint64, filePath string)

func (*ShardsFragmentsGroup) MoveToFileNotExist added in v1.1.0

func (sfg *ShardsFragmentsGroup) MoveToFileNotExist(des *ShardsFragmentsGroup)

func (*ShardsFragmentsGroup) SearchAndMoveExistFile added in v1.1.0

func (sfg *ShardsFragmentsGroup) SearchAndMoveExistFile(des *ShardsFragmentsGroup)

type ShardsFragmentsGroups added in v1.1.0

type ShardsFragmentsGroups struct {
	Items []*ShardsFragmentsGroup
	// contains filtered or unexported fields
}

func DistributeFragments added in v1.1.0

func DistributeFragments(frags ShardsFragments, parallel int) (*ShardsFragmentsGroups, error)

DistributeFragments used for balanced fragment allocation of data files. Get a complete view and assign it directly. Considering the total number of fragments, only about the same number of fragments are placed in a group during initialization.

func DistributeFragmentsV2 added in v1.1.0

func DistributeFragmentsV2(frags ShardsFragments, parallel int) (*ShardsFragmentsGroups, error)

DistributeFragmentsV2 allocates fragments based on the parallel number so that the difference number of fragments between two groups is less than or equal to 1.

func NewShardsFragmentsGroups added in v1.1.0

func NewShardsFragmentsGroups(parallel int) *ShardsFragmentsGroups

func (*ShardsFragmentsGroups) Balance added in v1.1.0

func (fgs *ShardsFragmentsGroups) Balance()

func (*ShardsFragmentsGroups) HasGroup added in v1.2.0

func (fgs *ShardsFragmentsGroups) HasGroup() bool

func (*ShardsFragmentsGroups) Len added in v1.1.0

func (fgs *ShardsFragmentsGroups) Len() int

func (*ShardsFragmentsGroups) Less added in v1.1.0

func (fgs *ShardsFragmentsGroups) Less(i, j int) bool

func (*ShardsFragmentsGroups) MoveFrags added in v1.1.0

func (fgs *ShardsFragmentsGroups) MoveFrags(from, to int)

func (*ShardsFragmentsGroups) NextGroup added in v1.2.0

func (fgs *ShardsFragmentsGroups) NextGroup() *ShardsFragmentsGroup

func (*ShardsFragmentsGroups) PeekGroup added in v1.2.0

func (fgs *ShardsFragmentsGroups) PeekGroup() *ShardsFragmentsGroup

func (*ShardsFragmentsGroups) Pop added in v1.1.0

func (fgs *ShardsFragmentsGroups) Pop() interface{}

func (*ShardsFragmentsGroups) Push added in v1.1.0

func (fgs *ShardsFragmentsGroups) Push(x interface{})

func (*ShardsFragmentsGroups) String added in v1.1.0

func (fgs *ShardsFragmentsGroups) String() string

func (*ShardsFragmentsGroups) Swap added in v1.1.0

func (fgs *ShardsFragmentsGroups) Swap(i, j int)

type SinkFromFunction

type SinkFromFunction struct {
	BaseProcessor

	Input    *ChunkPort
	Function func(chunk Chunk) error
}

func NewSinkFromFunction

func NewSinkFromFunction(rowDataType hybridqp.RowDataType, function func(chunk Chunk) error) *SinkFromFunction

func (*SinkFromFunction) Close

func (sink *SinkFromFunction) Close()

func (*SinkFromFunction) Explain

func (sink *SinkFromFunction) Explain() []ValuePair

func (*SinkFromFunction) GetInputNumber

func (sink *SinkFromFunction) GetInputNumber(_ Port) int

func (*SinkFromFunction) GetInputs

func (sink *SinkFromFunction) GetInputs() Ports

func (*SinkFromFunction) GetOutputNumber

func (sink *SinkFromFunction) GetOutputNumber(_ Port) int

func (*SinkFromFunction) GetOutputs

func (sink *SinkFromFunction) GetOutputs() Ports

func (*SinkFromFunction) Name

func (sink *SinkFromFunction) Name() string

func (*SinkFromFunction) Work

func (sink *SinkFromFunction) Work(ctx context.Context) error

type SliceItem added in v1.3.0

type SliceItem[T util.ExceptBool] struct {
	// contains filtered or unexported fields
}

func NewSliceItem added in v1.3.0

func NewSliceItem[T util.ExceptBool]() *SliceItem[T]

func (*SliceItem[T]) AppendItem added in v1.3.0

func (f *SliceItem[T]) AppendItem(c Chunk, ordinal, start, end int, values []T)

func (*SliceItem[T]) Len added in v1.3.0

func (f *SliceItem[T]) Len() int

func (*SliceItem[T]) Less added in v1.3.0

func (f *SliceItem[T]) Less(i, j int) bool

func (*SliceItem[T]) Reset added in v1.3.0

func (f *SliceItem[T]) Reset()

func (*SliceItem[T]) Swap added in v1.3.0

func (f *SliceItem[T]) Swap(i, j int)

type SliceReduce added in v1.3.0

type SliceReduce[T util.NumberOnly] func(floatItem *SliceItem[T]) (index int, time int64, value float64, isNil bool)

func NewPercentileReduce added in v1.3.0

func NewPercentileReduce[T util.NumberOnly](percentile float64) SliceReduce[T]

func NewStdDevReduce added in v1.3.0

func NewStdDevReduce[T util.NumberOnly]() SliceReduce[T]

func NewStdReduce added in v1.3.0

func NewStdReduce(isStddev bool) SliceReduce[float64]

func QuantileReduce added in v1.3.0

func QuantileReduce(percentile float64) SliceReduce[float64]

type SlideWindowSpreadRule

type SlideWindowSpreadRule struct {
	OptRuleBase
}

func NewSlideWindowSpreadRule

func NewSlideWindowSpreadRule(description string) *SlideWindowSpreadRule

func (*SlideWindowSpreadRule) Category added in v1.5.0

func (r *SlideWindowSpreadRule) Category() OptRuleCategory

func (*SlideWindowSpreadRule) Equals

func (r *SlideWindowSpreadRule) Equals(rhs OptRule) bool

func (*SlideWindowSpreadRule) OnMatch

func (r *SlideWindowSpreadRule) OnMatch(call *OptRuleCall)

func (*SlideWindowSpreadRule) ToString

func (r *SlideWindowSpreadRule) ToString() string

type SlidingWindow added in v1.3.0

type SlidingWindow[T util.ExceptString] struct {
	// contains filtered or unexported fields
}

func NewSlidingWindow added in v1.3.0

func NewSlidingWindow[T util.ExceptString](slidingNum int) *SlidingWindow[T]

func (*SlidingWindow[T]) IsNil added in v1.3.0

func (w *SlidingWindow[T]) IsNil() bool

func (*SlidingWindow[T]) Len added in v1.3.0

func (w *SlidingWindow[T]) Len() int

func (*SlidingWindow[T]) Reset added in v1.3.0

func (w *SlidingWindow[T]) Reset()

func (*SlidingWindow[T]) SetPoint added in v1.3.0

func (w *SlidingWindow[T]) SetPoint(value T, isNil bool, index int)

type SlidingWindowBooleanIterator added in v1.3.0

type SlidingWindowBooleanIterator struct {
	// contains filtered or unexported fields
}

func NewSlidingWindowBooleanIterator added in v1.3.0

func NewSlidingWindowBooleanIterator(
	fwr BooleanColBooleanWindowReduce,
	fpm PointMerge[bool],
	fwm WindowMerge[bool],
	inOrdinal, outOrdinal int, slidingNum int,
) *SlidingWindowBooleanIterator

func (*SlidingWindowBooleanIterator) Next added in v1.3.0

type SlidingWindowFloatIterator added in v1.3.0

type SlidingWindowFloatIterator struct {
	// contains filtered or unexported fields
}

func NewSlidingWindowFloatIterator added in v1.3.0

func NewSlidingWindowFloatIterator(
	fwr ColReduceFunc[float64],
	fpm PointMerge[float64],
	fwm WindowMerge[float64],
	inOrdinal, outOrdinal int, slidingNum int,
) *SlidingWindowFloatIterator

func (*SlidingWindowFloatIterator) Next added in v1.3.0

type SlidingWindowIntegerIterator added in v1.3.0

type SlidingWindowIntegerIterator struct {
	// contains filtered or unexported fields
}

func NewSlidingWindowIntegerIterator added in v1.3.0

func NewSlidingWindowIntegerIterator(
	fwr ColReduceFunc[int64],
	fpm PointMerge[int64],
	fwm WindowMerge[int64],
	inOrdinal, outOrdinal int, slidingNum int,
) *SlidingWindowIntegerIterator

func (*SlidingWindowIntegerIterator) Next added in v1.3.0

type SlidingWindowTransform

type SlidingWindowTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewSlidingWindowTransform

func NewSlidingWindowTransform(
	inRowDataType, outRowDataType []hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions,
	opt *query.ProcessorOptions, schema hybridqp.Catalog,
) *SlidingWindowTransform

func (*SlidingWindowTransform) Close

func (trans *SlidingWindowTransform) Close()

func (*SlidingWindowTransform) Explain

func (trans *SlidingWindowTransform) Explain() []ValuePair

func (*SlidingWindowTransform) GetInputNumber

func (trans *SlidingWindowTransform) GetInputNumber(port Port) int

func (*SlidingWindowTransform) GetInputs

func (trans *SlidingWindowTransform) GetInputs() Ports

func (*SlidingWindowTransform) GetOutputNumber

func (trans *SlidingWindowTransform) GetOutputNumber(port Port) int

func (*SlidingWindowTransform) GetOutputs

func (trans *SlidingWindowTransform) GetOutputs() Ports

func (*SlidingWindowTransform) Name

func (trans *SlidingWindowTransform) Name() string

func (*SlidingWindowTransform) Work

func (trans *SlidingWindowTransform) Work(ctx context.Context) error

type SlidingWindowTransformCreator

type SlidingWindowTransformCreator struct {
}

func (*SlidingWindowTransformCreator) Create

type SortAppendTransf added in v1.0.0

type SortAppendTransf struct {
}

func (*SortAppendTransf) CostName added in v1.0.0

func (t *SortAppendTransf) CostName() string

func (*SortAppendTransf) GetType added in v1.0.0

func (t *SortAppendTransf) GetType() MergeTransformType

func (*SortAppendTransf) InitColumnsIteratorHelper added in v1.0.0

func (t *SortAppendTransf) InitColumnsIteratorHelper(rt hybridqp.RowDataType) CoProcessor

func (*SortAppendTransf) InitHeapItems added in v1.0.0

func (t *SortAppendTransf) InitHeapItems(inRowDataLen int, _ hybridqp.RowDataType, schema *QuerySchema) BaseHeapItems

func (*SortAppendTransf) Name added in v1.0.0

func (t *SortAppendTransf) Name() string

type SortAppendTransformCreator

type SortAppendTransformCreator struct {
}

func (*SortAppendTransformCreator) Create

type SortMergeJoinTransform added in v1.5.0

type SortMergeJoinTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewSortMergeJoinTransform added in v1.5.0

func NewSortMergeJoinTransform(inRowDataTypes []hybridqp.RowDataType, outRowDataType hybridqp.RowDataType,
	joinCase *influxql.Join, schema hybridqp.Catalog) (*SortMergeJoinTransform, error)

func (*SortMergeJoinTransform) Close added in v1.5.0

func (trans *SortMergeJoinTransform) Close()

func (*SortMergeJoinTransform) Explain added in v1.5.0

func (trans *SortMergeJoinTransform) Explain() []ValuePair

func (*SortMergeJoinTransform) GetInputNumber added in v1.5.0

func (trans *SortMergeJoinTransform) GetInputNumber(_ Port) int

func (*SortMergeJoinTransform) GetInputs added in v1.5.0

func (trans *SortMergeJoinTransform) GetInputs() Ports

func (*SortMergeJoinTransform) GetOutputNumber added in v1.5.0

func (trans *SortMergeJoinTransform) GetOutputNumber(_ Port) int

func (*SortMergeJoinTransform) GetOutputs added in v1.5.0

func (trans *SortMergeJoinTransform) GetOutputs() Ports

func (*SortMergeJoinTransform) Name added in v1.5.0

func (trans *SortMergeJoinTransform) Name() string

func (*SortMergeJoinTransform) SendChunk added in v1.5.0

func (trans *SortMergeJoinTransform) SendChunk()

func (*SortMergeJoinTransform) Work added in v1.5.0

func (trans *SortMergeJoinTransform) Work(ctx context.Context) error

type SortMergeTransf added in v1.0.0

type SortMergeTransf struct {
}

func (*SortMergeTransf) CostName added in v1.0.0

func (t *SortMergeTransf) CostName() string

func (*SortMergeTransf) GetType added in v1.0.0

func (t *SortMergeTransf) GetType() MergeTransformType

func (*SortMergeTransf) InitColumnsIteratorHelper added in v1.0.0

func (t *SortMergeTransf) InitColumnsIteratorHelper(rt hybridqp.RowDataType) CoProcessor

func (*SortMergeTransf) InitHeapItems added in v1.0.0

func (t *SortMergeTransf) InitHeapItems(inRowDataLen int, rt hybridqp.RowDataType, schema *QuerySchema) BaseHeapItems

func (*SortMergeTransf) Name added in v1.0.0

func (t *SortMergeTransf) Name() string

type SortMergeTransformCreator

type SortMergeTransformCreator struct {
}

func (*SortMergeTransformCreator) Create

type SortTransform added in v1.1.0

type SortTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewSortTransform added in v1.1.0

func NewSortTransform(
	inRowDataType, outRowDataType []hybridqp.RowDataType, s *QuerySchema, sortFields influxql.SortFields) (*SortTransform, error)

func (*SortTransform) Close added in v1.1.0

func (trans *SortTransform) Close()

func (*SortTransform) Explain added in v1.1.0

func (trans *SortTransform) Explain() []ValuePair

func (*SortTransform) GetInputNumber added in v1.1.0

func (trans *SortTransform) GetInputNumber(_ Port) int

func (*SortTransform) GetInputs added in v1.1.0

func (trans *SortTransform) GetInputs() Ports

func (*SortTransform) GetOutputNumber added in v1.1.0

func (trans *SortTransform) GetOutputNumber(_ Port) int

func (*SortTransform) GetOutputs added in v1.1.0

func (trans *SortTransform) GetOutputs() Ports

func (*SortTransform) Name added in v1.1.0

func (trans *SortTransform) Name() string

func (*SortTransform) Release added in v1.2.0

func (trans *SortTransform) Release() error

func (*SortTransform) Work added in v1.1.0

func (trans *SortTransform) Work(ctx context.Context) error

type SortTransformCreator added in v1.1.0

type SortTransformCreator struct {
}

func (*SortTransformCreator) Create added in v1.1.0

type SortedBreakPoint

type SortedBreakPoint struct {
	Tag           ChunkTags
	Time          int64
	ValuePosition int

	AuxCompareHelpers SortedMergeAuxHelpers
	// contains filtered or unexported fields
}

SortedBreakPoint is the point we peek from the 2nd. less chunk, if the value is bigger than the SortedBreakPoint, which means we Need to change the chunk.

type SortedHeapItems

type SortedHeapItems struct {
	Items []*Item

	AuxCompareHelpers SortedMergeAuxHelpers
	// contains filtered or unexported fields
}

func (*SortedHeapItems) GetBreakPoint added in v1.0.0

func (h *SortedHeapItems) GetBreakPoint() BaseBreakPoint

GetBreakPoint used to get the break point of the records

func (*SortedHeapItems) GetOption added in v1.0.0

func (h *SortedHeapItems) GetOption() *query.ProcessorOptions

func (*SortedHeapItems) Len

func (h *SortedHeapItems) Len() int

func (*SortedHeapItems) Less

func (h *SortedHeapItems) Less(i, j int) bool

func (*SortedHeapItems) Pop

func (h *SortedHeapItems) Pop() interface{}

func (*SortedHeapItems) Push

func (h *SortedHeapItems) Push(x interface{})

func (*SortedHeapItems) Swap

func (h *SortedHeapItems) Swap(i, j int)

type SortedMergeAuxHelper

type SortedMergeAuxHelper struct {
	// contains filtered or unexported fields
}

type SortedMergeAuxHelpers

type SortedMergeAuxHelpers []*SortedMergeAuxHelper

func (SortedMergeAuxHelpers) Len

func (s SortedMergeAuxHelpers) Len() int

func (SortedMergeAuxHelpers) Less

func (s SortedMergeAuxHelpers) Less(i, j int) bool

func (SortedMergeAuxHelpers) Swap

func (s SortedMergeAuxHelpers) Swap(i, j int)

type SourceFromSingleChunk

type SourceFromSingleChunk struct {
	BaseProcessor

	Output *ChunkPort
	Chunk  Chunk
}

func NewSourceFromSingleChunk

func NewSourceFromSingleChunk(rowDataType hybridqp.RowDataType, chunk Chunk) *SourceFromSingleChunk

func (*SourceFromSingleChunk) Close

func (source *SourceFromSingleChunk) Close()

func (*SourceFromSingleChunk) Explain

func (source *SourceFromSingleChunk) Explain() []ValuePair

func (*SourceFromSingleChunk) GetInputNumber

func (source *SourceFromSingleChunk) GetInputNumber(_ Port) int

func (*SourceFromSingleChunk) GetInputs

func (source *SourceFromSingleChunk) GetInputs() Ports

func (*SourceFromSingleChunk) GetOutputNumber

func (source *SourceFromSingleChunk) GetOutputNumber(_ Port) int

func (*SourceFromSingleChunk) GetOutputs

func (source *SourceFromSingleChunk) GetOutputs() Ports

func (*SourceFromSingleChunk) Name

func (source *SourceFromSingleChunk) Name() string

func (*SourceFromSingleChunk) Work

func (source *SourceFromSingleChunk) Work(ctx context.Context) error

type Spacer

type Spacer struct {
	Space int
}

func NewSpacer

func NewSpacer() *Spacer

func (*Spacer) Add

func (s *Spacer) Add(num int)

func (*Spacer) String

func (s *Spacer) String() string

func (*Spacer) Sub

func (s *Spacer) Sub(num int)

type SparseIndexScanTransform added in v1.1.0

type SparseIndexScanTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewSparseIndexScanTransform added in v1.1.0

func NewSparseIndexScanTransform(inRowDataType hybridqp.RowDataType, node hybridqp.QueryNode, ops []hybridqp.ExprOptions, info *IndexScanExtraInfo, schema hybridqp.Catalog) *SparseIndexScanTransform

func (*SparseIndexScanTransform) Abort added in v1.2.0

func (trans *SparseIndexScanTransform) Abort()

func (*SparseIndexScanTransform) Close added in v1.1.0

func (trans *SparseIndexScanTransform) Close()

func (*SparseIndexScanTransform) Counting added in v1.1.0

func (trans *SparseIndexScanTransform) Counting(ctx context.Context)

func (*SparseIndexScanTransform) Explain added in v1.1.0

func (trans *SparseIndexScanTransform) Explain() []ValuePair

func (*SparseIndexScanTransform) GetInputNumber added in v1.1.0

func (trans *SparseIndexScanTransform) GetInputNumber(_ Port) int

func (*SparseIndexScanTransform) GetInputs added in v1.1.0

func (trans *SparseIndexScanTransform) GetInputs() Ports

func (*SparseIndexScanTransform) GetOutputNumber added in v1.1.0

func (trans *SparseIndexScanTransform) GetOutputNumber(_ Port) int

func (*SparseIndexScanTransform) GetOutputs added in v1.1.0

func (trans *SparseIndexScanTransform) GetOutputs() Ports

func (*SparseIndexScanTransform) IsSink added in v1.2.0

func (trans *SparseIndexScanTransform) IsSink() bool

func (*SparseIndexScanTransform) Name added in v1.1.0

func (trans *SparseIndexScanTransform) Name() string

func (*SparseIndexScanTransform) QueryPreAgging

func (trans *SparseIndexScanTransform) QueryPreAgging(ctx context.Context)

func (*SparseIndexScanTransform) Release added in v1.1.0

func (trans *SparseIndexScanTransform) Release() error

func (*SparseIndexScanTransform) Running added in v1.1.0

func (trans *SparseIndexScanTransform) Running(ctx context.Context)

func (*SparseIndexScanTransform) SetPreAggRecord

func (trans *SparseIndexScanTransform) SetPreAggRecord(preAggRecord *record.Record)

func (*SparseIndexScanTransform) TransAggToChunk

func (trans *SparseIndexScanTransform) TransAggToChunk() Chunk

func (*SparseIndexScanTransform) Work added in v1.1.0

func (trans *SparseIndexScanTransform) Work(ctx context.Context) error

func (*SparseIndexScanTransform) WorkHelper added in v1.1.0

func (trans *SparseIndexScanTransform) WorkHelper(ctx context.Context) error

type SparseIndexScanTransformCreator added in v1.1.0

type SparseIndexScanTransformCreator struct {
}

func (*SparseIndexScanTransformCreator) Create added in v1.1.0

type SplitTransformTransform added in v0.2.0

type SplitTransformTransform struct {
	BaseProcessor

	CoProcessor CoProcessor
	NewChunk    Chunk
	// contains filtered or unexported fields
}

func NewSplitTransformTransform added in v0.2.0

func NewSplitTransformTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions) *SplitTransformTransform

func (*SplitTransformTransform) Close added in v0.2.0

func (trans *SplitTransformTransform) Close()

func (*SplitTransformTransform) Explain added in v0.2.0

func (trans *SplitTransformTransform) Explain() []ValuePair

func (*SplitTransformTransform) GetInputNumber added in v0.2.0

func (trans *SplitTransformTransform) GetInputNumber(_ Port) int

func (*SplitTransformTransform) GetInputs added in v0.2.0

func (trans *SplitTransformTransform) GetInputs() Ports

func (*SplitTransformTransform) GetOutputNumber added in v0.2.0

func (trans *SplitTransformTransform) GetOutputNumber(_ Port) int

func (*SplitTransformTransform) GetOutputs added in v0.2.0

func (trans *SplitTransformTransform) GetOutputs() Ports

func (*SplitTransformTransform) Name added in v0.2.0

func (trans *SplitTransformTransform) Name() string

func (*SplitTransformTransform) Release added in v0.2.0

func (trans *SplitTransformTransform) Release() error

func (*SplitTransformTransform) SendChunk added in v0.2.0

func (trans *SplitTransformTransform) SendChunk()

func (*SplitTransformTransform) Work added in v0.2.0

func (trans *SplitTransformTransform) Work(ctx context.Context) error

type SplitTransformTransformCreator added in v0.2.0

type SplitTransformTransformCreator struct {
}

func (*SplitTransformTransformCreator) Create added in v0.2.0

type StdoutChunkWriter

type StdoutChunkWriter struct{}

func NewStdoutChunkWriter

func NewStdoutChunkWriter() *StdoutChunkWriter

func (*StdoutChunkWriter) Close

func (w *StdoutChunkWriter) Close()

func (*StdoutChunkWriter) Write

func (w *StdoutChunkWriter) Write(chunk Chunk)

type StoreExchangeTraits

type StoreExchangeTraits struct {
	// contains filtered or unexported fields
}

func NewStoreExchangeTraits

func NewStoreExchangeTraits(w spdy.Responser, mapShardsToReaders map[uint64][][]interface{}) *StoreExchangeTraits

func (*StoreExchangeTraits) HasReader

func (t *StoreExchangeTraits) HasReader() bool

func (*StoreExchangeTraits) HasShard

func (t *StoreExchangeTraits) HasShard() bool

func (*StoreExchangeTraits) NextReader

func (t *StoreExchangeTraits) NextReader() []interface{}

func (*StoreExchangeTraits) NextShard

func (t *StoreExchangeTraits) NextShard() uint64

func (*StoreExchangeTraits) PeekReader

func (t *StoreExchangeTraits) PeekReader() []interface{}

func (*StoreExchangeTraits) PeekShard

func (t *StoreExchangeTraits) PeekShard() uint64

func (*StoreExchangeTraits) Readers

func (t *StoreExchangeTraits) Readers(shard uint64) [][]interface{}

func (*StoreExchangeTraits) Reset

func (t *StoreExchangeTraits) Reset()

type StreamAggregateTransform

type StreamAggregateTransform struct {
	BaseProcessor

	Inputs  ChunkPorts
	Outputs ChunkPorts
	// contains filtered or unexported fields
}

func NewStreamAggregateTransform

func NewStreamAggregateTransform(
	inRowDataType, outRowDataType []hybridqp.RowDataType, exprOpt []hybridqp.ExprOptions, opt *query.ProcessorOptions, schema Schema, isSubQuery bool,
) (*StreamAggregateTransform, error)

func (*StreamAggregateTransform) Close

func (trans *StreamAggregateTransform) Close()

func (*StreamAggregateTransform) Explain

func (trans *StreamAggregateTransform) Explain() []ValuePair

func (*StreamAggregateTransform) GetInputNumber

func (trans *StreamAggregateTransform) GetInputNumber(port Port) int

func (*StreamAggregateTransform) GetInputs

func (trans *StreamAggregateTransform) GetInputs() Ports

func (*StreamAggregateTransform) GetOutputNumber

func (trans *StreamAggregateTransform) GetOutputNumber(port Port) int

func (*StreamAggregateTransform) GetOutputs

func (trans *StreamAggregateTransform) GetOutputs() Ports

func (*StreamAggregateTransform) Name

func (trans *StreamAggregateTransform) Name() string

func (*StreamAggregateTransform) NextChunk added in v1.4.0

func (trans *StreamAggregateTransform) NextChunk()

func (*StreamAggregateTransform) Work

func (trans *StreamAggregateTransform) Work(ctx context.Context) error

type StreamAggregateTransformCreator

type StreamAggregateTransformCreator struct {
}

func (*StreamAggregateTransformCreator) Create

type StringAppendIterator

type StringAppendIterator struct {
	// contains filtered or unexported fields
}

func NewStringAppendIterator

func NewStringAppendIterator() *StringAppendIterator

func (*StringAppendIterator) Next

func (f *StringAppendIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

nolint

type StringColReduceSliceReduce

type StringColReduceSliceReduce func(stringItem *SliceItem[string]) (index int, time int64, value float64, isNil bool)

type StringColStringDistinctIterator

type StringColStringDistinctIterator struct {
	// contains filtered or unexported fields
}

func NewStringColStringDistinctIterator

func NewStringColStringDistinctIterator(
	inOrdinal, outOrdinal int,
) *StringColStringDistinctIterator

func (*StringColStringDistinctIterator) Next

type StringColStringSampleIterator

type StringColStringSampleIterator struct {
	// contains filtered or unexported fields
}

func NewStringColStringSampleIterator

func NewStringColStringSampleIterator(sampleNum int,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *StringColStringSampleIterator

func (*StringColStringSampleIterator) Next

type StringColStringSliceIterator

type StringColStringSliceIterator struct {
	// contains filtered or unexported fields
}

func NewStringColStringSliceIterator

func NewStringColStringSliceIterator(fn StringColReduceSliceReduce,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *StringColStringSliceIterator

func (*StringColStringSliceIterator) Next

type StringFieldValuer added in v1.0.0

type StringFieldValuer struct {
	// contains filtered or unexported fields
}

func (*StringFieldValuer) At added in v1.0.0

func (valuer *StringFieldValuer) At(col Column, pos int, field *influx.Field) bool

type StringIntervalIterator

type StringIntervalIterator struct{}

func NewStringIntervalIterator

func NewStringIntervalIterator() *StringIntervalIterator

func (*StringIntervalIterator) Next

type StringIterator added in v1.3.0

type StringIterator struct {
	// contains filtered or unexported fields
}

func NewStringIterator added in v1.3.0

func NewStringIterator(fn ColReduceFunc[string], fv StringMerge,
	isSingleCall bool, inOrdinal, outOrdinal int, auxProcessor []*AuxProcessor, rowDataType hybridqp.RowDataType,
) *StringIterator

func (*StringIterator) Next added in v1.3.0

type StringLimitIterator

type StringLimitIterator struct {
	// contains filtered or unexported fields
}

func NewStringLimitIterator

func NewStringLimitIterator() *StringLimitIterator

func (*StringLimitIterator) Next

func (f *StringLimitIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type StringLinearFillProcessor

type StringLinearFillProcessor struct {
	// contains filtered or unexported fields
}

func NewStringLinearFillProcessor

func NewStringLinearFillProcessor(inOrdinal, outOrdinal int) *StringLinearFillProcessor

type StringMerge added in v1.3.0

type StringMerge func(prevPoint, currPoint *StringPoint)

type StringMergeIterator

type StringMergeIterator struct {
	BaseMergeIterator
	// contains filtered or unexported fields
}

func NewStringMergeIterator

func NewStringMergeIterator() *StringMergeIterator

func (*StringMergeIterator) Next

func (f *StringMergeIterator) Next(endpoint *IteratorEndpoint, params *IteratorParams)

type StringNullFillProcessor

type StringNullFillProcessor struct {
	// contains filtered or unexported fields
}

func NewStringNullFillProcessor

func NewStringNullFillProcessor(inOrdinal, outOrdinal int) *StringNullFillProcessor

type StringNumberFillProcessor

type StringNumberFillProcessor struct {
	// contains filtered or unexported fields
}

func NewStringNumberFillProcessor

func NewStringNumberFillProcessor(inOrdinal, outOrdinal int) *StringNumberFillProcessor

type StringPoint

type StringPoint struct {
	// contains filtered or unexported fields
}

func (*StringPoint) Assign

func (p *StringPoint) Assign(c *StringPoint)

func (*StringPoint) Reset

func (p *StringPoint) Reset()

func (*StringPoint) Set

func (p *StringPoint) Set(index int, time int64, value string)

type StringPreviousFillProcessor

type StringPreviousFillProcessor struct {
	// contains filtered or unexported fields
}

func NewStringPreviousFillProcessor

func NewStringPreviousFillProcessor(inOrdinal, outOrdinal int) *StringPreviousFillProcessor

type StringTimeColStringIterator

type StringTimeColStringIterator struct {
	// contains filtered or unexported fields
}

func NewStringTimeColStringIterator

func NewStringTimeColStringIterator(
	fn TimeColReduceFunc[string], fv StringTimeColStringMerge, inOrdinal, outOrdinal int,
) *StringTimeColStringIterator

func (*StringTimeColStringIterator) Next

type StringTimeColStringMerge

type StringTimeColStringMerge func(prevPoint, currPoint *StringPoint)

type SubQueryBuilder

type SubQueryBuilder struct {
	// contains filtered or unexported fields
}

func (*SubQueryBuilder) Build

type SubQueryTransform

type SubQueryTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewSubQueryTransform

func NewSubQueryTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions) *SubQueryTransform

func (*SubQueryTransform) Close

func (trans *SubQueryTransform) Close()

func (*SubQueryTransform) Explain

func (trans *SubQueryTransform) Explain() []ValuePair

func (*SubQueryTransform) GetInputNumber

func (trans *SubQueryTransform) GetInputNumber(_ Port) int

func (*SubQueryTransform) GetInputs

func (trans *SubQueryTransform) GetInputs() Ports

func (*SubQueryTransform) GetOutputNumber

func (trans *SubQueryTransform) GetOutputNumber(_ Port) int

func (*SubQueryTransform) GetOutputs

func (trans *SubQueryTransform) GetOutputs() Ports

func (*SubQueryTransform) Name

func (trans *SubQueryTransform) Name() string

func (*SubQueryTransform) Release

func (trans *SubQueryTransform) Release() error

func (*SubQueryTransform) Work

func (trans *SubQueryTransform) Work(ctx context.Context) error

type SubQueryTransformCreator

type SubQueryTransformCreator struct {
}

func (*SubQueryTransformCreator) Create

type TCountMinSketch added in v1.4.0

type TCountMinSketch struct {
	Depth int
	Width int
	Table [][]TCounter
}

func NewTCountMinSketch added in v1.4.0

func NewTCountMinSketch(depth int, widthLog2 int) TCountMinSketch

func (*TCountMinSketch) Add added in v1.4.0

func (cms *TCountMinSketch) Add(indices []int, value TCounter)

func (*TCountMinSketch) Get added in v1.4.0

func (cms *TCountMinSketch) Get(indices []int) TCounter

func (*TCountMinSketch) GetCounter added in v1.4.0

func (cms *TCountMinSketch) GetCounter(depth int, index int) TCounter

func (*TCountMinSketch) Reset added in v1.5.0

func (cms *TCountMinSketch) Reset()

type TCounter added in v1.4.0

type TCounter float64
const NoCounter TCounter = math.MaxInt64

type TDdcm added in v1.4.0

type TDdcm struct {
	Parameters                       TDdcmParameters
	RandomEngine                     *rand.Rand
	TotalCount                       TCounter
	LargeSketch                      TCountMinSketch
	Hashers                          []THasher
	HashersLastLevels                []int
	CycleManager                     TDdcmCycleManagerBase
	SmallSketches                    []TCountMinSketch
	SmallSketchesHasherIds           []int
	SmallSketchesNextSameHasherLevel []int

	// for hash calculation, only surrport single hash per sketch, todo
	PrecalculatedHashes []THashVector
	SmallSketchIndices  []int
	LargeSketchIndices  []int
	SargeSketchHashes   THashVector
	LargeSketchHashes   THashVector
}

func CreateDdcmWithWdm added in v1.4.0

func CreateDdcmWithWdm(failureProbability, wrongAnswerProbability, errRate, minSearchableFrequency float64, seed uint64) *TDdcm

sigma, delta, epsilon, Phi

func NewTDdcm added in v1.4.0

func NewTDdcm(parameters *TDdcmParameters, seed uint64, cycleManager TDdcmCycleManagerBase) *TDdcm

func (*TDdcm) AddKeyCount added in v1.4.0

func (ddcm *TDdcm) AddKeyCount(key TKey, value TCounter)

func (*TDdcm) CalcHashesAndGetIndices added in v1.4.0

func (ddcm *TDdcm) CalcHashesAndGetIndices(key TKey, level int, indices *[]int)

func (*TDdcm) CalcLargeSketchIndices added in v1.4.0

func (ddcm *TDdcm) CalcLargeSketchIndices(key TKey, indices *[]int)

func (*TDdcm) ExtendLevels added in v1.4.0

func (ddcm *TDdcm) ExtendLevels(size int)

func (*TDdcm) GetFrequentKeys added in v1.4.0

func (ddcm *TDdcm) GetFrequentKeys(countLowerBound TCounter) ([]TKey, []TCounter)

func (*TDdcm) GetFrequentKeysOfSize added in v1.4.0

func (ddcm *TDdcm) GetFrequentKeysOfSize(countLowerBound TCounter, size int, keys *[]TKey, values *[]TCounter)

func (*TDdcm) GetKeyCount added in v1.4.0

func (ddcm *TDdcm) GetKeyCount(key TKey) TCounter

func (*TDdcm) GetTotalCount added in v1.4.0

func (ddcm *TDdcm) GetTotalCount() TCounter

func (*TDdcm) Name added in v1.5.0

func (ddcm *TDdcm) Name() string

type TDdcmCycleManagerBase added in v1.4.0

type TDdcmCycleManagerBase interface {
	AddNewLevel() int
	Copy() TDdcmCycleManagerBase
}

type TDdcmCycleManagerWdm added in v1.4.0

type TDdcmCycleManagerWdm struct {
	CyclesNumberExponent       float64
	CurrentNumCyclesFractional float64
	CurrentNumCycles           int
	CurrentCyclePos            int
	CurrentCycleLength         int
	CurrentPosInCycle          int
}

func NewTDdcmCycleManagerWdm added in v1.4.0

func NewTDdcmCycleManagerWdm(cyclesNumberExponent float64, firstCycleLength int) *TDdcmCycleManagerWdm

func (*TDdcmCycleManagerWdm) AddNewLevel added in v1.4.0

func (ddcmw *TDdcmCycleManagerWdm) AddNewLevel() int

func (*TDdcmCycleManagerWdm) Copy added in v1.4.0

type TDdcmParameters added in v1.4.0

type TDdcmParameters struct {
	CountMinSketchWidthLog2  int
	SmallCountMinSketchDepth int
	LargeCountMinSketchDepth int
}

func CalcDdcmParameters added in v1.4.0

func CalcDdcmParameters(errRate, minSearchableFrequency, wrongAnswerProbability float64) (TDdcmParameters, error)

type TExtendedHashValue added in v1.4.0

type TExtendedHashValue uint64
const MersennePrime TExtendedHashValue = (TExtendedHashValue(1) << MersennePower) - 1

type THashValue added in v1.4.0

type THashValue uint32

func ModuleHash added in v1.4.0

func ModuleHash(hash TExtendedHashValue) THashValue

type THashVector added in v1.4.0

type THashVector THashValue

only support single hash per sketch, todo

type THasher added in v1.4.0

type THasher struct {
	Depth                          int
	WidthLog2                      int
	PolynomialHashBases            []TExtendedHashValue
	MultiplicativeHashCoefficients [][]TExtendedHashValue
}

func NewTHasher added in v1.4.0

func NewTHasher(depth int, widthLog2 int, seed uint64) THasher

func (*THasher) GetInitialHashVector added in v1.4.0

func (h *THasher) GetInitialHashVector(hashVector *THashVector)

only support single hash per sketch, todo

func (*THasher) TurnHashesIntoIndices added in v1.4.0

func (h *THasher) TurnHashesIntoIndices(hashes THashVector, indices *[]int)

only support single hash per sketch, todo

func (*THasher) UpdateHashes added in v1.4.0

func (h *THasher) UpdateHashes(key TKey, numBitsBefore int, numBitsAfter int, hashes *THashVector)

only support single hash per sketch, todo

func (*THasher) UpdateHashesBytewise added in v1.4.0

func (h *THasher) UpdateHashesBytewise(key TKey, numBytesBefore int, numBytesAfter int, hashes *THashVector)

only support single hash per sketch, todo

type THeavyHitterInfo added in v1.5.0

type THeavyHitterInfo struct {
	// contains filtered or unexported fields
}

type TKey added in v1.4.0

type TKey struct {
	Data []byte
}

func (*TKey) ChangeBit added in v1.4.0

func (k *TKey) ChangeBit(i int)

func (*TKey) Copy added in v1.4.0

func (k *TKey) Copy() *TKey

func (*TKey) Eq added in v1.5.0

func (k *TKey) Eq(o TKey) bool

func (*TKey) Get added in v1.4.0

func (k *TKey) Get(i int) bool

[] func

func (*TKey) GetByte added in v1.4.0

func (k *TKey) GetByte(i int) byte

func (*TKey) GetByteSize added in v1.5.0

func (k *TKey) GetByteSize() int

func (*TKey) GetSize added in v1.4.0

func (k *TKey) GetSize() int

func (*TKey) Less added in v1.4.0

func (k *TKey) Less(o TKey) bool

type TMajorityDetector added in v1.5.0

type TMajorityDetector struct {
	TotalCount TCounter
	BitCounts  []TCounter
}

func (*TMajorityDetector) AddKeyCount added in v1.5.0

func (detector *TMajorityDetector) AddKeyCount(key *TKey, count TCounter)

func (*TMajorityDetector) GetMajorityKey added in v1.5.0

func (detector *TMajorityDetector) GetMajorityKey() (TKey, bool)

func (*TMajorityDetector) GetTotalCount added in v1.5.0

func (detector *TMajorityDetector) GetTotalCount() TCounter

func (*TMajorityDetector) Reset added in v1.5.0

func (detector *TMajorityDetector) Reset()

type TSmallCounterSet added in v1.5.0

type TSmallCounterSet struct {
	Counters []TCounter
	Min      TCounter
}

func (*TSmallCounterSet) Replace added in v1.5.0

func (set *TSmallCounterSet) Replace(before, after TCounter)

type TSubstream added in v1.5.0

type TSubstream struct {
	// contains filtered or unexported fields
}

func (*TSubstream) Reset added in v1.5.0

func (sub *TSubstream) Reset()

type TTopnCM05Optimized added in v1.5.0

type TTopnCM05Optimized struct {
	Parameters               TTopnCM05OptimizedParameters
	RandomEngine             *rand.Rand // not reuse
	HasherIntoSubstreams     THasher    // not reuse
	Substreams               [][]TSubstream
	FrequencyEstimator       TCountMinSketch
	FrequencyEstimatorHasher THasher // not reuse
	TotalCount               TCounter
	HeavyHitters             map[string]*THeavyHitterInfo
	// TODO: make an id for each heavy hitter and use it here
	KeysByIndices [][]map[string]struct{}
}

func CreateTTopnCM05Optimized added in v1.5.0

func CreateTTopnCM05Optimized(failureProbability, wrongAnswerProbability, errRate, minSearchableFrequency float64, seed uint64) *TTopnCM05Optimized

func DefaultCreateTTopnCM05Optimized added in v1.5.0

func DefaultCreateTTopnCM05Optimized() *TTopnCM05Optimized

func (*TTopnCM05Optimized) AddKeyCount added in v1.5.0

func (nagt *TTopnCM05Optimized) AddKeyCount(key TKey, value TCounter)

func (*TTopnCM05Optimized) GetFrequentKeys added in v1.5.0

func (nagt *TTopnCM05Optimized) GetFrequentKeys(countLowerBound TCounter) ([]TKey, []TCounter)

func (*TTopnCM05Optimized) GetKeyCount added in v1.5.0

func (nagt *TTopnCM05Optimized) GetKeyCount(key TKey) TCounter

func (*TTopnCM05Optimized) GetTotalCount added in v1.5.0

func (nagt *TTopnCM05Optimized) GetTotalCount() TCounter

func (*TTopnCM05Optimized) Instance added in v1.5.0

func (nagt *TTopnCM05Optimized) Instance() *TTopnCM05Optimized

func (*TTopnCM05Optimized) MemSize added in v1.5.0

func (nagt *TTopnCM05Optimized) MemSize() int

func (*TTopnCM05Optimized) Name added in v1.5.0

func (nagt *TTopnCM05Optimized) Name() string

func (*TTopnCM05Optimized) Ready added in v1.5.0

func (nagt *TTopnCM05Optimized) Ready(seed uint64)

func (*TTopnCM05Optimized) Reset added in v1.5.0

func (nagt *TTopnCM05Optimized) Reset()

type TTopnCM05OptimizedParameters added in v1.5.0

type TTopnCM05OptimizedParameters struct {
	// contains filtered or unexported fields
}

func ConstructParametersFromGuarantees added in v1.5.0

func ConstructParametersFromGuarantees(failureProbability, wrongAnswerProbability, errRate, minSearchableFrequency float64) TTopnCM05OptimizedParameters

type TableFunctionFactory added in v1.5.0

type TableFunctionFactory map[string]TableFunctionOperator

type TableFunctionOperator added in v1.5.0

type TableFunctionOperator interface {
	Run(params *TableFunctionParams) ([]Chunk, error)
}

func GetTableFunctionOperator added in v1.5.0

func GetTableFunctionOperator(name string) TableFunctionOperator

type TableFunctionParams added in v1.5.0

type TableFunctionParams struct {
	ChunkPortsWithMetas []*ChunkPortsWithMeta
	Param               string
}

type TableFunctionTransform added in v1.5.0

type TableFunctionTransform struct {
	BaseProcessor

	InputsWithMetas []*ChunkPortsWithMeta
	Output          *ChunkPort

	TableFunctionName string
	// contains filtered or unexported fields
}

func NewTableFunctionTransform added in v1.5.0

func NewTableFunctionTransform(plan LogicalPlan) *TableFunctionTransform

func (*TableFunctionTransform) Close added in v1.5.0

func (trans *TableFunctionTransform) Close()

func (*TableFunctionTransform) Explain added in v1.5.0

func (trans *TableFunctionTransform) Explain() []ValuePair

func (*TableFunctionTransform) GetInputNumber added in v1.5.0

func (trans *TableFunctionTransform) GetInputNumber(port Port) int

func (*TableFunctionTransform) GetInputs added in v1.5.0

func (trans *TableFunctionTransform) GetInputs() Ports

func (*TableFunctionTransform) GetOutputNumber added in v1.5.0

func (trans *TableFunctionTransform) GetOutputNumber(port Port) int

func (*TableFunctionTransform) GetOutputs added in v1.5.0

func (trans *TableFunctionTransform) GetOutputs() Ports

func (*TableFunctionTransform) Name added in v1.5.0

func (trans *TableFunctionTransform) Name() string

func (*TableFunctionTransform) Work added in v1.5.0

func (trans *TableFunctionTransform) Work(ctx context.Context) error

type TableFunctionTransformCreator added in v1.5.0

type TableFunctionTransformCreator struct {
}

func (*TableFunctionTransformCreator) Create added in v1.5.0

type TableScanFromSingleChunk

type TableScanFromSingleChunk struct {
	BaseProcessor

	Output  *ChunkPort
	Table   *QueryTable
	Builder *ChunkBuilder
	Chunk   Chunk
}

func NewTableScanFromSingleChunk

func NewTableScanFromSingleChunk(rowDataType hybridqp.RowDataType, table *QueryTable, _ query.ProcessorOptions) *TableScanFromSingleChunk

func (*TableScanFromSingleChunk) Close

func (scan *TableScanFromSingleChunk) Close()

func (*TableScanFromSingleChunk) Explain

func (scan *TableScanFromSingleChunk) Explain() []ValuePair

func (*TableScanFromSingleChunk) GetInputNumber

func (scan *TableScanFromSingleChunk) GetInputNumber(_ Port) int

func (*TableScanFromSingleChunk) GetInputs

func (scan *TableScanFromSingleChunk) GetInputs() Ports

func (*TableScanFromSingleChunk) GetOutputNumber

func (scan *TableScanFromSingleChunk) GetOutputNumber(_ Port) int

func (*TableScanFromSingleChunk) GetOutputs

func (scan *TableScanFromSingleChunk) GetOutputs() Ports

func (*TableScanFromSingleChunk) Name

func (scan *TableScanFromSingleChunk) Name() string

func (*TableScanFromSingleChunk) Work

type TagValues added in v1.0.0

type TagValues [][]string

func (TagValues) Len added in v1.0.0

func (t TagValues) Len() int

func (TagValues) Less added in v1.0.0

func (t TagValues) Less(i, j int) bool

func (TagValues) Swap added in v1.0.0

func (t TagValues) Swap(i, j int)

type TargetTable added in v1.0.0

type TargetTable struct {
	// contains filtered or unexported fields
}

BatchRows is not thread safe. It can not be used in multi-threading model.

func NewTargetTable added in v1.0.0

func NewTargetTable(rowCap int, tupleCap int) *TargetTable

func (*TargetTable) Active added in v1.0.0

func (tt *TargetTable) Active() []influx.Row

func (*TargetTable) Allocate added in v1.0.0

func (tt *TargetTable) Allocate() (*influx.Row, *TargetTuple)

func (*TargetTable) CheckAndAllocate added in v1.0.0

func (tt *TargetTable) CheckAndAllocate() (*influx.Row, *TargetTuple, bool)

func (*TargetTable) Commit added in v1.0.0

func (tt *TargetTable) Commit()

func (*TargetTable) Instance added in v1.5.0

func (tt *TargetTable) Instance() *TargetTable

func (*TargetTable) MemSize added in v1.5.0

func (tt *TargetTable) MemSize() int

func (*TargetTable) Reset added in v1.0.0

func (tt *TargetTable) Reset()

type TargetTablePool added in v1.0.0

type TargetTablePool struct {
	// contains filtered or unexported fields
}

func NewTargetTablePool added in v1.0.0

func NewTargetTablePool(rowCap int, tupleCap int) *TargetTablePool

func (*TargetTablePool) Get added in v1.0.0

func (p *TargetTablePool) Get() *TargetTable

func (*TargetTablePool) Put added in v1.0.0

func (p *TargetTablePool) Put(table *TargetTable)

type TargetTransform added in v1.0.0

type TargetTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func NewTargetTransform added in v1.0.0

func NewTargetTransform(inRowDataType hybridqp.RowDataType, outRowDataType hybridqp.RowDataType, ops []hybridqp.ExprOptions, opt *query.ProcessorOptions, schema *QuerySchema, mst *influxql.Measurement) (*TargetTransform, error)

func (*TargetTransform) Close added in v1.0.0

func (trans *TargetTransform) Close()

func (*TargetTransform) CloseWriteChan added in v1.0.0

func (trans *TargetTransform) CloseWriteChan()

func (*TargetTransform) Explain added in v1.0.0

func (trans *TargetTransform) Explain() []ValuePair

func (*TargetTransform) GetInputNumber added in v1.0.0

func (trans *TargetTransform) GetInputNumber(_ Port) int

func (*TargetTransform) GetInputs added in v1.0.0

func (trans *TargetTransform) GetInputs() Ports

func (*TargetTransform) GetOutputNumber added in v1.0.0

func (trans *TargetTransform) GetOutputNumber(_ Port) int

func (*TargetTransform) GetOutputs added in v1.0.0

func (trans *TargetTransform) GetOutputs() Ports

func (*TargetTransform) Name added in v1.0.0

func (trans *TargetTransform) Name() string

func (*TargetTransform) Work added in v1.0.0

func (trans *TargetTransform) Work(ctx context.Context) error

type TargetTransformCreator added in v1.0.0

type TargetTransformCreator struct {
}

func (*TargetTransformCreator) Create added in v1.0.0

type TargetTuple added in v1.0.0

type TargetTuple struct {
	// contains filtered or unexported fields
}

func NewTargetTuple added in v1.0.0

func NewTargetTuple(cap int) *TargetTuple

func (*TargetTuple) Active added in v1.0.0

func (t *TargetTuple) Active() []influx.Field

func (*TargetTuple) Allocate added in v1.0.0

func (t *TargetTuple) Allocate() *influx.Field

func (*TargetTuple) CheckAndAllocate added in v1.0.0

func (t *TargetTuple) CheckAndAllocate() (*influx.Field, bool)

func (*TargetTuple) Commit added in v1.0.0

func (t *TargetTuple) Commit()

func (*TargetTuple) Len added in v1.0.0

func (t *TargetTuple) Len() int

func (*TargetTuple) Reset added in v1.0.0

func (t *TargetTuple) Reset()

type TimeColReduceFunc added in v1.3.0

type TimeColReduceFunc[T util.ExceptBool] func(c Chunk, values []T, ordinal, start, end int) (index int, value T, isNil bool)

type TimeFuncState added in v1.1.0

type TimeFuncState uint32

type TopNTransform added in v1.4.0

type TopNTransform struct {
	BaseProcessor
	// contains filtered or unexported fields
}

func (*TopNTransform) Close added in v1.4.0

func (trans *TopNTransform) Close()

func (*TopNTransform) Explain added in v1.4.0

func (trans *TopNTransform) Explain() []ValuePair

func (*TopNTransform) GetInputNumber added in v1.4.0

func (trans *TopNTransform) GetInputNumber(_ Port) int

func (*TopNTransform) GetInputs added in v1.4.0

func (trans *TopNTransform) GetInputs() Ports

func (*TopNTransform) GetOutputNumber added in v1.4.0

func (trans *TopNTransform) GetOutputNumber(_ Port) int

func (*TopNTransform) GetOutputs added in v1.4.0

func (trans *TopNTransform) GetOutputs() Ports

func (*TopNTransform) InitFuncs added in v1.4.0

func (trans *TopNTransform) InitFuncs(inRowDataType hybridqp.RowDataType, fn string, opt hybridqp.ExprOptions) error

func (*TopNTransform) Name added in v1.4.0

func (trans *TopNTransform) Name() string

func (*TopNTransform) Work added in v1.4.0

func (trans *TopNTransform) Work(ctx context.Context) error

DDCM algo for topN problem

type TopoInfo added in v1.5.0

type TopoInfo struct {
	Vertex []GraphNode `json:"vertex"`
	Edges  []GraphEdge `json:"edges"`
}

type TransItem

type TransItem interface {
	AppendItem(Chunk, int, int, int, bool)
	Reset()
	Len() int
	PrevNil() bool
	ResetPrev()
	GetBaseTransData() BaseTransData
}

type TransformCreator

type TransformCreator interface {
	Create(LogicalPlan, *query.ProcessorOptions) (Processor, error)
}

type TransformCreatorFactory

type TransformCreatorFactory struct {
	// contains filtered or unexported fields
}

func GetTransformFactoryInstance

func GetTransformFactoryInstance() *TransformCreatorFactory

func NewTransformCreatorFactory

func NewTransformCreatorFactory() *TransformCreatorFactory

func (*TransformCreatorFactory) Add

func (r *TransformCreatorFactory) Add(name string, creator TransformCreator)

func (*TransformCreatorFactory) Find

type TransformDag

type TransformDag struct {
	// contains filtered or unexported fields
}

func NewTransformDag

func NewTransformDag() *TransformDag

func (*TransformDag) AddEdge

func (dag *TransformDag) AddEdge(from *TransformVertex, to *TransformVertex) bool

func (*TransformDag) AddVertex

func (dag *TransformDag) AddVertex(vertex *TransformVertex) bool

func (*TransformDag) Contains

func (dag *TransformDag) Contains(vertex *TransformVertex) bool

func (*TransformDag) DepthFirstWalkVertex

func (dag *TransformDag) DepthFirstWalkVertex(visitor TransformVertexVisitor, vertex *TransformVertex)

func (*TransformDag) SetVertexToInfo added in v1.0.0

func (dag *TransformDag) SetVertexToInfo(vertex *TransformVertex, info *TransformVertexInfo)

SetVertexToInfo de

func (*TransformDag) WalkVertex

func (dag *TransformDag) WalkVertex(vertex *TransformVertex, fn func(to, from *TransformVertex))

type TransformEdge

type TransformEdge struct {
	// contains filtered or unexported fields
}

func NewTransformEdge

func NewTransformEdge(from *TransformVertex, to *TransformVertex) *TransformEdge

type TransformVertex

type TransformVertex struct {
	// contains filtered or unexported fields
}

func NewTransformVertex

func NewTransformVertex(node hybridqp.QueryNode, transform Processor) *TransformVertex

func (*TransformVertex) GetNode added in v1.5.0

func (t *TransformVertex) GetNode() hybridqp.QueryNode

func (*TransformVertex) GetTransform added in v1.0.0

func (t *TransformVertex) GetTransform() Processor

type TransformVertexInfo

type TransformVertexInfo struct {
	// contains filtered or unexported fields
}

func NewTransformVertexInfo

func NewTransformVertexInfo() *TransformVertexInfo

func (*TransformVertexInfo) AddBackwardEdge

func (info *TransformVertexInfo) AddBackwardEdge(edge *TransformEdge)

func (*TransformVertexInfo) AddDirectEdge

func (info *TransformVertexInfo) AddDirectEdge(edge *TransformEdge)

type TransformVertexVisitor

type TransformVertexVisitor interface {
	Visit(*TransformVertex) TransformVertexVisitor
}

type UnRefDbPt

type UnRefDbPt struct {
	Db string
	Pt uint32
}

type ValueCount added in v1.3.0

type ValueCount struct {
	// contains filtered or unexported fields
}

type ValueKeeper added in v1.5.0

type ValueKeeper struct {
	// contains filtered or unexported fields
}

func GetBooleanAlignValue added in v1.5.0

func GetBooleanAlignValue(inColumn Column, start int, end int) (*ValueKeeper, bool)

func GetFloatAlignValue added in v1.5.0

func GetFloatAlignValue(inColumn Column, start int, end int) (*ValueKeeper, bool)

func GetIntegerAlignValue added in v1.5.0

func GetIntegerAlignValue(inColumn Column, start int, end int) (*ValueKeeper, bool)

func GetStringAlignValue added in v1.5.0

func GetStringAlignValue(inColumn Column, start int, end int) (*ValueKeeper, bool)

func (*ValueKeeper) Reset added in v1.5.0

func (v *ValueKeeper) Reset()

func (*ValueKeeper) SetStringValue added in v1.5.0

func (v *ValueKeeper) SetStringValue(value string)

type ValuePair

type ValuePair struct {
	First  string
	Second interface{}
}

func NewValuePair

func NewValuePair(first string, second interface{}) *ValuePair

type Vertex

type Vertex struct {
	Processor   Processor
	Id          VertexId
	DirectEdges Edges
	BackEdges   Edges
}

func NewVertex

func NewVertex(id VertexId, processor Processor) *Vertex

func (*Vertex) Degree

func (vertex *Vertex) Degree() int

func (*Vertex) Explain

func (vertex *Vertex) Explain(w VertexWriter)

func (*Vertex) InDegree

func (vertex *Vertex) InDegree() int

func (*Vertex) OutDegree

func (vertex *Vertex) OutDegree() int

type VertexId

type VertexId int

type VertexWriter

type VertexWriter interface {
	Explain(*Vertex)
	Item(string, interface{})
	String() string
}

type VertexWriterImpl

type VertexWriterImpl struct {
	DAG     *DAG
	Builder *strings.Builder
	Values  *list.List
	Spacer  *Spacer
}

func NewVertexWriterImpl

func NewVertexWriterImpl(dag *DAG, builder *strings.Builder) *VertexWriterImpl

func (*VertexWriterImpl) Explain

func (w *VertexWriterImpl) Explain(vertex *Vertex)

func (*VertexWriterImpl) ExplainEdges

func (w *VertexWriterImpl) ExplainEdges(edges Edges)

func (*VertexWriterImpl) Item

func (w *VertexWriterImpl) Item(term string, value interface{})

func (*VertexWriterImpl) String

func (w *VertexWriterImpl) String() string

type Vertexs

type Vertexs []*Vertex

type WalkFn

type WalkFn func(vertex *Vertex, m map[VertexId]int) error

type WideCoProcessorImpl added in v0.2.0

type WideCoProcessorImpl struct {
	Routine *WideRoutineImpl
}

func NewWideCoProcessorImpl added in v0.2.0

func NewWideCoProcessorImpl(routine *WideRoutineImpl) *WideCoProcessorImpl

func (*WideCoProcessorImpl) WorkOnChunk added in v0.2.0

func (p *WideCoProcessorImpl) WorkOnChunk(in Chunk, out Chunk, params *IteratorParams)

type WideIterator added in v0.2.0

type WideIterator struct {
	// contains filtered or unexported fields
}

func NewWideIterator added in v0.2.0

func NewWideIterator(fn WideReduce, params ...interface{}) *WideIterator

func (*WideIterator) Next added in v0.2.0

func (r *WideIterator) Next(ie *IteratorEndpoint, p *IteratorParams)

type WideMultiColIterator added in v1.5.0

type WideMultiColIterator struct {
	// contains filtered or unexported fields
}

func NewWideMultiColIterator added in v1.5.0

func NewWideMultiColIterator(fn WideReduce, params ...interface{}) *WideMultiColIterator

func (*WideMultiColIterator) Next added in v1.5.0

type WideReduce added in v0.2.0

type WideReduce func(input []Chunk, out Chunk, p ...interface{}) error

func CastorReduce added in v0.2.0

func CastorReduce(fn recordToChunkFunc) WideReduce

type WideRoutineImpl added in v0.2.0

type WideRoutineImpl struct {
	// contains filtered or unexported fields
}

func NewWideRoutineImpl added in v0.2.0

func NewWideRoutineImpl(iterator Iterator) *WideRoutineImpl

func (*WideRoutineImpl) WorkOnChunk added in v0.2.0

func (r *WideRoutineImpl) WorkOnChunk(in Chunk, out Chunk, params *IteratorParams)

type WindowMerge added in v1.3.0

type WindowMerge[T util.ExceptString] func(prevWindow, currWindow *SlidingWindow[T], fpm PointMerge[T])

Source Files

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL