influxdb/influxql/iterator.gen.go.tmpl

755 lines
19 KiB
Cheetah
Raw Normal View History

2015-11-04 21:06:06 +00:00
package influxql
import (
2015-12-24 04:42:10 +00:00
"container/heap"
2015-11-04 21:06:06 +00:00
"fmt"
"math"
2015-11-04 21:06:06 +00:00
"sort"
"sync"
)
{{range .}}
// {{.Name}}Iterator represents a stream of {{.name}} points.
type {{.Name}}Iterator interface {
Iterator
Next() *{{.Name}}Point
}
// {{.Name}}Iterators represents a list of {{.name}} iterators.
type {{.Name}}Iterators []{{.Name}}Iterator
// Close closes all iterators.
func (a {{.Name}}Iterators) Close() error {
for _, itr := range a {
itr.Close()
}
return nil
}
// new{{.Name}}Iterators converts a slice of Iterator to a slice of {{.Name}}Iterator.
// Panic if any iterator in itrs is not a {{.Name}}Iterator.
func new{{.Name}}Iterators(itrs []Iterator) []{{.Name}}Iterator {
a := make([]{{.Name}}Iterator, len(itrs))
for i, itr := range itrs {
a[i] = itr.({{.Name}}Iterator)
}
return a
}
// buf{{.Name}}Iterator represents a buffered {{.Name}}Iterator.
type buf{{.Name}}Iterator struct {
itr {{.Name}}Iterator
buf *{{.Name}}Point
}
// newBuf{{.Name}}Iterator returns a buffered {{.Name}}Iterator.
func newBuf{{.Name}}Iterator(itr {{.Name}}Iterator) *buf{{.Name}}Iterator {
return &buf{{.Name}}Iterator{
itr: itr,
}
}
// Close closes the underlying iterator.
func (itr *buf{{.Name}}Iterator) Close() error { return itr.itr.Close() }
2015-11-04 21:06:06 +00:00
// peek returns the next point without removing it from the iterator.
func (itr *buf{{.Name}}Iterator) peek() *{{.Name}}Point {
p := itr.Next()
itr.unread(p)
return p
}
// peekTime returns the time of the next point.
2015-11-04 21:06:06 +00:00
// Returns zero time if no more points available.
2015-11-04 21:06:06 +00:00
func (itr *buf{{.Name}}Iterator) peekTime() int64 {
p := itr.peek()
if p == nil {
2015-11-04 21:06:06 +00:00
return ZeroTime
}
2015-11-04 21:06:06 +00:00
return p.Time
2015-11-04 21:06:06 +00:00
}
// Next returns the current buffer, if exists, or calls the underlying iterator.
func (itr *buf{{.Name}}Iterator) Next() *{{.Name}}Point {
if itr.buf != nil {
buf := itr.buf
itr.buf = nil
return buf
}
return itr.itr.Next()
}
// NextInWindow returns the next value if it is between [startTime, endTime).
// If the next value is outside the range then it is moved to the buffer.
func (itr *buf{{.Name}}Iterator) NextInWindow(startTime, endTime int64) *{{.Name}}Point {
v := itr.Next()
if v == nil {
return nil
} else if v.Time < startTime || v.Time >= endTime {
itr.unread(v)
return nil
}
return v
}
// unread sets v to the buffer. It is read on the next call to Next().
func (itr *buf{{.Name}}Iterator) unread(v *{{.Name}}Point) { itr.buf = v }
2015-11-04 21:06:06 +00:00
// {{.name}}MergeIterator represents an iterator that combines multiple {{.name}} iterators.
type {{.name}}MergeIterator struct {
inputs []{{.Name}}Iterator
heap *{{.name}}MergeHeap
2015-11-04 21:06:06 +00:00
2015-11-04 21:06:06 +00:00
// Current iterator and window.
curr *{{.name}}MergeHeapItem
window struct {
startTime int64
endTime int64
2015-11-04 21:06:06 +00:00
}
}
2015-11-04 21:06:06 +00:00
// new{{.Name}}MergeIterator returns a new instance of {{.name}}MergeIterator.
func new{{.Name}}MergeIterator(inputs []{{.Name}}Iterator, opt IteratorOptions) *{{.name}}MergeIterator {
itr := &{{.name}}MergeIterator{
inputs: inputs,
heap: &{{.name}}MergeHeap{
items: make([]*{{.name}}MergeHeapItem, 0, len(inputs)),
opt: opt,
},
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
// Initialize heap items.
for _, input := range inputs {
// Wrap in buffer, ignore any inputs without anymore points.
bufInput := newBuf{{.Name}}Iterator(input)
if bufInput.peek() == nil {
2015-11-04 21:06:06 +00:00
continue
}
2015-11-04 21:06:06 +00:00
// Append to the heap.
itr.heap.items = append(itr.heap.items, &{{.name}}MergeHeapItem{itr: bufInput})
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
heap.Init(itr.heap)
2015-11-04 21:06:06 +00:00
2015-11-04 21:06:06 +00:00
return itr
}
2015-11-04 21:06:06 +00:00
2015-11-04 21:06:06 +00:00
// Close closes the underlying iterators.
func (itr *{{.name}}MergeIterator) Close() error {
for _, input := range itr.inputs {
return input.Close()
}
return nil
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
// Next returns the next point from the iterator.
func (itr *{{.name}}MergeIterator) Next() *{{.Name}}Point {
for {
// Retrieve the next iterator if we don't have one.
if itr.curr == nil {
if len(itr.heap.items) == 0 {
return nil
}
itr.curr = heap.Pop(itr.heap).(*{{.name}}MergeHeapItem)
// Read point and set current window.
p := itr.curr.itr.Next()
itr.window.startTime, itr.window.endTime = itr.heap.opt.Window(p.Time)
return p
}
// Read the next point from the current iterator.
p := itr.curr.itr.Next()
// If there are no more points then remove iterator from heap and find next.
2015-11-04 21:06:06 +00:00
if p == nil {
2015-11-04 21:06:06 +00:00
itr.curr = nil
2015-11-04 21:06:06 +00:00
continue
}
2015-11-04 21:06:06 +00:00
// If it's outside our window then push iterator back on the heap and find new iterator.
if (itr.heap.opt.Ascending && p.Time >= itr.window.endTime) || (!itr.heap.opt.Ascending && p.Time < itr.window.startTime) {
itr.curr.itr.unread(p)
heap.Push(itr.heap, itr.curr)
itr.curr = nil
continue
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
return p
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
2015-11-04 21:06:06 +00:00
// {{.name}}MergeHeap represents a heap of {{.name}}MergeHeapItems.
// Items are sorted by their next window and then by name/tags.
type {{.name}}MergeHeap struct {
opt IteratorOptions
items []*{{.name}}MergeHeapItem
}
func (h {{.name}}MergeHeap) Len() int { return len(h.items) }
func (h {{.name}}MergeHeap) Swap(i, j int) { h.items[i], h.items[j] = h.items[j], h.items[i] }
func (h {{.name}}MergeHeap) Less(i, j int) bool {
x, y := h.items[i].itr.peek(), h.items[j].itr.peek()
xt, _ := h.opt.Window(x.Time)
yt, _ := h.opt.Window(y.Time)
if h.opt.Ascending {
if xt != yt {
return xt < yt
} else if x.Name != y.Name {
return x.Name < y.Name
} else if x.Tags.ID() != y.Tags.ID() {
return x.Tags.ID() < y.Tags.ID()
}
return x.Time < y.Time
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
if xt != yt {
return xt > yt
} else if x.Name != y.Name {
return x.Name > y.Name
} else if x.Tags.ID() != y.Tags.ID() {
return x.Tags.ID() > y.Tags.ID()
}
return x.Time > y.Time
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
func (h *{{.name}}MergeHeap) Push(x interface{}) {
h.items = append(h.items, x.(*{{.name}}MergeHeapItem))
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
func (h *{{.name}}MergeHeap) Pop() interface{} {
old := h.items
n := len(old)
item := old[n-1]
h.items = old[0 : n-1]
return item
2015-11-04 21:06:06 +00:00
}
2015-11-04 21:06:06 +00:00
type {{.name}}MergeHeapItem struct {
itr *buf{{.Name}}Iterator
}
2015-11-04 21:06:06 +00:00
2015-11-04 21:06:06 +00:00
// {{.name}}SortedMergeIterator is an iterator that sorts and merges multiple iterators into one.
type {{.name}}SortedMergeIterator struct {
inputs []{{.Name}}Iterator
opt IteratorOptions
heap {{.name}}SortedMergeHeap
2015-11-04 21:06:06 +00:00
}
2015-12-24 04:42:10 +00:00
// new{{.Name}}SortedMergeIterator returns an instance of {{.name}}SortedMergeIterator.
func new{{.Name}}SortedMergeIterator(inputs []{{.Name}}Iterator, opt IteratorOptions) Iterator {
itr := &{{.name}}SortedMergeIterator{
2015-11-04 21:06:06 +00:00
inputs: inputs,
heap: make({{.name}}SortedMergeHeap, 0, len(inputs)),
2015-12-24 04:42:10 +00:00
opt: opt,
}
// Initialize heap.
for _, input := range inputs {
// Read next point.
p := input.Next()
if p == nil {
continue
}
// Append to the heap.
2015-11-04 21:06:06 +00:00
itr.heap = append(itr.heap, &{{.name}}SortedMergeHeapItem{point: p, itr: input, ascending: opt.Ascending})
2015-12-24 04:42:10 +00:00
}
heap.Init(&itr.heap)
return itr
}
// Close closes the underlying iterators.
2015-11-04 21:06:06 +00:00
func (itr *{{.name}}SortedMergeIterator) Close() error {
for _, input := range itr.inputs {
input.Close()
}
return nil
}
2015-12-24 04:42:10 +00:00
// Next returns the next points from the iterator.
func (itr *{{.name}}SortedMergeIterator) Next() *{{.Name}}Point { return itr.pop() }
// pop returns the next point from the heap.
// Reads the next point from item's cursor and puts it back on the heap.
func (itr *{{.name}}SortedMergeIterator) pop() *{{.Name}}Point {
if len(itr.heap) == 0 {
return nil
}
// Read the next item from the heap.
2015-11-04 21:06:06 +00:00
item := heap.Pop(&itr.heap).(*{{.name}}SortedMergeHeapItem)
2015-12-24 04:42:10 +00:00
// Copy the point for return.
p := item.point.Clone()
// Read the next item from the cursor. Push back to heap if one exists.
if item.point = item.itr.Next(); item.point != nil {
heap.Push(&itr.heap, item)
}
return p
}
2015-11-04 21:06:06 +00:00
// {{.name}}SortedMergeHeap represents a heap of {{.name}}SortedMergeHeapItems.
type {{.name}}SortedMergeHeap []*{{.name}}SortedMergeHeapItem
2015-12-24 04:42:10 +00:00
2015-11-04 21:06:06 +00:00
func (h {{.name}}SortedMergeHeap) Len() int { return len(h) }
func (h {{.name}}SortedMergeHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] }
func (h {{.name}}SortedMergeHeap) Less(i, j int) bool {
2015-12-24 04:42:10 +00:00
x, y := h[i].point, h[j].point
if h[i].ascending {
if x.Name != y.Name {
return x.Name < y.Name
} else if !x.Tags.Equals(&y.Tags) {
return x.Tags.ID() < y.Tags.ID()
}
return x.Time < y.Time
}
if x.Name != y.Name {
return x.Name > y.Name
} else if !x.Tags.Equals(&y.Tags) {
return x.Tags.ID() > y.Tags.ID()
}
return x.Time > y.Time
}
2015-11-04 21:06:06 +00:00
func (h *{{.name}}SortedMergeHeap) Push(x interface{}) {
*h = append(*h, x.(*{{.name}}SortedMergeHeapItem))
2015-12-24 04:42:10 +00:00
}
2015-11-04 21:06:06 +00:00
func (h *{{.name}}SortedMergeHeap) Pop() interface{} {
2015-12-24 04:42:10 +00:00
old := *h
n := len(old)
item := old[n-1]
*h = old[0 : n-1]
return item
}
2015-11-04 21:06:06 +00:00
type {{.name}}SortedMergeHeapItem struct {
2015-12-24 04:42:10 +00:00
point *{{.Name}}Point
itr {{.Name}}Iterator
ascending bool
}
2015-12-24 18:46:31 +00:00
// {{.name}}LimitIterator represents an iterator that limits points per group.
type {{.name}}LimitIterator struct {
input {{.Name}}Iterator
opt IteratorOptions
n int
prev struct {
name string
tags Tags
}
}
// new{{.Name}}LimitIterator returns a new instance of {{.name}}LimitIterator.
func new{{.Name}}LimitIterator(input {{.Name}}Iterator, opt IteratorOptions) *{{.name}}LimitIterator {
return &{{.name}}LimitIterator{
input: input,
opt: opt,
}
}
// Close closes the underlying iterators.
func (itr *{{.name}}LimitIterator) Close() error { return itr.input.Close() }
// Next returns the next point from the iterator.
func (itr *{{.name}}LimitIterator) Next() *{{.Name}}Point {
for {
p := itr.input.Next()
if p == nil {
return nil
}
// Reset window and counter if a new window is encountered.
if p.Name != itr.prev.name || !p.Tags.Equals(&itr.prev.tags) {
itr.prev.name = p.Name
itr.prev.tags = p.Tags
itr.n = 0
}
// Increment counter.
itr.n++
// Read next point if not beyond the offset.
if itr.n <= itr.opt.Offset {
continue
}
// Read next point if we're beyond the limit.
if itr.opt.Limit > 0 && (itr.n-itr.opt.Offset) > itr.opt.Limit {
continue
}
return p
}
}
2015-11-04 21:06:06 +00:00
// {{.name}}JoinIterator represents a join iterator that processes {{.name}} values.
type {{.name}}JoinIterator struct {
input {{.Name}}Iterator
buf *{{.Name}}Point // next value from input
c chan *{{.Name}}Point // streaming output channel
once sync.Once
}
// new{{.Name}}JoinIterator returns a new join iterator that wraps input.
func new{{.Name}}JoinIterator(input {{.Name}}Iterator) *{{.name}}JoinIterator {
return &{{.name}}JoinIterator{
input: input,
c: make(chan *{{.Name}}Point, 1),
}
}
// Close close the iterator.
func (itr *{{.name}}JoinIterator) Close() error {
itr.once.Do(func() { close(itr.c) })
return nil
}
// Next returns the next point from the streaming channel.
func (itr *{{.name}}JoinIterator) Next() *{{.Name}}Point { return <-itr.c }
// loadBuf reads the next value from the input into the buffer.
func (itr *{{.name}}JoinIterator) loadBuf() (t int64, name string, tags Tags) {
if itr.buf != nil {
return itr.buf.Time, itr.buf.Name, itr.buf.Tags
}
itr.buf = itr.input.Next()
if itr.buf == nil {
return ZeroTime, "", Tags{}
}
return itr.buf.Time, itr.buf.Name, itr.buf.Tags
}
// emitAt emits the buffered point if its timestamp equals t.
// Otherwise it emits a null value with the timestamp t.
func (itr *{{.name}}JoinIterator) emitAt(t int64, name string, tags Tags) {
var v *{{.Name}}Point
if itr.buf == nil || itr.buf.Time != t || itr.buf.Name != name || !itr.buf.Tags.Equals(&tags) {
v = &{{.Name}}Point{Name: name, Tags: tags, Time: t, Value: {{.Nil}}}
} else {
v, itr.buf = itr.buf, nil
}
itr.c <- v
}
// {{.name}}AuxIterator represents a {{.name}} implementation of AuxIterator.
type {{.name}}AuxIterator struct {
input *buf{{.Name}}Iterator
output chan *{{.Name}}Point
fields auxIteratorFields
}
func new{{.Name}}AuxIterator(input {{.Name}}Iterator, opt IteratorOptions) *{{.name}}AuxIterator {
itr := &{{.name}}AuxIterator{
input: newBuf{{.Name}}Iterator(input),
output: make(chan *{{.Name}}Point, 1),
fields: newAuxIteratorFields(opt),
}
// Initialize auxilary fields.
if p := itr.input.Next(); p != nil {
itr.output <- p
itr.fields.init(p)
}
go itr.stream()
return itr
}
func (itr *{{.name}}AuxIterator) Close() error { return itr.input.Close() }
func (itr *{{.name}}AuxIterator) Next() *{{.Name}}Point { return <-itr.output }
func (itr *{{.name}}AuxIterator) Iterator(name string) Iterator { return itr.fields.iterator(name) }
2015-11-04 21:06:06 +00:00
func (itr *{{.name}}AuxIterator) CreateIterator(opt IteratorOptions) (Iterator, error) {
expr := opt.Expr
if expr == nil {
panic("unable to create an iterator with no expression from an aux iterator")
}
switch expr := expr.(type) {
case *VarRef:
return itr.fields.iterator(expr.Val), nil
default:
panic(fmt.Sprintf("invalid expression type for an aux iterator: %T", expr))
}
}
func (itr *{{.name}}AuxIterator) FieldDimensions(sources Sources) (fields, dimensions map[string]struct{}, err error) {
panic("not implemented")
}
2015-11-04 21:06:06 +00:00
func (itr *{{.name}}AuxIterator) stream() {
for {
// Read next point.
p := itr.input.Next()
if p == nil {
break
}
// Send point to output and to each field iterator.
itr.output <- p
itr.fields.send(p)
}
close(itr.output)
itr.fields.close()
}
// {{.name}}ChanIterator represents a new instance of {{.name}}ChanIterator.
type {{.name}}ChanIterator struct {
c chan *{{.Name}}Point
once sync.Once
}
func (itr *{{.name}}ChanIterator) Close() error {
itr.once.Do(func() { close(itr.c) })
return nil
}
func (itr *{{.name}}ChanIterator) Next() *{{.Name}}Point { return <-itr.c }
// {{.name}}ReduceIterator executes a reducer for every interval and buffers the result.
type {{.name}}ReduceIterator struct {
input *buf{{.Name}}Iterator
fn {{.name}}ReduceFunc
opt IteratorOptions
points []*{{.Name}}Point
}
// Close closes the iterator and all child iterators.
func (itr *{{.name}}ReduceIterator) Close() error { return itr.input.Close() }
// Next returns the minimum value for the next available interval.
func (itr *{{.name}}ReduceIterator) Next() *{{.Name}}Point {
// Calculate next window if we have no more points.
if len(itr.points) == 0 {
itr.points = itr.reduce()
if len(itr.points) == 0 {
return nil
}
}
// Pop next point off the stack.
p := itr.points[len(itr.points)-1]
itr.points = itr.points[:len(itr.points)-1]
return p
}
// reduce executes fn once for every point in the next window.
// The previous value for the dimension is passed to fn.
func (itr *{{.name}}ReduceIterator) reduce() []*{{.Name}}Point {
// Calculate next window.
2015-11-04 21:06:06 +00:00
startTime, endTime := itr.opt.Window(itr.input.peekTime())
2015-11-04 21:06:06 +00:00
var reduceOptions = reduceOptions{
startTime: startTime,
endTime: endTime,
}
// Create points by tags.
m := make(map[string]*{{.Name}}Point)
for {
// Read next point.
curr := itr.input.NextInWindow(startTime, endTime)
if curr == nil {
break
}
tags := curr.Tags.Subset(itr.opt.Dimensions)
// Pass previous and current points to reducer.
prev := m[tags.ID()]
t, v, aux := itr.fn(prev, curr, &reduceOptions)
if t == ZeroTime {
continue
}
// If previous value didn't exist, create it and copy values.
if prev == nil {
prev = &{{.Name}}Point{Name: curr.Name, Tags: tags}
m[tags.ID()] = prev
}
prev.Time = t
prev.Value = v
prev.Aux = aux
}
// Reverse sort points by name & tag.
keys := make([]string, 0, len(m))
for k := range m {
keys = append(keys, k)
}
sort.Sort(sort.Reverse(sort.StringSlice(keys)))
a := make([]*{{.Name}}Point, len(m))
for i, k := range keys {
a[i] = m[k]
}
return a
}
// {{.name}}ReduceFunc is the function called by a {{.Name}}Point reducer.
type {{.name}}ReduceFunc func(prev, curr *{{.Name}}Point, opt *reduceOptions) (t int64, v {{.Type}}, aux []interface{})
// {{.name}}ReduceSliceIterator executes a reducer on all points in a window and buffers the result.
type {{.name}}ReduceSliceIterator struct {
input *buf{{.Name}}Iterator
fn {{.name}}ReduceSliceFunc
opt IteratorOptions
points []{{.Name}}Point
}
// Close closes the iterator and all child iterators.
func (itr *{{.name}}ReduceSliceIterator) Close() error { return itr.input.Close() }
// Next returns the minimum value for the next available interval.
func (itr *{{.name}}ReduceSliceIterator) Next() *{{.Name}}Point {
// Calculate next window if we have no more points.
if len(itr.points) == 0 {
itr.points = itr.reduce()
if len(itr.points) == 0 {
return nil
}
}
// Pop next point off the stack.
p := itr.points[len(itr.points)-1]
itr.points = itr.points[:len(itr.points)-1]
return &p
}
// reduce executes fn once for every point in the next window.
// The previous value for the dimension is passed to fn.
func (itr *{{.name}}ReduceSliceIterator) reduce() []{{.Name}}Point {
// Calculate next window.
2015-11-04 21:06:06 +00:00
startTime, endTime := itr.opt.Window(itr.input.peekTime())
2015-11-04 21:06:06 +00:00
var reduceOptions = reduceOptions{
startTime: startTime,
endTime: endTime,
}
// Group points by name and tagset.
groups := make(map[string]struct {
name string
tags Tags
points []{{.Name}}Point
})
for {
// Read next point.
p := itr.input.NextInWindow(startTime, endTime)
if p == nil {
break
}
tags := p.Tags.Subset(itr.opt.Dimensions)
// Append point to dimension.
id := tags.ID()
g := groups[id]
g.name = p.Name
g.tags = tags
g.points = append(g.points, *p)
groups[id] = g
}
// Reduce each set into a set of values.
results := make(map[string][]{{.Name}}Point)
for key, g := range groups {
a := itr.fn(g.points, &reduceOptions)
if len(a) == 0 {
continue
}
// Update name and tags for each returned point.
for i := range a {
a[i].Name = g.name
a[i].Tags = g.tags
}
results[key] = a
}
// Reverse sort points by name & tag.
keys := make([]string, 0, len(results))
for k := range results {
keys = append(keys, k)
}
sort.Sort(sort.Reverse(sort.StringSlice(keys)))
// Reverse order points within each key.
a := make([]{{.Name}}Point, 0, len(results))
for _, k := range keys {
for i := len(results[k]) - 1; i >= 0; i-- {
a = append(a, results[k][i])
}
}
return a
}
// {{.name}}ReduceSliceFunc is the function called by a {{.Name}}Point slice reducer.
type {{.name}}ReduceSliceFunc func(a []{{.Name}}Point, opt *reduceOptions) []{{.Name}}Point
2015-11-04 21:06:06 +00:00
// {{.name}}ReduceIterator executes a function to modify an existing point for every
// output of the input iterator.
type {{.name}}TransformIterator struct {
input {{.Name}}Iterator
fn {{.name}}TransformFunc
}
// Close closes the iterator and all child iterators.
func (itr *{{.name}}TransformIterator) Close() error { return itr.input.Close() }
// Next returns the minimum value for the next available interval.
func (itr *{{.name}}TransformIterator) Next() *{{.Name}}Point {
p := itr.input.Next()
if p != nil {
p = itr.fn(p)
}
return p
}
// {{.name}}TransformFunc creates or modifies a point.
// The point passed in may be modified and returned rather than allocating a
// new point if possible.
type {{.name}}TransformFunc func(p *{{.Name}}Point) *{{.Name}}Point
// {{.name}}ReduceIterator executes a function to modify an existing point for every
// output of the input iterator.
type {{.name}}BoolTransformIterator struct {
input {{.Name}}Iterator
fn {{.name}}BoolTransformFunc
}
// Close closes the iterator and all child iterators.
func (itr *{{.name}}BoolTransformIterator) Close() error { return itr.input.Close() }
// Next returns the minimum value for the next available interval.
func (itr *{{.name}}BoolTransformIterator) Next() *BooleanPoint {
p := itr.input.Next()
if p != nil {
return itr.fn(p)
}
return nil
}
// {{.name}}BoolTransformFunc creates or modifies a point.
// The point passed in may be modified and returned rather than allocating a
// new point if possible.
type {{.name}}BoolTransformFunc func(p *{{.Name}}Point) *BooleanPoint
2015-11-04 21:06:06 +00:00
{{end}}