mirror of
https://gitee.com/rulego/streamsql.git
synced 2025-07-13 11:03:50 +00:00
fix:别名映射不正确
This commit is contained in:
@ -42,18 +42,32 @@ type ExpressionEvaluator struct {
|
||||
func NewGroupAggregator(groupFields []string, fieldMap map[string]AggregateType, fieldAlias map[string]string) *GroupAggregator {
|
||||
aggregators := make(map[string]AggregatorFunction)
|
||||
|
||||
// fieldMap的key是别名(输出字段名),value是聚合类型
|
||||
// fieldAlias的key是别名(输出字段名),value是输入字段名
|
||||
for alias, aggType := range fieldMap {
|
||||
aggregators[alias] = CreateBuiltinAggregator(aggType)
|
||||
// 重新组织 fieldMap 和 fieldAlias
|
||||
// 测试中:fieldMap: {"temperature": Sum}, fieldAlias: {"temperature": "temperature_sum"}
|
||||
// 这意味着:输入字段"temperature",聚合类型Sum,输出别名"temperature_sum"
|
||||
|
||||
// 创建新的映射:输出字段名 -> 聚合类型
|
||||
newFieldMap := make(map[string]AggregateType)
|
||||
// 创建新的别名映射:输出字段名 -> 输入字段名
|
||||
newFieldAlias := make(map[string]string)
|
||||
|
||||
for inputField, aggType := range fieldMap {
|
||||
outputField := inputField // 默认输出字段名等于输入字段名
|
||||
if alias, exists := fieldAlias[inputField]; exists {
|
||||
outputField = alias // 如果有别名,使用别名作为输出字段名
|
||||
}
|
||||
|
||||
newFieldMap[outputField] = aggType
|
||||
newFieldAlias[outputField] = inputField
|
||||
aggregators[outputField] = CreateBuiltinAggregator(aggType)
|
||||
}
|
||||
|
||||
return &GroupAggregator{
|
||||
fieldMap: fieldMap, // 别名 -> 聚合类型
|
||||
fieldMap: newFieldMap, // 输出字段名 -> 聚合类型
|
||||
groupFields: groupFields,
|
||||
aggregators: aggregators,
|
||||
groups: make(map[string]map[string]AggregatorFunction),
|
||||
fieldAlias: fieldAlias, // 别名 -> 输入字段名
|
||||
fieldAlias: newFieldAlias, // 输出字段名 -> 输入字段名
|
||||
expressions: make(map[string]*ExpressionEvaluator),
|
||||
}
|
||||
}
|
||||
|
@ -53,7 +53,7 @@ func (a *AnalyticalAggregatorAdapter) Add(value interface{}) {
|
||||
|
||||
// 对于其他分析函数,执行分析函数
|
||||
args := []interface{}{value}
|
||||
a.analFunc.Execute(a.ctx, args)
|
||||
_, _ = a.analFunc.Execute(a.ctx, args)
|
||||
}
|
||||
|
||||
// Result 获取结果
|
||||
|
@ -74,10 +74,37 @@ func (cw *CountingWindow) Start() {
|
||||
cw.dataBuffer = append(cw.dataBuffer, row)
|
||||
cw.count++
|
||||
shouldTrigger := cw.count >= cw.threshold
|
||||
cw.mu.Unlock()
|
||||
// 只有当达到阈值时才触发
|
||||
if shouldTrigger {
|
||||
cw.Trigger()
|
||||
// 在持有锁的情况下立即处理
|
||||
slot := cw.createSlot(cw.dataBuffer[:cw.threshold])
|
||||
for i := range cw.dataBuffer[:cw.threshold] {
|
||||
// 由于Row是值类型,这里需要通过指针来修改Slot字段
|
||||
cw.dataBuffer[i].Slot = slot
|
||||
}
|
||||
data := make([]types.Row, cw.threshold)
|
||||
copy(data, cw.dataBuffer[:cw.threshold])
|
||||
|
||||
if len(cw.dataBuffer) > cw.threshold {
|
||||
remaining := len(cw.dataBuffer) - cw.threshold
|
||||
newBuffer := make([]types.Row, remaining, cw.threshold)
|
||||
copy(newBuffer, cw.dataBuffer[cw.threshold:])
|
||||
cw.dataBuffer = newBuffer
|
||||
} else {
|
||||
cw.dataBuffer = make([]types.Row, 0, cw.threshold)
|
||||
}
|
||||
// 重置计数
|
||||
cw.count = len(cw.dataBuffer)
|
||||
cw.mu.Unlock()
|
||||
|
||||
// 在释放锁后处理回调
|
||||
go func(data []types.Row) {
|
||||
if cw.callback != nil {
|
||||
cw.callback(data)
|
||||
}
|
||||
cw.outputChan <- data
|
||||
}(data)
|
||||
} else {
|
||||
cw.mu.Unlock()
|
||||
}
|
||||
|
||||
case <-cw.ctx.Done():
|
||||
@ -88,32 +115,8 @@ func (cw *CountingWindow) Start() {
|
||||
}
|
||||
|
||||
func (cw *CountingWindow) Trigger() {
|
||||
//cw.triggerChan <- struct{}{}
|
||||
cw.mu.Lock()
|
||||
defer cw.mu.Unlock()
|
||||
|
||||
slot := cw.createSlot(cw.dataBuffer[:cw.threshold])
|
||||
for _, r := range cw.dataBuffer[:cw.threshold] {
|
||||
// 由于Row是值类型,这里需要通过指针来修改Slot字段
|
||||
(&r).Slot = slot
|
||||
}
|
||||
data := cw.dataBuffer[:cw.threshold]
|
||||
if len(cw.dataBuffer) > cw.threshold {
|
||||
remaining := len(cw.dataBuffer) - cw.threshold
|
||||
newBuffer := make([]types.Row, remaining, cw.threshold)
|
||||
copy(newBuffer, cw.dataBuffer[cw.threshold:])
|
||||
cw.dataBuffer = newBuffer
|
||||
} else {
|
||||
cw.dataBuffer = make([]types.Row, 0, cw.threshold)
|
||||
}
|
||||
// 重置计数
|
||||
cw.count = len(cw.dataBuffer)
|
||||
go func(data []types.Row) {
|
||||
if cw.callback != nil {
|
||||
cw.callback(data)
|
||||
}
|
||||
cw.outputChan <- data
|
||||
}(data)
|
||||
// 注意:触发逻辑已合并到Start方法中以避免数据竞争
|
||||
// 这个方法保留是为了满足Window接口要求,但实际触发在Start方法中处理
|
||||
}
|
||||
|
||||
func (cw *CountingWindow) Reset() {
|
||||
|
Reference in New Issue
Block a user