From 79561c266043ed44a43635b3ecca398a50673f71 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 4 Jan 2019 11:04:02 -0500 Subject: [PATCH 01/17] Rebase on master --- src/query/ts/m3db/encoded_series_iterator.go | 40 +++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/src/query/ts/m3db/encoded_series_iterator.go b/src/query/ts/m3db/encoded_series_iterator.go index dfbc8c9502..8361d65cdc 100644 --- a/src/query/ts/m3db/encoded_series_iterator.go +++ b/src/query/ts/m3db/encoded_series_iterator.go @@ -22,7 +22,6 @@ package m3db import ( "math" - "sync" "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" @@ -32,10 +31,11 @@ import ( ) type encodedSeriesIter struct { - mu sync.RWMutex idx int + err error meta block.Metadata bounds models.Bounds + series block.Series seriesMeta []block.SeriesMeta seriesIters []encoding.SeriesIterator consolidator *consolidators.SeriesLookbackConsolidator @@ -60,8 +60,23 @@ func (b *encodedBlock) seriesIter() block.SeriesIter { } } -func (it *encodedSeriesIter) Current() (block.Series, error) { - it.mu.RLock() +func (it *encodedSeriesIter) Err() error { + return it.err +} + +func (it *encodedSeriesIter) Current() block.Series { + return it.series +} + +func (it *encodedSeriesIter) Next() bool { + if it.err != nil { + return false + } + + it.idx++ + next := it.idx < len(it.seriesIters) + it.consolidator.Reset(it.bounds.Start) + iter := it.seriesIters[it.idx] values := make([]float64, it.bounds.Steps()) xts.Memset(values, math.NaN()) @@ -88,9 +103,8 @@ func (it *encodedSeriesIter) Current() (block.Series, error) { } } - if err := iter.Err(); err != nil { - it.mu.RUnlock() - return block.Series{}, err + if it.err = iter.Err(); it.err != nil { + return false } // Consolidate any remaining points iff has not been finished @@ -99,17 +113,7 @@ func (it *encodedSeriesIter) Current() (block.Series, error) { values[i] = it.consolidator.ConsolidateAndMoveToNext() } - series := block.NewSeries(values, it.seriesMeta[it.idx]) - it.mu.RUnlock() - return series, nil -} - -func (it *encodedSeriesIter) Next() bool { - it.mu.Lock() - it.idx++ - next := it.idx < len(it.seriesIters) - it.consolidator.Reset(it.bounds.Start) - it.mu.Unlock() + it.series = block.NewSeries(values, it.seriesMeta[it.idx]) return next } From 22e522d633c0b696f25fc15d6ed51ad753e0136c Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 28 Dec 2018 09:35:01 -0800 Subject: [PATCH 02/17] Refactor iterators WIP --- src/query/block/column.go | 70 +++++++++++------ src/query/block/scalar.go | 40 ++++++---- src/query/block/scalar_test.go | 25 +++--- src/query/block/types.go | 77 ++++++++++--------- src/query/executor/transform/lazy.go | 66 ++++++++++++---- src/query/functions/aggregation/base.go | 12 +-- .../functions/aggregation/count_values.go | 10 +-- src/query/functions/aggregation/take.go | 12 +-- src/query/functions/binary/and.go | 23 +++--- src/query/functions/binary/binary.go | 32 ++++---- src/query/functions/binary/common.go | 10 +-- src/query/functions/binary/or.go | 14 ++-- src/query/functions/linear/base.go | 12 +-- src/query/functions/temporal/base.go | 16 ++-- .../functions/unconsolidated/timestamp.go | 12 +-- src/query/storage/block.go | 22 +++--- src/query/storage/consolidated.go | 26 +++---- .../encoded_unconsolidated_step_iterator.go | 19 ++--- 18 files changed, 280 insertions(+), 218 deletions(-) diff --git a/src/query/block/column.go b/src/query/block/column.go index 624049fe0a..567876ff70 100644 --- a/src/query/block/column.go +++ b/src/query/block/column.go @@ -95,10 +95,12 @@ func (c *columnBlock) Close() error { } type colBlockIter struct { - columns []column - seriesMeta []SeriesMeta - meta Metadata - idx int + idx int + timeForStep time.Time + err error + meta Metadata + seriesMeta []SeriesMeta + columns []column } func (c *colBlockIter) SeriesMeta() []SeriesMeta { @@ -115,23 +117,31 @@ func (c *colBlockIter) StepCount() int { // Next returns true if iterator has more values remaining func (c *colBlockIter) Next() bool { + if c.err != nil { + return false + } + c.idx++ + c.timeForStep, c.err = c.meta.Bounds.TimeForIndex(c.idx) + if c.err != nil { + return false + } + return c.idx < len(c.columns) } +// Next returns true if iterator has more values remaining +func (c *colBlockIter) Err() error { + return c.err +} + // Current returns the current step -func (c *colBlockIter) Current() (Step, error) { +func (c *colBlockIter) Current() Step { col := c.columns[c.idx] - t, err := c.meta.Bounds.TimeForIndex(c.idx) - // TODO: Test panic case - if err != nil { - panic(err) - } - return ColStep{ - time: t, + time: c.timeForStep, values: col.Values, - }, nil + } } // Close frees up resources @@ -210,9 +220,10 @@ type column struct { // columnBlockSeriesIter is used to iterate over a column. Assumes that all columns have the same length type columnBlockSeriesIter struct { - columns []column idx int blockMeta Metadata + values []float64 + columns []column seriesMeta []SeriesMeta } @@ -220,8 +231,18 @@ func (m *columnBlockSeriesIter) Meta() Metadata { return m.blockMeta } -func newColumnBlockSeriesIter(columns []column, blockMeta Metadata, seriesMeta []SeriesMeta) SeriesIter { - return &columnBlockSeriesIter{columns: columns, blockMeta: blockMeta, seriesMeta: seriesMeta, idx: -1} +func newColumnBlockSeriesIter( + columns []column, + blockMeta Metadata, + seriesMeta []SeriesMeta, +) SeriesIter { + return &columnBlockSeriesIter{ + columns: columns, + blockMeta: blockMeta, + seriesMeta: seriesMeta, + idx: -1, + values: make([]float64, len(columns)), + } } func (m *columnBlockSeriesIter) SeriesMeta() []SeriesMeta { @@ -237,19 +258,22 @@ func (m *columnBlockSeriesIter) SeriesCount() int { return len(cols[0].Values) } -func (m *columnBlockSeriesIter) Next() bool { - m.idx++ - return m.idx < m.SeriesCount() +func (m *columnBlockSeriesIter) Err() error { + return nil } -func (m *columnBlockSeriesIter) Current() (Series, error) { +func (m *columnBlockSeriesIter) Next() bool { + m.idx++ cols := m.columns - values := make([]float64, len(cols)) for i := 0; i < len(cols); i++ { - values[i] = cols[i].Values[m.idx] + m.values[i] = cols[i].Values[m.idx] } - return NewSeries(values, m.seriesMeta[m.idx]), nil + return m.idx < m.SeriesCount() +} + +func (m *columnBlockSeriesIter) Current() Series { + return NewSeries(m.values, m.seriesMeta[m.idx]) } // TODO: Actually free resources once we do pooling diff --git a/src/query/block/scalar.go b/src/query/block/scalar.go index fa62b8daa4..6fe4e5ad0c 100644 --- a/src/query/block/scalar.go +++ b/src/query/block/scalar.go @@ -105,9 +105,11 @@ func (b *Scalar) Value(t time.Time) float64 { } type scalarStepIter struct { - s ScalarFunc - meta Metadata numVals, idx int + stepTime time.Time + err error + meta Metadata + s ScalarFunc } // build an empty SeriesMeta @@ -118,29 +120,31 @@ func buildSeriesMeta() SeriesMeta { } func (it *scalarStepIter) Close() { /* No-op*/ } +func (it *scalarStepIter) Err() error { return it.err } func (it *scalarStepIter) StepCount() int { return it.numVals } func (it *scalarStepIter) SeriesMeta() []SeriesMeta { return []SeriesMeta{buildSeriesMeta()} } func (it *scalarStepIter) Meta() Metadata { return it.meta } func (it *scalarStepIter) Next() bool { - it.idx++ - return it.idx < it.numVals -} - -func (it *scalarStepIter) Current() (Step, error) { - if it.idx >= it.numVals || it.idx < 0 { - return nil, fmt.Errorf("invalid scalar index: %d, numVals: %d", it.idx, it.numVals) + if it.err != nil { + return false } - t, err := it.Meta().Bounds.TimeForIndex(it.idx) - if err != nil { - return nil, err + it.idx++ + it.stepTime, it.err = it.Meta().Bounds.TimeForIndex(it.idx) + if it.err != nil { + return false } + return it.idx < it.numVals +} + +func (it *scalarStepIter) Current() Step { + t := it.stepTime return &scalarStep{ vals: []float64{it.s(t)}, time: t, - }, nil + } } type scalarStep struct { @@ -158,6 +162,7 @@ type scalarSeriesIter struct { } func (it *scalarSeriesIter) Close() { /* No-op*/ } +func (it *scalarSeriesIter) Err() error { return nil } func (it *scalarSeriesIter) SeriesCount() int { return 1 } func (it *scalarSeriesIter) SeriesMeta() []SeriesMeta { return []SeriesMeta{buildSeriesMeta()} } func (it *scalarSeriesIter) Meta() Metadata { return it.meta } @@ -166,12 +171,15 @@ func (it *scalarSeriesIter) Next() bool { return it.idx == 0 } -func (it *scalarSeriesIter) Current() (Series, error) { +func (it *scalarSeriesIter) Current() Series { if it.idx != 0 { - return Series{}, fmt.Errorf("invalid scalar index: %d, numVals: 1", it.idx) + // Indicates an error with the caller, having either not called Next() or attempted + // to get Current after Next() is false + panic("scalar iterator out of bounds") } + return Series{ Meta: buildSeriesMeta(), values: it.vals, - }, nil + } } diff --git a/src/query/block/scalar_test.go b/src/query/block/scalar_test.go index c1a0ae871d..86caa503fe 100644 --- a/src/query/block/scalar_test.go +++ b/src/query/block/scalar_test.go @@ -52,14 +52,13 @@ func TestScalarBlock(t *testing.T) { verifyMetas(t, stepIter.Meta(), stepIter.SeriesMeta()) assert.Equal(t, 6, stepIter.StepCount()) - v, err := stepIter.Current() - require.Error(t, err) + v := stepIter.Current() + require.Error(t, stepIter.Err()) require.Nil(t, v) valCounts := 0 for stepIter.Next() { - v, err = stepIter.Current() - require.NoError(t, err) + v = stepIter.Current() require.NotNil(t, v) expectedTime := start.Add(time.Duration(valCounts) * 10 * time.Second) @@ -71,9 +70,11 @@ func TestScalarBlock(t *testing.T) { valCounts++ } + + require.NoError(t, stepIter.Err()) assert.Equal(t, 6, valCounts) - v, err = stepIter.Current() - require.Error(t, err) + v = stepIter.Current() + require.Error(t, stepIter.Err()) require.Nil(t, v) seriesIter, err := block.SeriesIter() @@ -83,12 +84,12 @@ func TestScalarBlock(t *testing.T) { verifyMetas(t, seriesIter.Meta(), seriesIter.SeriesMeta()) require.Equal(t, 1, seriesIter.SeriesCount()) - series, err := seriesIter.Current() - require.Error(t, err) + series := seriesIter.Current() + require.Error(t, seriesIter.Err()) require.True(t, seriesIter.Next()) - series, err = seriesIter.Current() - require.NoError(t, err) + series = seriesIter.Current() + require.NoError(t, seriesIter.Err()) assert.Equal(t, 6, series.Len()) vals := series.Values() @@ -101,9 +102,9 @@ func TestScalarBlock(t *testing.T) { assert.Equal(t, "", series.Meta.Name) require.False(t, seriesIter.Next()) - series, err = seriesIter.Current() - require.Error(t, err) + require.Error(t, seriesIter.Err()) + series = seriesIter.Current() err = block.Close() require.NoError(t, err) } diff --git a/src/query/block/types.go b/src/query/block/types.go index fc9acb97d6..8ef96f3263 100644 --- a/src/query/block/types.go +++ b/src/query/block/types.go @@ -30,14 +30,16 @@ import ( "github.com/m3db/m3/src/query/ts" ) -// Block represents a group of series across a time bound +// Block represents a group of series across a time bound. type Block interface { io.Closer - // Unconsolidated returns the unconsolidated version of the block + // Unconsolidated returns the unconsolidated version of the block. Unconsolidated() (UnconsolidatedBlock, error) - // StepIter returns a StepIterator + // StepIter returns a step-wise block iterator, giving consolidated values + // across all series comprising the box at a single time step. StepIter() (StepIter, error) - // SeriesIter returns a SeriesIterator + // SeriesIter returns a series-wise block iterator, giving consolidated values + // by series. SeriesIter() (SeriesIter, error) // WithMetadata returns a block with updated meta and series metadata. WithMetadata(Metadata, []SeriesMeta) (Block, error) @@ -46,106 +48,109 @@ type Block interface { // UnconsolidatedBlock represents a group of unconsolidated series across a time bound type UnconsolidatedBlock interface { io.Closer - // StepIter returns an UnconsolidatedStepIter + // StepIter returns a step-wise block iterator, giving unconsolidated values + // across all series comprising the box at a single time step. StepIter() (UnconsolidatedStepIter, error) - // SeriesIter returns an UnconsolidatedSeriesIter + // SeriesIter returns a series-wise block iterator, giving unconsolidated + // by series. SeriesIter() (UnconsolidatedSeriesIter, error) - // Consolidate an unconsolidated block + // Consolidate attempts to consolidate the unconsolidated block. Consolidate() (Block, error) // WithMetadata returns a block with updated meta and series metadata. WithMetadata(Metadata, []SeriesMeta) (UnconsolidatedBlock, error) } -// SeriesMeta is metadata data for the series +// SeriesMeta is metadata data for the series. type SeriesMeta struct { Tags models.Tags Name string } -// Iterator is the base iterator +// Iterator is the base iterator. type Iterator interface { Next() bool + Err() error Close() } -// MetaIter is implemented by iterators which provide meta information +// MetaIter is implemented by iterators which provide meta information. type MetaIter interface { - // SeriesMeta returns the metadata for each series in the block + // SeriesMeta returns the metadata for each series in the block. SeriesMeta() []SeriesMeta - // Meta returns the metadata for the block + // Meta returns the metadata for the block. Meta() Metadata } -// SeriesMetaIter is implemented by series iterators which provide meta information +// SeriesMetaIter is implemented by series iterators which provide meta information. type SeriesMetaIter interface { MetaIter - // SeriesCount returns the number of series + // SeriesCount returns the number of series. SeriesCount() int } -// SeriesIter iterates through a block horizontally +// SeriesIter iterates through a block horizontally. type SeriesIter interface { Iterator SeriesMetaIter - // Current returns the current series for the block - Current() (Series, error) + // Current returns the current series for the block. + Current() Series } -// UnconsolidatedSeriesIter iterates through a block horizontally +// UnconsolidatedSeriesIter iterates through a block horizontally. type UnconsolidatedSeriesIter interface { Iterator SeriesMetaIter - // Current returns the current series for the block - Current() (UnconsolidatedSeries, error) + // Current returns the current series for the block. + Current() UnconsolidatedSeries } -// StepMetaIter is implemented by step iterators which provide meta information +// StepMetaIter is implemented by step iterators which provide meta information. type StepMetaIter interface { MetaIter - // StepCount returns the number of steps + // StepCount returns the number of steps. StepCount() int } -// StepIter iterates through a block vertically +// StepIter iterates through a block vertically. type StepIter interface { Iterator StepMetaIter - // Current returns the current step for the block - Current() (Step, error) + // Current returns the current step for the block. + Current() Step } -// UnconsolidatedStepIter iterates through a block vertically +// UnconsolidatedStepIter iterates through a block vertically. type UnconsolidatedStepIter interface { Iterator StepMetaIter - // Current returns the current step for the block - Current() (UnconsolidatedStep, error) + // Current returns the current step for the block. + Current() UnconsolidatedStep } -// Step is a single time step within a block +// Step is a single time step within a block. type Step interface { Time() time.Time Values() []float64 } -// UnconsolidatedStep is a single unconsolidated time step within a block +// UnconsolidatedStep is a single unconsolidated time step within a block. type UnconsolidatedStep interface { Time() time.Time Values() []ts.Datapoints } -// Metadata is metadata for a block +// Metadata is metadata for a block. type Metadata struct { Bounds models.Bounds Tags models.Tags // Common tags across different series } -// String returns a string representation of metadata +// String returns a string representation of metadata. func (m Metadata) String() string { return fmt.Sprintf("Bounds: %v, Tags: %v", m.Bounds, m.Tags) } -// Builder builds a new block +// Builder builds a new block. type Builder interface { AppendValue(idx int, value float64) error AppendValues(idx int, values []float64) error @@ -153,15 +158,15 @@ type Builder interface { AddCols(num int) error } -// Result is the result from a block query +// Result is the result from a block query. type Result struct { Blocks []Block } -// ConsolidationFunc consolidates a bunch of datapoints into a single float value +// ConsolidationFunc consolidates a bunch of datapoints into a single float value. type ConsolidationFunc func(datapoints ts.Datapoints) float64 -// TakeLast is a consolidation function which takes the last datapoint which has non nan value +// TakeLast is a consolidation function which takes the last datapoint which has non nan value. func TakeLast(values ts.Datapoints) float64 { for i := len(values) - 1; i >= 0; i-- { if !math.IsNaN(values[i].Value) { diff --git a/src/query/executor/transform/lazy.go b/src/query/executor/transform/lazy.go index b18c65cfce..b4c9fab9bd 100644 --- a/src/query/executor/transform/lazy.go +++ b/src/query/executor/transform/lazy.go @@ -69,7 +69,9 @@ func (f *lazyNode) Process(ID parser.NodeID, block block.Block) error { } type stepIter struct { + err error node StepNode + step block.Step iter block.StepIter } @@ -86,25 +88,42 @@ func (s *stepIter) StepCount() int { } func (s *stepIter) Next() bool { - return s.iter.Next() + if s.err != nil { + return false + } + + next := s.iter.Next() + step := s.iter.Current() + s.step, s.err = s.node.ProcessStep(step) + if s.err != nil { + return false + } + + if err := s.iter.Err(); err != nil { + s.err = err + return false + } + + return next } func (s *stepIter) Close() { s.iter.Close() } -func (s *stepIter) Current() (block.Step, error) { - bStep, err := s.iter.Current() - if err != nil { - return nil, err - } +func (s *stepIter) Err() error { + return s.err +} - return s.node.ProcessStep(bStep) +func (s *stepIter) Current() block.Step { + return s.step } type seriesIter struct { - node SeriesNode - iter block.SeriesIter + err error + series block.Series + node SeriesNode + iter block.SeriesIter } func (s *seriesIter) Meta() block.Metadata { @@ -123,17 +142,32 @@ func (s *seriesIter) Close() { s.iter.Close() } -func (s *seriesIter) Current() (block.Series, error) { - bSeries, err := s.iter.Current() - if err != nil { - return block.Series{}, err - } +func (s *seriesIter) Err() error { + return s.err +} - return s.node.ProcessSeries(bSeries) +func (s *seriesIter) Current() block.Series { + return s.series } func (s *seriesIter) Next() bool { - return s.iter.Next() + if s.err != nil { + return false + } + + next := s.iter.Next() + step := s.iter.Current() + s.series, s.err = s.node.ProcessSeries(step) + if s.err != nil { + return false + } + + if err := s.iter.Err(); err != nil { + s.err = err + return false + } + + return next } type lazyBlock struct { diff --git a/src/query/functions/aggregation/base.go b/src/query/functions/aggregation/base.go index 58518163ad..540fc3693b 100644 --- a/src/query/functions/aggregation/base.go +++ b/src/query/functions/aggregation/base.go @@ -131,17 +131,13 @@ func (n *baseNode) Process(ID parser.NodeID, b block.Block) error { return err } - if err := builder.AddCols(stepIter.StepCount()); err != nil { + if err = builder.AddCols(stepIter.StepCount()); err != nil { return err } aggregatedValues := make([]float64, len(buckets)) for index := 0; stepIter.Next(); index++ { - step, err := stepIter.Current() - if err != nil { - return err - } - + step := stepIter.Current() values := step.Values() for i, bucket := range buckets { aggregatedValues[i] = n.op.aggFn(values, bucket) @@ -150,6 +146,10 @@ func (n *baseNode) Process(ID parser.NodeID, b block.Block) error { builder.AppendValues(index, aggregatedValues) } + if err = stepIter.Err(); err != nil { + return err + } + nextBlock := builder.Build() defer nextBlock.Close() return n.controller.Process(nextBlock) diff --git a/src/query/functions/aggregation/count_values.go b/src/query/functions/aggregation/count_values.go index 1a117832c9..3aaabafb67 100644 --- a/src/query/functions/aggregation/count_values.go +++ b/src/query/functions/aggregation/count_values.go @@ -164,11 +164,7 @@ func (n *countValuesNode) Process(ID parser.NodeID, b block.Block) error { } for columnIndex := 0; stepIter.Next(); columnIndex++ { - step, err := stepIter.Current() - if err != nil { - return err - } - + step := stepIter.Current() values := step.Values() for bucketIndex, bucket := range buckets { processBlockBucketAtColumn( @@ -180,6 +176,10 @@ func (n *countValuesNode) Process(ID parser.NodeID, b block.Block) error { } } + if err = stepIter.Err(); err != nil { + return err + } + numSeries := 0 for _, bucketBlock := range intermediateBlock { numSeries += bucketBlock.columnLength diff --git a/src/query/functions/aggregation/take.go b/src/query/functions/aggregation/take.go index 3b1524e5cb..23d7bf2caa 100644 --- a/src/query/functions/aggregation/take.go +++ b/src/query/functions/aggregation/take.go @@ -133,21 +133,21 @@ func (n *takeNode) Process(ID parser.NodeID, b block.Block) error { return err } - if err := builder.AddCols(stepIter.StepCount()); err != nil { + if err = builder.AddCols(stepIter.StepCount()); err != nil { return err } for index := 0; stepIter.Next(); index++ { - step, err := stepIter.Current() - if err != nil { - return err - } - + step := stepIter.Current() values := step.Values() aggregatedValues := n.op.takeFunc(values, buckets) builder.AppendValues(index, aggregatedValues) } + if err = stepIter.Err(); err != nil { + return err + } + nextBlock := builder.Build() defer nextBlock.Close() return n.controller.Process(nextBlock) diff --git a/src/query/functions/binary/and.go b/src/query/functions/binary/and.go index 7ff33b8ac1..d62981d771 100644 --- a/src/query/functions/binary/and.go +++ b/src/query/functions/binary/and.go @@ -43,25 +43,16 @@ func makeAndBlock( return nil, err } - if err := builder.AddCols(lIter.StepCount()); err != nil { + if err = builder.AddCols(lIter.StepCount()); err != nil { return nil, err } intersection := andIntersect(matching, lIter.SeriesMeta(), rIter.SeriesMeta()) for index := 0; lIter.Next() && rIter.Next(); index++ { - lStep, err := lIter.Current() - if err != nil { - return nil, err - } - + lStep := lIter.Current() lValues := lStep.Values() - rStep, err := rIter.Current() - if err != nil { - return nil, err - } - + rStep := rIter.Current() rValues := rStep.Values() - for idx, value := range lValues { rIdx := intersection[idx] if rIdx < 0 || math.IsNaN(rValues[rIdx]) { @@ -73,6 +64,14 @@ func makeAndBlock( } } + if err = lIter.Err(); err != nil { + return nil, err + } + + if err = rIter.Err(); err != nil { + return nil, err + } + return builder.Build(), nil } diff --git a/src/query/functions/binary/binary.go b/src/query/functions/binary/binary.go index 8259c054b4..67d427191d 100644 --- a/src/query/functions/binary/binary.go +++ b/src/query/functions/binary/binary.go @@ -134,22 +134,22 @@ func processSingleBlock( return nil, err } - if err := builder.AddCols(it.StepCount()); err != nil { + if err = builder.AddCols(it.StepCount()); err != nil { return nil, err } for index := 0; it.Next(); index++ { - step, err := it.Current() - if err != nil { - return nil, err - } - + step := it.Current() values := step.Values() for _, value := range values { builder.AppendValue(index, fn(value)) } } + if err = it.Err(); err != nil { + return nil, err + } + return builder.Build(), nil } @@ -183,17 +183,9 @@ func processBothSeries( } for index := 0; lIter.Next() && rIter.Next(); index++ { - lStep, err := lIter.Current() - if err != nil { - return nil, err - } - - rStep, err := rIter.Current() - if err != nil { - return nil, err - } - + lStep := lIter.Current() lValues := lStep.Values() + rStep := rIter.Current() rValues := rStep.Values() for seriesIdx, lIdx := range takeLeft { @@ -205,6 +197,14 @@ func processBothSeries( } } + if err = lIter.Err(); err != nil { + return nil, err + } + + if err = rIter.Err(); err != nil { + return nil, err + } + return builder.Build(), nil } diff --git a/src/query/functions/binary/common.go b/src/query/functions/binary/common.go index f9ec76c1da..13f6f7e70a 100644 --- a/src/query/functions/binary/common.go +++ b/src/query/functions/binary/common.go @@ -153,16 +153,16 @@ func combineMetaAndSeriesMeta( func appendValuesAtIndices(idxArray []int, iter block.StepIter, builder block.Builder) error { for index := 0; iter.Next(); index++ { - step, err := iter.Current() - if err != nil { - return err - } - + step := iter.Current() values := step.Values() for _, idx := range idxArray { builder.AppendValue(index, values[idx]) } } + if err := iter.Err(); err != nil { + return err + } + return nil } diff --git a/src/query/functions/binary/or.go b/src/query/functions/binary/or.go index 8acc437078..4db009e153 100644 --- a/src/query/functions/binary/or.go +++ b/src/query/functions/binary/or.go @@ -55,22 +55,22 @@ func makeOrBlock( return nil, err } - if err := builder.AddCols(lIter.StepCount()); err != nil { + if err = builder.AddCols(lIter.StepCount()); err != nil { return nil, err } index := 0 for ; lIter.Next(); index++ { - lStep, err := lIter.Current() - if err != nil { - return nil, err - } - + lStep := lIter.Current() lValues := lStep.Values() builder.AppendValues(index, lValues) } - if err := appendValuesAtIndices(missingIndices, rIter, builder); err != nil { + if err = lIter.Err(); err != nil { + return nil, err + } + + if err = appendValuesAtIndices(missingIndices, rIter, builder); err != nil { return nil, err } diff --git a/src/query/functions/linear/base.go b/src/query/functions/linear/base.go index ff924b8603..b07196ab18 100644 --- a/src/query/functions/linear/base.go +++ b/src/query/functions/linear/base.go @@ -91,22 +91,22 @@ func (c *baseNode) Process(ID parser.NodeID, b block.Block) error { return err } - if err := builder.AddCols(stepIter.StepCount()); err != nil { + if err = builder.AddCols(stepIter.StepCount()); err != nil { return err } for index := 0; stepIter.Next(); index++ { - step, err := stepIter.Current() - if err != nil { - return err - } - + step := stepIter.Current() values := c.processor.Process(step.Values()) for _, value := range values { builder.AppendValue(index, value) } } + if err = stepIter.Err(); err != nil { + return err + } + nextBlock := builder.Build() defer nextBlock.Close() return c.controller.Process(nextBlock) diff --git a/src/query/functions/temporal/base.go b/src/query/functions/temporal/base.go index 01d263211a..56be2b20f5 100644 --- a/src/query/functions/temporal/base.go +++ b/src/query/functions/temporal/base.go @@ -280,19 +280,11 @@ func (c *baseNode) processSingleRequest(request processRequest) error { return fmt.Errorf("incorrect number of series for block: %d", i) } - s, err := iter.Current() - if err != nil { - return err - } - + s := iter.Current() values = append(values, s.Datapoints()...) } - series, err := seriesIter.Current() - if err != nil { - return err - } - + series := seriesIter.Current() for i := 0; i < series.Len(); i++ { val := series.DatapointsAtStep(i) values = append(values, val) @@ -326,6 +318,10 @@ func (c *baseNode) processSingleRequest(request processRequest) error { } } + if err = seriesIter.Err(); err != nil { + return err + } + nextBlock := builder.Build() defer nextBlock.Close() return c.controller.Process(nextBlock) diff --git a/src/query/functions/unconsolidated/timestamp.go b/src/query/functions/unconsolidated/timestamp.go index f3e74644ab..359e5926ed 100644 --- a/src/query/functions/unconsolidated/timestamp.go +++ b/src/query/functions/unconsolidated/timestamp.go @@ -99,16 +99,12 @@ func (n *timestampNode) Process(ID parser.NodeID, b block.Block) error { return err } - if err := builder.AddCols(iter.StepCount()); err != nil { + if err = builder.AddCols(iter.StepCount()); err != nil { return err } for index := 0; iter.Next(); index++ { - step, err := iter.Current() - if err != nil { - return err - } - + step := iter.Current() values := make([]float64, len(step.Values())) ts.Memset(values, math.NaN()) for i, dps := range step.Values() { @@ -124,6 +120,10 @@ func (n *timestampNode) Process(ID parser.NodeID, b block.Block) error { } } + if err = iter.Err(); err != nil { + return err + } + nextBlock := builder.Build() defer nextBlock.Close() diff --git a/src/query/storage/block.go b/src/query/storage/block.go index c53a640340..86c6e89e37 100644 --- a/src/query/storage/block.go +++ b/src/query/storage/block.go @@ -268,10 +268,11 @@ func (m *multiSeriesBlockStepIter) Next() bool { return m.index < len(m.values[0]) } -func (m *multiSeriesBlockStepIter) Current() ( - block.UnconsolidatedStep, - error, -) { +func (m *multiSeriesBlockStepIter) Err() error { + return nil +} + +func (m *multiSeriesBlockStepIter) Current() block.UnconsolidatedStep { values := make([]ts.Datapoints, len(m.values)) for i, series := range m.values { values[i] = series[m.index] @@ -279,7 +280,7 @@ func (m *multiSeriesBlockStepIter) Current() ( bounds := m.block.meta.Bounds t, _ := bounds.TimeForIndex(m.index) - return unconsolidatedStep{time: t, values: values}, nil + return unconsolidatedStep{time: t, values: values} } func (m *multiSeriesBlockStepIter) StepCount() int { @@ -319,10 +320,7 @@ func (m *multiSeriesBlockSeriesIter) Next() bool { return m.index < m.SeriesCount() } -func (m *multiSeriesBlockSeriesIter) Current() ( - block.UnconsolidatedSeries, - error, -) { +func (m *multiSeriesBlockSeriesIter) Current() block.UnconsolidatedSeries { s := m.block.seriesList[m.index] values := make([]ts.Datapoints, m.block.StepCount()) seriesValues := s.Values().AlignToBounds(m.block.meta.Bounds) @@ -338,7 +336,11 @@ func (m *multiSeriesBlockSeriesIter) Current() ( return block.NewUnconsolidatedSeries(values, block.SeriesMeta{ Tags: s.Tags, Name: s.Name(), - }), nil + }) +} + +func (m *multiSeriesBlockSeriesIter) Err() error { + return nil } func (m *multiSeriesBlockSeriesIter) Close() { diff --git a/src/query/storage/consolidated.go b/src/query/storage/consolidated.go index 8c5b7513f1..196a4e0379 100644 --- a/src/query/storage/consolidated.go +++ b/src/query/storage/consolidated.go @@ -76,19 +76,19 @@ func (c *consolidatedStepIter) Close() { c.unconsolidated.Close() } -func (c *consolidatedStepIter) Current() (block.Step, error) { - step, err := c.unconsolidated.Current() - if err != nil { - return nil, err - } +func (c *consolidatedStepIter) Err() error { + return c.unconsolidated.Err() +} +func (c *consolidatedStepIter) Current() block.Step { + step := c.unconsolidated.Current() stepValues := step.Values() consolidatedValues := make([]float64, len(stepValues)) for i, singleSeriesValues := range stepValues { consolidatedValues[i] = c.consolidationFunc(singleSeriesValues) } - return block.NewColStep(step.Time(), consolidatedValues), nil + return block.NewColStep(step.Time(), consolidatedValues) } func (c *consolidatedStepIter) StepCount() int { @@ -112,17 +112,17 @@ func (c *consolidatedSeriesIter) Next() bool { return c.unconsolidated.Next() } +func (c *consolidatedSeriesIter) Err() error { + return c.unconsolidated.Err() +} + func (c *consolidatedSeriesIter) Close() { c.unconsolidated.Close() } -func (c *consolidatedSeriesIter) Current() (block.Series, error) { - series, err := c.unconsolidated.Current() - if err != nil { - return block.Series{}, err - } - - return series.Consolidated(c.consolidationFunc), nil +func (c *consolidatedSeriesIter) Current() block.Series { + series := c.unconsolidated.Current() + return series.Consolidated(c.consolidationFunc) } func (c *consolidatedSeriesIter) SeriesCount() int { diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index 70b1e39ad1..7f02d0dc77 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -22,7 +22,6 @@ package m3db import ( "fmt" - "sync" "time" "github.com/m3db/m3/src/dbnode/encoding" @@ -32,7 +31,6 @@ import ( ) type encodedStepIterUnconsolidated struct { - mu sync.RWMutex lastBlock bool exhausted bool err error @@ -42,18 +40,12 @@ type encodedStepIterUnconsolidated struct { seriesIters []encoding.SeriesIterator } -func (it *encodedStepIterUnconsolidated) Current() ( - block.UnconsolidatedStep, - error, -) { - it.mu.RLock() +func (it *encodedStepIterUnconsolidated) Current() block.UnconsolidatedStep { if it.exhausted { - it.mu.RUnlock() return nil, fmt.Errorf("out of bounds") } if it.err != nil { - it.mu.RUnlock() return nil, it.err } @@ -77,13 +69,10 @@ func (it *encodedStepIterUnconsolidated) Current() ( } step := storage.NewUnconsolidatedStep(t, values) - it.mu.RUnlock() - return step, nil + return step } func (it *encodedStepIterUnconsolidated) Next() bool { - it.mu.Lock() - defer it.mu.Unlock() if it.exhausted { return false } @@ -114,6 +103,10 @@ func (it *encodedStepIterUnconsolidated) StepCount() int { return it.meta.Bounds.Steps() } +func (it *encodedStepIterUnconsolidated) Err() error { + return it.err +} + func (it *encodedStepIterUnconsolidated) SeriesMeta() []block.SeriesMeta { return it.seriesMeta } From 9a9b99d564cee22e116ed463e9406ac1f76cf3f5 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 28 Dec 2018 17:33:59 -0800 Subject: [PATCH 03/17] Finish refactor, fix output values from unconsolidated step iterator --- .../handler/prometheus/native/read_common.go | 8 +- src/query/block/block_mock.go | 38 ++++- src/query/block/column.go | 22 ++- src/query/block/scalar.go | 9 +- src/query/block/scalar_test.go | 20 +-- src/query/functions/binary/common.go | 10 +- src/query/functions/binary/common_test.go | 21 ++- src/query/functions/binary/unless.go | 4 +- src/query/storage/consolidated_test.go | 5 +- src/query/storage/unconsolidated_test.go | 155 ++++++++++++++++++ src/query/test/executor/transform.go | 9 +- .../ts/m3db/encoded_block_unconsolidated.go | 13 -- src/query/ts/m3db/encoded_series_iterator.go | 5 +- .../ts/m3db/encoded_series_iterator_test.go | 5 +- src/query/ts/m3db/encoded_step_iterator.go | 33 ++-- .../ts/m3db/encoded_step_iterator_test.go | 5 +- .../encoded_unconsolidated_iterator_test.go | 75 +++++---- .../encoded_unconsolidated_series_iterator.go | 60 +++---- .../encoded_unconsolidated_step_iterator.go | 95 +++++------ 19 files changed, 387 insertions(+), 205 deletions(-) create mode 100644 src/query/storage/unconsolidated_test.go diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index e5363f7e82..d6bc964969 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -180,13 +180,13 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { return nil, fmt.Errorf("invalid number of datapoints for series: %d, block: %d", i, idx) } - blockSeries, err := iter.Current() - if err != nil { + if err = iter.Err(); err != nil { return nil, err } - for i := 0; i < blockSeries.Len(); i++ { - values.SetValueAt(valIdx, blockSeries.ValueAtStep(i)) + blockSeries := iter.Current() + for j := 0; j < blockSeries.Len(); j++ { + values.SetValueAt(valIdx, blockSeries.ValueAtStep(j)) valIdx++ } } diff --git a/src/query/block/block_mock.go b/src/query/block/block_mock.go index 13e51cbeee..395a7bd690 100644 --- a/src/query/block/block_mock.go +++ b/src/query/block/block_mock.go @@ -164,12 +164,11 @@ func (mr *MockStepIterMockRecorder) Close() *gomock.Call { } // Current mocks base method -func (m *MockStepIter) Current() (Step, error) { +func (m *MockStepIter) Current() Step { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Current") ret0, _ := ret[0].(Step) - ret1, _ := ret[1].(error) - return ret0, ret1 + return ret0 } // Current indicates an expected call of Current @@ -178,6 +177,20 @@ func (mr *MockStepIterMockRecorder) Current() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockStepIter)(nil).Current)) } +// Err mocks base method +func (m *MockStepIter) Err() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Err") + ret0, _ := ret[0].(error) + return ret0 +} + +// Err indicates an expected call of Err +func (mr *MockStepIterMockRecorder) Err() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockStepIter)(nil).Err)) +} + // Meta mocks base method func (m *MockStepIter) Meta() Metadata { m.ctrl.T.Helper() @@ -270,12 +283,11 @@ func (mr *MockSeriesIterMockRecorder) Close() *gomock.Call { } // Current mocks base method -func (m *MockSeriesIter) Current() (Series, error) { +func (m *MockSeriesIter) Current() Series { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Current") ret0, _ := ret[0].(Series) - ret1, _ := ret[1].(error) - return ret0, ret1 + return ret0 } // Current indicates an expected call of Current @@ -284,6 +296,20 @@ func (mr *MockSeriesIterMockRecorder) Current() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockSeriesIter)(nil).Current)) } +// Err mocks base method +func (m *MockSeriesIter) Err() error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Err") + ret0, _ := ret[0].(error) + return ret0 +} + +// Err indicates an expected call of Err +func (mr *MockSeriesIterMockRecorder) Err() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockSeriesIter)(nil).Err)) +} + // Meta mocks base method func (m *MockSeriesIter) Meta() Metadata { m.ctrl.T.Helper() diff --git a/src/query/block/column.go b/src/query/block/column.go index 567876ff70..a53c080427 100644 --- a/src/query/block/column.go +++ b/src/query/block/column.go @@ -122,12 +122,15 @@ func (c *colBlockIter) Next() bool { } c.idx++ - c.timeForStep, c.err = c.meta.Bounds.TimeForIndex(c.idx) - if c.err != nil { - return false + next := c.idx < len(c.columns) + if next { + c.timeForStep, c.err = c.meta.Bounds.TimeForIndex(c.idx) + if c.err != nil { + return false + } } - return c.idx < len(c.columns) + return next } // Next returns true if iterator has more values remaining @@ -264,12 +267,15 @@ func (m *columnBlockSeriesIter) Err() error { func (m *columnBlockSeriesIter) Next() bool { m.idx++ - cols := m.columns - for i := 0; i < len(cols); i++ { - m.values[i] = cols[i].Values[m.idx] + next := m.idx < m.SeriesCount() + if next { + cols := m.columns + for i := 0; i < len(cols); i++ { + m.values[i] = cols[i].Values[m.idx] + } } - return m.idx < m.SeriesCount() + return next } func (m *columnBlockSeriesIter) Current() Series { diff --git a/src/query/block/scalar.go b/src/query/block/scalar.go index 6fe4e5ad0c..723b9e4ccf 100644 --- a/src/query/block/scalar.go +++ b/src/query/block/scalar.go @@ -131,9 +131,12 @@ func (it *scalarStepIter) Next() bool { } it.idx++ - it.stepTime, it.err = it.Meta().Bounds.TimeForIndex(it.idx) - if it.err != nil { - return false + next := it.idx < it.numVals + if next { + it.stepTime, it.err = it.Meta().Bounds.TimeForIndex(it.idx) + if it.err != nil { + return false + } } return it.idx < it.numVals diff --git a/src/query/block/scalar_test.go b/src/query/block/scalar_test.go index 86caa503fe..89098fbac0 100644 --- a/src/query/block/scalar_test.go +++ b/src/query/block/scalar_test.go @@ -52,13 +52,9 @@ func TestScalarBlock(t *testing.T) { verifyMetas(t, stepIter.Meta(), stepIter.SeriesMeta()) assert.Equal(t, 6, stepIter.StepCount()) - v := stepIter.Current() - require.Error(t, stepIter.Err()) - require.Nil(t, v) - valCounts := 0 for stepIter.Next() { - v = stepIter.Current() + v := stepIter.Current() require.NotNil(t, v) expectedTime := start.Add(time.Duration(valCounts) * 10 * time.Second) @@ -73,9 +69,6 @@ func TestScalarBlock(t *testing.T) { require.NoError(t, stepIter.Err()) assert.Equal(t, 6, valCounts) - v = stepIter.Current() - require.Error(t, stepIter.Err()) - require.Nil(t, v) seriesIter, err := block.SeriesIter() require.NoError(t, err) @@ -84,11 +77,8 @@ func TestScalarBlock(t *testing.T) { verifyMetas(t, seriesIter.Meta(), seriesIter.SeriesMeta()) require.Equal(t, 1, seriesIter.SeriesCount()) - series := seriesIter.Current() - require.Error(t, seriesIter.Err()) - require.True(t, seriesIter.Next()) - series = seriesIter.Current() + series := seriesIter.Current() require.NoError(t, seriesIter.Err()) assert.Equal(t, 6, series.Len()) @@ -102,11 +92,9 @@ func TestScalarBlock(t *testing.T) { assert.Equal(t, "", series.Meta.Name) require.False(t, seriesIter.Next()) - require.Error(t, seriesIter.Err()) + require.NoError(t, seriesIter.Err()) - series = seriesIter.Current() - err = block.Close() - require.NoError(t, err) + require.NoError(t, block.Close()) } func verifyMetas(t *testing.T, meta Metadata, seriesMeta []SeriesMeta) { diff --git a/src/query/functions/binary/common.go b/src/query/functions/binary/common.go index 13f6f7e70a..db660c062a 100644 --- a/src/query/functions/binary/common.go +++ b/src/query/functions/binary/common.go @@ -156,13 +156,11 @@ func appendValuesAtIndices(idxArray []int, iter block.StepIter, builder block.Bu step := iter.Current() values := step.Values() for _, idx := range idxArray { - builder.AppendValue(index, values[idx]) + if err := builder.AppendValue(index, values[idx]); err != nil { + return err + } } } - if err := iter.Err(); err != nil { - return err - } - - return nil + return iter.Err() } diff --git a/src/query/functions/binary/common_test.go b/src/query/functions/binary/common_test.go index 7a35f97a06..09f1e1b175 100644 --- a/src/query/functions/binary/common_test.go +++ b/src/query/functions/binary/common_test.go @@ -47,13 +47,15 @@ func builderMockWithExpectedValues(ctrl *gomock.Controller, indices []int, value func stepIterWithExpectedValues(ctrl *gomock.Controller, _ []int, values [][]float64) block.StepIter { stepIter := block.NewMockStepIter(ctrl) + stepIter.EXPECT().Next().Return(true).Times(len(values)) for _, val := range values { - stepIter.EXPECT().Next().Return(true) bl := block.NewMockStep(ctrl) bl.EXPECT().Values().Return(val) - stepIter.EXPECT().Current().Return(bl, nil) + stepIter.EXPECT().Current().Return(bl) } + stepIter.EXPECT().Next().Return(false) + stepIter.EXPECT().Err().Return(nil) return stepIter } @@ -64,7 +66,7 @@ var appendAtIndicesTests = []struct { builderValues, expectedValues [][]float64 }{ { - "no indecis", + "no indices", []int{}, []int{}, [][]float64{[]float64{1, 2}, []float64{3, 4}}, @@ -113,12 +115,19 @@ func TestAddAtIndicesErrors(t *testing.T) { defer ctrl.Finish() builder := block.NewMockBuilder(ctrl) + builder.EXPECT().AppendValue(gomock.Any(), gomock.Any()).Return(nil) stepIter := block.NewMockStepIter(ctrl) msg := "err" - stepIter.EXPECT().Next().Return(true) - stepIter.EXPECT().Current().Return(nil, fmt.Errorf(msg)) - err := appendValuesAtIndices([]int{1}, stepIter, builder) + stepIter.EXPECT().Next().Return(true).Times(1) + bl := block.NewMockStep(ctrl) + bl.EXPECT().Values().Return([]float64{0}) + stepIter.EXPECT().Current().Return(bl) + + stepIter.EXPECT().Next().Return(false).Times(1) + stepIter.EXPECT().Err().Return(fmt.Errorf(msg)) + + err := appendValuesAtIndices([]int{0}, stepIter, builder) assert.EqualError(t, err, msg) } diff --git a/src/query/functions/binary/unless.go b/src/query/functions/binary/unless.go index 2f8b98f8c7..b8a5e983fe 100644 --- a/src/query/functions/binary/unless.go +++ b/src/query/functions/binary/unless.go @@ -57,11 +57,11 @@ func makeUnlessBlock( return nil, err } - if err := builder.AddCols(lIter.StepCount()); err != nil { + if err = builder.AddCols(lIter.StepCount()); err != nil { return nil, err } - if err := appendValuesAtIndices(lIds, lIter, builder); err != nil { + if err = appendValuesAtIndices(lIds, lIter, builder); err != nil { return nil, err } diff --git a/src/query/storage/consolidated_test.go b/src/query/storage/consolidated_test.go index 4a38137dfe..58c01485b5 100644 --- a/src/query/storage/consolidated_test.go +++ b/src/query/storage/consolidated_test.go @@ -182,11 +182,12 @@ func TestConsolidation(t *testing.T) { i := 0 for iter.Next() { - step, err := iter.Current() - assert.NoError(t, err) + step := iter.Current() equalsWithNans(t, step.Values(), tt.expected[i]) i++ } + + assert.NoError(t, iter.Err()) } } diff --git a/src/query/storage/unconsolidated_test.go b/src/query/storage/unconsolidated_test.go new file mode 100644 index 0000000000..151fe9e836 --- /dev/null +++ b/src/query/storage/unconsolidated_test.go @@ -0,0 +1,155 @@ +// Copyright (c) 2018 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package storage + +import ( + "fmt" + "testing" + "time" + + "github.com/m3db/m3/src/query/block" + "github.com/m3db/m3/src/query/models" + "github.com/m3db/m3/src/query/ts" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func buildUnconsolidatedBlock(t *testing.T) block.UnconsolidatedBlock { + start := time.Now().Truncate(time.Hour) + datapoints := makeDatapoints(start) + + seriesList := make(ts.SeriesList, len(datapoints)) + for i, dp := range datapoints { + seriesList[i] = ts.NewSeries( + fmt.Sprintf("name_%d", i), + dp, + models.Tags{ + Opts: models.NewTagOptions(), + Tags: []models.Tag{{ + Name: []byte("a"), + Value: []byte(fmt.Sprintf("b_%d", i)), + }}, + }, + ) + } + + fetchQuery := &FetchQuery{ + Start: start, + End: start.Add(time.Minute * 30), + Interval: time.Minute, + } + + unconsolidated, err := NewMultiSeriesBlock(seriesList, fetchQuery) + require.NoError(t, err) + return unconsolidated +} + +func datapointsToFloatSlices(t *testing.T, dps []ts.Datapoints) [][]float64 { + vals := make([][]float64, len(dps)) + for i, dp := range dps { + vals[i] = dp.Values() + } + + return vals +} + +func TestUnconsolidatedStep(t *testing.T) { + expected := [][][]float64{ + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {10}, {}}, + {{}, {20}, {100}}, + {{}, {20}, {100}}, + {{}, {}, {}}, + {{1}, {}, {}}, + {{1}, {}, {}}, + {{2, 3}, {}, {}}, + {{4}, {}, {200}}, + {{5}, {}, {200}}, + {{6}, {}, {}}, + {{7}, {}, {}}, + {{7}, {30}, {}}, + {{}, {30}, {}}, + {{}, {}, {300}}, + {{}, {}, {300}}, + {{8}, {}, {}}, + {{8}, {}, {}}, + {{}, {40}, {}}, + {{}, {}, {}}, + {{}, {}, {400}}, + {{9}, {}, {400}}, + {{9}, {}, {}}, + {{}, {}, {500}}, + {{}, {}, {500}}, + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {}, {}}, + } + + unconsolidated := buildUnconsolidatedBlock(t) + iter, err := unconsolidated.StepIter() + assert.NoError(t, err) + + i := 0 + for iter.Next() { + step := iter.Current() + dps := datapointsToFloatSlices(t, step.Values()) + assert.Equal(t, expected[i], dps) + i++ + } + + assert.Equal(t, len(expected), i) + assert.NoError(t, iter.Err()) +} + +func TestUnconsolidatedSeries(t *testing.T) { + expected := [][][]float64{ + { + {}, {}, {}, {}, {}, {}, {1}, {1}, {2, 3}, {4}, {5}, {6}, {7}, {7}, + {}, {}, {}, {8}, {8}, {}, {}, {}, {9}, {9}, {}, {}, {}, {}, {}, {}, + }, + { + {}, {}, {10}, {20}, {20}, {}, {}, {}, {}, {}, {}, {}, {}, {30}, + {30}, {}, {}, {}, {}, {40}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, + }, + { + {}, {}, {}, {100}, {100}, {}, {}, {}, {}, {200}, {200}, {}, {}, {}, {}, + {300}, {300}, {}, {}, {}, {}, {400}, {400}, {}, {500}, {500}, {}, {}, {}, {}, + }, + } + + unconsolidated := buildUnconsolidatedBlock(t) + iter, err := unconsolidated.SeriesIter() + assert.NoError(t, err) + + i := 0 + for iter.Next() { + series := iter.Current() + dps := datapointsToFloatSlices(t, series.Datapoints()) + assert.Equal(t, expected[i], dps) + i++ + } + + assert.Equal(t, len(expected), i) + assert.NoError(t, iter.Err()) +} diff --git a/src/query/test/executor/transform.go b/src/query/test/executor/transform.go index e676ac4c79..402606c246 100644 --- a/src/query/test/executor/transform.go +++ b/src/query/test/executor/transform.go @@ -57,10 +57,7 @@ func (s *SinkNode) Process(ID parser.NodeID, block block.Block) error { anySeries := false for iter.Next() { anySeries = true - val, err := iter.Current() - if err != nil { - return err - } + val := iter.Current() values := make([]float64, val.Len()) for i := 0; i < val.Len(); i++ { @@ -70,6 +67,10 @@ func (s *SinkNode) Process(ID parser.NodeID, block block.Block) error { s.Metas = append(s.Metas, val.Meta) } + if err = iter.Err(); err != nil { + return err + } + if !anySeries { s.Metas = iter.SeriesMeta() } diff --git a/src/query/ts/m3db/encoded_block_unconsolidated.go b/src/query/ts/m3db/encoded_block_unconsolidated.go index a6294c7e39..92d041203f 100644 --- a/src/query/ts/m3db/encoded_block_unconsolidated.go +++ b/src/query/ts/m3db/encoded_block_unconsolidated.go @@ -69,19 +69,6 @@ func (b *encodedBlockUnconsolidated) WithMetadata( }, nil } -func (b *encodedBlockUnconsolidated) StepIter() ( - block.UnconsolidatedStepIter, - error, -) { - return &encodedStepIterUnconsolidated{ - meta: b.meta, - seriesMeta: b.seriesMetas, - seriesIters: b.seriesBlockIterators, - lastBlock: b.lastBlock, - validIters: make([]bool, len(b.seriesBlockIterators)), - }, nil -} - func (b *encodedBlockUnconsolidated) SeriesIter() ( block.UnconsolidatedSeriesIter, error, diff --git a/src/query/ts/m3db/encoded_series_iterator.go b/src/query/ts/m3db/encoded_series_iterator.go index 8361d65cdc..4fef40cbc4 100644 --- a/src/query/ts/m3db/encoded_series_iterator.go +++ b/src/query/ts/m3db/encoded_series_iterator.go @@ -75,8 +75,11 @@ func (it *encodedSeriesIter) Next() bool { it.idx++ next := it.idx < len(it.seriesIters) - it.consolidator.Reset(it.bounds.Start) + if !next { + return false + } + it.consolidator.Reset(it.bounds.Start) iter := it.seriesIters[it.idx] values := make([]float64, it.bounds.Steps()) xts.Memset(values, math.NaN()) diff --git a/src/query/ts/m3db/encoded_series_iterator_test.go b/src/query/ts/m3db/encoded_series_iterator_test.go index f78ae88deb..4b72b03ed8 100644 --- a/src/query/ts/m3db/encoded_series_iterator_test.go +++ b/src/query/ts/m3db/encoded_series_iterator_test.go @@ -106,11 +106,12 @@ func TestConsolidatedSeriesIterator(t *testing.T) { verifyMetas(t, i, bounds, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - series, err := iters.Current() - require.NoError(t, err) + series := iters.Current() test.EqualsWithNans(t, tt.expected[j], series.Values()) j++ } + + require.NoError(t, iters.Err()) } } } diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index f5e94ccb5b..d3b591fb94 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -21,8 +21,6 @@ package m3db import ( - "fmt" - "sync" "time" "github.com/m3db/m3/src/dbnode/encoding" @@ -39,7 +37,6 @@ type peekValue struct { } type encodedStepIter struct { - mu sync.RWMutex lastBlock bool started bool currentTime time.Time @@ -80,26 +77,11 @@ type encodedStep struct { func (s *encodedStep) Time() time.Time { return s.time } func (s *encodedStep) Values() []float64 { return s.values } -func (it *encodedStepIter) Current() (block.Step, error) { - it.mu.RLock() - currentTime := it.currentTime - if !it.started || currentTime.After(it.bounds.End()) { - it.mu.RUnlock() - return nil, fmt.Errorf("out of bounds") - } - - if it.err != nil { - it.mu.RUnlock() - return nil, it.err - } - - step := &encodedStep{ - time: currentTime, +func (it *encodedStepIter) Current() block.Step { + return &encodedStep{ + time: it.currentTime, values: it.consolidator.ConsolidateAndMoveToNext(), } - - it.mu.RUnlock() - return step, nil } // Moves to the next consolidated step for the i-th series in the block, @@ -202,8 +184,9 @@ func (it *encodedStepIter) initialStep() { } func (it *encodedStepIter) Next() bool { - it.mu.Lock() - defer it.mu.Unlock() + if it.err != nil { + return false + } if !it.started { it.initialStep() @@ -230,6 +213,10 @@ func (it *encodedStepIter) Meta() block.Metadata { return it.meta } +func (it *encodedStepIter) Err() error { + return it.err +} + func (it *encodedStepIter) Close() { // noop, as the resources at the step may still be in use; // instead call Close() on the encodedBlock that generated this diff --git a/src/query/ts/m3db/encoded_step_iterator_test.go b/src/query/ts/m3db/encoded_step_iterator_test.go index db9c421a39..11ea56e30c 100644 --- a/src/query/ts/m3db/encoded_step_iterator_test.go +++ b/src/query/ts/m3db/encoded_step_iterator_test.go @@ -119,12 +119,13 @@ func TestConsolidatedStepIterator(t *testing.T) { verifyMetas(t, i, bounds, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - step, err := iters.Current() - require.NoError(t, err) + step := iters.Current() vals := step.Values() test.EqualsWithNans(t, tt.expected[j], vals) j++ } + + require.NoError(t, iters.Err()) } } } diff --git a/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go b/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go index fbac626252..248b209cab 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go +++ b/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go @@ -30,32 +30,47 @@ import ( "github.com/stretchr/testify/require" ) -func datapointsToFloats(t *testing.T, dps []ts.Datapoints) []float64 { - vals := make([]float64, len(dps)) +func datapointsToFloatSlices(t *testing.T, dps []ts.Datapoints) [][]float64 { + vals := make([][]float64, len(dps)) for i, dp := range dps { - l := len(dp) - require.True(t, l < 2) - if l == 0 { - vals[i] = nan - } else if l == 1 { - vals[i] = dp[0].Value - } + vals[i] = dp.Values() } return vals } func TestUnconsolidatedStepIterator(t *testing.T) { - expected := [][]float64{ - {1, 10, 100}, - {2, 20, 200}, - {3, 30, 300}, - {4, 40, 400}, - {5, nan, 500}, - {6, nan, nan}, - {7, nan, nan}, - {8, nan, nan}, - {9, nan, nan}, + expected := [][][]float64{ + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {10}, {}}, + {{}, {20}, {100}}, + {{}, {20}, {100}}, + {{}, {}, {}}, + {{1}, {}, {}}, + {{1}, {}, {}}, + {{2, 3}, {}, {}}, + {{4}, {}, {200}}, + {{5}, {}, {200}}, + {{6}, {}, {}}, + {{7}, {}, {}}, + {{7}, {30}, {}}, + {{}, {30}, {}}, + {{}, {}, {300}}, + {{}, {}, {300}}, + {{8}, {}, {}}, + {{8}, {}, {}}, + {{}, {40}, {}}, + {{}, {}, {}}, + {{}, {}, {400}}, + {{9}, {}, {400}}, + {{9}, {}, {}}, + {{}, {}, {500}}, + {{}, {}, {500}}, + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {}, {}}, + {{}, {}, {}}, } j := 0 @@ -69,23 +84,17 @@ func TestUnconsolidatedStepIterator(t *testing.T) { verifyMetas(t, i, bounds, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - step, err := iters.Current() - require.NoError(t, err) + step := iters.Current() vals := step.Values() - actual := datapointsToFloats(t, vals) + actual := datapointsToFloatSlices(t, vals) + test.EqualsWithNans(t, expected[j], actual) j++ } - } -} -func datapointsToFloatSlices(t *testing.T, dps []ts.Datapoints) [][]float64 { - vals := make([][]float64, len(dps)) - for i, dp := range dps { - vals[i] = dp.Values() + require.Equal(t, len(expected), j) + require.NoError(t, iters.Err()) } - - return vals } func TestUnconsolidatedSeriesIterator(t *testing.T) { @@ -115,12 +124,14 @@ func TestUnconsolidatedSeriesIterator(t *testing.T) { verifyMetas(t, i, bounds, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - series, err := iters.Current() - require.NoError(t, err) + series := iters.Current() vals := series.Datapoints() actual := datapointsToFloatSlices(t, vals) test.EqualsWithNans(t, expected[j], actual) j++ } + + require.Equal(t, len(expected), j) + require.NoError(t, iters.Err()) } } diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index ccf3930993..4bd486cc31 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -21,53 +21,55 @@ package m3db import ( - "sync" - "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" xts "github.com/m3db/m3/src/query/ts" ) type encodedSeriesIterUnconsolidated struct { - mu sync.RWMutex idx int + err error meta block.Metadata + series block.UnconsolidatedSeries seriesMeta []block.SeriesMeta seriesIters []encoding.SeriesIterator } -func (it *encodedSeriesIterUnconsolidated) Current() ( - block.UnconsolidatedSeries, - error, -) { - it.mu.RLock() - iter := it.seriesIters[it.idx] - values := make(xts.Datapoints, 0, initBlockReplicaLength) - for iter.Next() { - dp, _, _ := iter.Current() - values = append(values, - xts.Datapoint{ - Timestamp: dp.Timestamp, - Value: dp.Value, - }) - } - - if err := iter.Err(); err != nil { - it.mu.RUnlock() - return block.UnconsolidatedSeries{}, err - } +func (it *encodedSeriesIterUnconsolidated) Current() block.UnconsolidatedSeries { + return it.series +} - alignedValues := values.AlignToBounds(it.meta.Bounds) - series := block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) - it.mu.RUnlock() - return series, nil +func (it *encodedSeriesIterUnconsolidated) Err() error { + return it.err } func (it *encodedSeriesIterUnconsolidated) Next() bool { - it.mu.Lock() + if it.err != nil { + return false + } + it.idx++ next := it.idx < len(it.seriesIters) - it.mu.Unlock() + if next { + iter := it.seriesIters[it.idx] + values := make(xts.Datapoints, 0, initBlockReplicaLength) + for iter.Next() { + dp, _, _ := iter.Current() + values = append(values, + xts.Datapoint{ + Timestamp: dp.Timestamp, + Value: dp.Value, + }) + } + + if it.err = iter.Err(); it.err != nil { + return false + } + + alignedValues := values.AlignToBounds(it.meta.Bounds) + it.series = block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) + } + return next } diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index 7f02d0dc77..c4972db151 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -21,9 +21,6 @@ package m3db import ( - "fmt" - "time" - "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/storage" @@ -31,71 +28,77 @@ import ( ) type encodedStepIterUnconsolidated struct { - lastBlock bool - exhausted bool - err error - meta block.Metadata - validIters []bool - seriesMeta []block.SeriesMeta - seriesIters []encoding.SeriesIterator + lastBlock bool + idx int + err error + meta block.Metadata + seriesMeta []block.SeriesMeta + seriesIters []encoding.SeriesIterator + expandedSeries [][]xts.Datapoints +} + +func (b *encodedBlockUnconsolidated) StepIter() ( + block.UnconsolidatedStepIter, + error, +) { + return &encodedStepIterUnconsolidated{ + idx: -1, + meta: b.meta, + seriesMeta: b.seriesMetas, + seriesIters: b.seriesBlockIterators, + lastBlock: b.lastBlock, + expandedSeries: make([][]xts.Datapoints, len(b.seriesBlockIterators)), + }, nil } func (it *encodedStepIterUnconsolidated) Current() block.UnconsolidatedStep { - if it.exhausted { - return nil, fmt.Errorf("out of bounds") + stepTime, _ := it.meta.Bounds.TimeForIndex(it.idx) + stepValues := make([]xts.Datapoints, len(it.expandedSeries)) + for j, series := range it.expandedSeries { + stepValues[j] = series[it.idx] } - if it.err != nil { - return nil, it.err - } + step := storage.NewUnconsolidatedStep(stepTime, stepValues) + return step +} - var t time.Time - values := make([]xts.Datapoints, len(it.seriesIters)) +func (it *encodedStepIterUnconsolidated) decodeSeries() bool { + values := make(xts.Datapoints, 0, initBlockReplicaLength) for i, iter := range it.seriesIters { - // Skip this iterator if it's not valid - if !it.validIters[i] { - continue + values = values[:0] + for iter.Next() { + dp, _, _ := iter.Current() + values = append(values, + xts.Datapoint{ + Timestamp: dp.Timestamp, + Value: dp.Value, + }) } - dp, _, _ := iter.Current() - if i == 0 { - t = dp.Timestamp + if it.err = iter.Err(); it.err != nil { + return false } - values[i] = []xts.Datapoint{{ - Timestamp: dp.Timestamp, - Value: dp.Value, - }} + it.expandedSeries[i] = values.AlignToBounds(it.meta.Bounds) } - step := storage.NewUnconsolidatedStep(t, values) - return step + return true } func (it *encodedStepIterUnconsolidated) Next() bool { - if it.exhausted { + if it.err != nil { return false } - var anyNext bool - for i, iter := range it.seriesIters { - if iter.Next() { - anyNext = true - it.validIters[i] = true - } else { - it.validIters[i] = false + it.idx++ + if it.idx == 0 { + // decode the series on initial Next() call + if success := it.decodeSeries(); !success { + return success } - - if err := iter.Err(); err != nil { - it.err = err - } - } - - if !anyNext { - it.exhausted = true } - return anyNext + return it.idx < it.meta.Bounds.Steps() } func (it *encodedStepIterUnconsolidated) StepCount() int { From 8efc5459fed36abf128f196784eb307950662452 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 11 Jan 2019 17:43:13 -0500 Subject: [PATCH 04/17] PR response flipping nesting and comments --- src/query/block/column.go | 42 +++++++++---------- src/query/block/scalar.go | 20 ++++----- src/query/block/types.go | 9 +++- src/query/executor/transform/lazy.go | 20 ++++----- src/query/ts/m3db/encoded_step_iterator.go | 5 +++ .../encoded_unconsolidated_series_iterator.go | 34 ++++++++------- 6 files changed, 68 insertions(+), 62 deletions(-) diff --git a/src/query/block/column.go b/src/query/block/column.go index a53c080427..0289355229 100644 --- a/src/query/block/column.go +++ b/src/query/block/column.go @@ -36,17 +36,14 @@ type columnBlock struct { seriesMeta []SeriesMeta } -// Unconsolidated returns the unconsolidated version for the block func (c *columnBlock) Unconsolidated() (UnconsolidatedBlock, error) { return nil, fmt.Errorf("unconsolidated view not supported for block, meta: %s", c.meta) } -// Meta returns the metadata for the block func (c *columnBlock) Meta() Metadata { return c.meta } -// StepIter returns a StepIterator func (c *columnBlock) StepIter() (StepIter, error) { if len(c.columns) != c.meta.Bounds.Steps() { return nil, fmt.Errorf("mismatch in block columns and meta bounds, columns: %d, bounds: %v", len(c.columns), c.meta.Bounds) @@ -61,7 +58,6 @@ func (c *columnBlock) StepIter() (StepIter, error) { } // TODO: allow series iteration -// SeriesIter returns a SeriesIterator func (c *columnBlock) SeriesIter() (SeriesIter, error) { return newColumnBlockSeriesIter(c.columns, c.meta, c.seriesMeta), nil } @@ -78,12 +74,10 @@ func (c *columnBlock) WithMetadata( } // TODO: allow series iteration -// SeriesMeta returns the metadata for each series in the block func (c *columnBlock) SeriesMeta() []SeriesMeta { return c.seriesMeta } -// StepCount returns the total steps func (c *columnBlock) StepCount() int { return len(c.columns) } @@ -115,7 +109,6 @@ func (c *colBlockIter) StepCount() int { return len(c.columns) } -// Next returns true if iterator has more values remaining func (c *colBlockIter) Next() bool { if c.err != nil { return false @@ -123,22 +116,22 @@ func (c *colBlockIter) Next() bool { c.idx++ next := c.idx < len(c.columns) - if next { - c.timeForStep, c.err = c.meta.Bounds.TimeForIndex(c.idx) - if c.err != nil { - return false - } + if !next { + return false + } + + c.timeForStep, c.err = c.meta.Bounds.TimeForIndex(c.idx) + if c.err != nil { + return false } return next } -// Next returns true if iterator has more values remaining func (c *colBlockIter) Err() error { return c.err } -// Current returns the current step func (c *colBlockIter) Current() Step { col := c.columns[c.idx] return ColStep{ @@ -147,9 +140,7 @@ func (c *colBlockIter) Current() Step { } } -// Close frees up resources -func (c *colBlockIter) Close() { -} +func (c *colBlockIter) Close() { /*no-op*/ } // ColStep is a single column containing data from multiple series at a given time step type ColStep struct { @@ -268,18 +259,23 @@ func (m *columnBlockSeriesIter) Err() error { func (m *columnBlockSeriesIter) Next() bool { m.idx++ next := m.idx < m.SeriesCount() - if next { - cols := m.columns - for i := 0; i < len(cols); i++ { - m.values[i] = cols[i].Values[m.idx] - } + if !next { + return false + } + + cols := m.columns + for i := 0; i < len(cols); i++ { + m.values[i] = cols[i].Values[m.idx] } return next } func (m *columnBlockSeriesIter) Current() Series { - return NewSeries(m.values, m.seriesMeta[m.idx]) + // TODO: pool these + vals := make([]float64, len(m.values)) + copy(vals, m.values) + return NewSeries(vals, m.seriesMeta[m.idx]) } // TODO: Actually free resources once we do pooling diff --git a/src/query/block/scalar.go b/src/query/block/scalar.go index 723b9e4ccf..f3aee0e45b 100644 --- a/src/query/block/scalar.go +++ b/src/query/block/scalar.go @@ -132,14 +132,16 @@ func (it *scalarStepIter) Next() bool { it.idx++ next := it.idx < it.numVals - if next { - it.stepTime, it.err = it.Meta().Bounds.TimeForIndex(it.idx) - if it.err != nil { - return false - } + if !next { + return false } - return it.idx < it.numVals + it.stepTime, it.err = it.Meta().Bounds.TimeForIndex(it.idx) + if it.err != nil { + return false + } + + return next } func (it *scalarStepIter) Current() Step { @@ -175,12 +177,6 @@ func (it *scalarSeriesIter) Next() bool { } func (it *scalarSeriesIter) Current() Series { - if it.idx != 0 { - // Indicates an error with the caller, having either not called Next() or attempted - // to get Current after Next() is false - panic("scalar iterator out of bounds") - } - return Series{ Meta: buildSeriesMeta(), values: it.vals, diff --git a/src/query/block/types.go b/src/query/block/types.go index 8ef96f3263..61d990de37 100644 --- a/src/query/block/types.go +++ b/src/query/block/types.go @@ -52,7 +52,7 @@ type UnconsolidatedBlock interface { // across all series comprising the box at a single time step. StepIter() (UnconsolidatedStepIter, error) // SeriesIter returns a series-wise block iterator, giving unconsolidated - // by series. + // by series. SeriesIter() (UnconsolidatedSeriesIter, error) // Consolidate attempts to consolidate the unconsolidated block. Consolidate() (Block, error) @@ -68,8 +68,15 @@ type SeriesMeta struct { // Iterator is the base iterator. type Iterator interface { + // Next moves to the next item in the iterator. It will return false if there + // are no more items, or if encountering an error. + // + // NB: it is important to check that Err() is nil after Next returns false, to + // ensure that any errors during iteration are detected and accounted for. Next() bool + // Err returns any error encountered during iteration. Err() error + // Close frees up resources held by the iterator. Close() } diff --git a/src/query/executor/transform/lazy.go b/src/query/executor/transform/lazy.go index b4c9fab9bd..d53e21c62a 100644 --- a/src/query/executor/transform/lazy.go +++ b/src/query/executor/transform/lazy.go @@ -93,14 +93,14 @@ func (s *stepIter) Next() bool { } next := s.iter.Next() - step := s.iter.Current() - s.step, s.err = s.node.ProcessStep(step) - if s.err != nil { + if !next { + s.err = s.iter.Err() return false } - if err := s.iter.Err(); err != nil { - s.err = err + step := s.iter.Current() + s.step, s.err = s.node.ProcessStep(step) + if s.err != nil { return false } @@ -156,14 +156,14 @@ func (s *seriesIter) Next() bool { } next := s.iter.Next() - step := s.iter.Current() - s.series, s.err = s.node.ProcessSeries(step) - if s.err != nil { + if !next { + s.err = s.iter.Err() return false } - if err := s.iter.Err(); err != nil { - s.err = err + step := s.iter.Current() + s.series, s.err = s.node.ProcessSeries(step) + if s.err != nil { return false } diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index d3b591fb94..4f5bb20613 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -188,6 +188,11 @@ func (it *encodedStepIter) Next() bool { return false } + checkNextTime := it.currentTime.Add(it.bounds.StepSize * 2) + if it.bounds.End().Before(checkNextTime) { + return false + } + if !it.started { it.initialStep() it.started = true diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index 4bd486cc31..9570903540 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -50,26 +50,28 @@ func (it *encodedSeriesIterUnconsolidated) Next() bool { it.idx++ next := it.idx < len(it.seriesIters) - if next { - iter := it.seriesIters[it.idx] - values := make(xts.Datapoints, 0, initBlockReplicaLength) - for iter.Next() { - dp, _, _ := iter.Current() - values = append(values, - xts.Datapoint{ - Timestamp: dp.Timestamp, - Value: dp.Value, - }) - } + if !next { + return false + } - if it.err = iter.Err(); it.err != nil { - return false - } + iter := it.seriesIters[it.idx] + values := make(xts.Datapoints, 0, initBlockReplicaLength) + for iter.Next() { + dp, _, _ := iter.Current() + values = append(values, + xts.Datapoint{ + Timestamp: dp.Timestamp, + Value: dp.Value, + }) + } - alignedValues := values.AlignToBounds(it.meta.Bounds) - it.series = block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) + if it.err = iter.Err(); it.err != nil { + return false } + alignedValues := values.AlignToBounds(it.meta.Bounds) + it.series = block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) + return next } From 033cc5e09f952311e6a3a61cc25fa49b461a0bdd Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 11 Jan 2019 20:07:39 -0500 Subject: [PATCH 05/17] PR response --- src/query/ts/m3db/encoded_block_iterator.go | 1 + .../ts/m3db/encoded_block_unconsolidated.go | 4 + src/query/ts/m3db/encoded_step_iterator.go | 8 +- .../encoded_unconsolidated_step_iterator.go | 113 +++++++++++++----- src/query/ts/m3db/types.go | 7 ++ 5 files changed, 97 insertions(+), 36 deletions(-) diff --git a/src/query/ts/m3db/encoded_block_iterator.go b/src/query/ts/m3db/encoded_block_iterator.go index 255ad244c9..606b219649 100644 --- a/src/query/ts/m3db/encoded_block_iterator.go +++ b/src/query/ts/m3db/encoded_block_iterator.go @@ -95,6 +95,7 @@ func newEncodedBlock( func (b *encodedBlock) Unconsolidated() (block.UnconsolidatedBlock, error) { return &encodedBlockUnconsolidated{ lastBlock: b.lastBlock, + lookback: b.lookback, meta: b.meta, tagOptions: b.tagOptions, consolidation: b.consolidation, diff --git a/src/query/ts/m3db/encoded_block_unconsolidated.go b/src/query/ts/m3db/encoded_block_unconsolidated.go index 92d041203f..8303aab8e5 100644 --- a/src/query/ts/m3db/encoded_block_unconsolidated.go +++ b/src/query/ts/m3db/encoded_block_unconsolidated.go @@ -21,6 +21,8 @@ package m3db import ( + "time" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/models" @@ -29,6 +31,7 @@ import ( type encodedBlockUnconsolidated struct { // There is slightly different execution for the last block in the series lastBlock bool + lookback time.Duration meta block.Metadata tagOptions models.TagOptions consolidation consolidationSettings @@ -39,6 +42,7 @@ type encodedBlockUnconsolidated struct { func (b *encodedBlockUnconsolidated) Consolidate() (block.Block, error) { return &encodedBlock{ lastBlock: b.lastBlock, + lookback: b.lookback, meta: b.meta, tagOptions: b.tagOptions, consolidation: b.consolidation, diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index 4f5bb20613..d23e98d9bd 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -30,12 +30,6 @@ import ( "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) -type peekValue struct { - started bool - finished bool - point ts.Datapoint -} - type encodedStepIter struct { lastBlock bool started bool @@ -120,7 +114,7 @@ func (it *encodedStepIter) nextConsolidatedForStep(i int) { iter := it.seriesIters[i] // Read through iterator until finding a data point outside of the // range of this consolidated step; then consolidate those points into - // a value, set the next peek value, and return true. + // a value, set the next peek value. for iter.Next() { dp, _, _ := iter.Current() diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index c4972db151..9074366dbc 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -21,10 +21,13 @@ package m3db import ( + "time" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/storage" xts "github.com/m3db/m3/src/query/ts" + "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIterUnconsolidated struct { @@ -35,12 +38,25 @@ type encodedStepIterUnconsolidated struct { seriesMeta []block.SeriesMeta seriesIters []encoding.SeriesIterator expandedSeries [][]xts.Datapoints + + stepTime time.Time + lookback time.Duration + accumulator *consolidators.StepLookbackAccumulator + seriesPeek []peekValue } func (b *encodedBlockUnconsolidated) StepIter() ( block.UnconsolidatedStepIter, error, ) { + cs := b.consolidation + accumulator := consolidators.NewStepLookbackAccumulator( + b.lookback, + cs.bounds.StepSize, + cs.currentTime, + len(b.seriesBlockIterators), + ) + return &encodedStepIterUnconsolidated{ idx: -1, meta: b.meta, @@ -48,41 +64,67 @@ func (b *encodedBlockUnconsolidated) StepIter() ( seriesIters: b.seriesBlockIterators, lastBlock: b.lastBlock, expandedSeries: make([][]xts.Datapoints, len(b.seriesBlockIterators)), + + lookback: b.lookback, + accumulator: accumulator, + seriesPeek: make([]peekValue, len(b.seriesBlockIterators)), }, nil } func (it *encodedStepIterUnconsolidated) Current() block.UnconsolidatedStep { - stepTime, _ := it.meta.Bounds.TimeForIndex(it.idx) - stepValues := make([]xts.Datapoints, len(it.expandedSeries)) - for j, series := range it.expandedSeries { - stepValues[j] = series[it.idx] - } - - step := storage.NewUnconsolidatedStep(stepTime, stepValues) - return step + points := it.accumulator.AccumulateAndMoveToNext() + return storage.NewUnconsolidatedStep(it.stepTime, points) } -func (it *encodedStepIterUnconsolidated) decodeSeries() bool { - values := make(xts.Datapoints, 0, initBlockReplicaLength) - for i, iter := range it.seriesIters { - values = values[:0] - for iter.Next() { - dp, _, _ := iter.Current() - values = append(values, - xts.Datapoint{ - Timestamp: dp.Timestamp, - Value: dp.Value, - }) - } +func (it *encodedStepIterUnconsolidated) nextForStep( + i int, + stepTime time.Time, +) { + peek := it.seriesPeek[i] + if peek.finished { + // No next value in this iterator + return + } - if it.err = iter.Err(); it.err != nil { - return false + if peek.started { + point := peek.point + if point.Timestamp.After(stepTime) { + // This point exists further than the current step + // There are next values, but current step is empty. + return } - it.expandedSeries[i] = values.AlignToBounds(it.meta.Bounds) + // clear peeked point. + it.seriesPeek[i].started = false + // Currently at a potentially viable data point. + // Record previously peeked value, and all potentially valid values. + it.accumulator.AddPointForIterator(point, i) + + // If at boundary, add the point as the current value. + if point.Timestamp.Equal(stepTime) { + return + } } - return true + iter := it.seriesIters[i] + // Read through iterator until finding a data point outside of the + // range of this step; then set the next peek value. + for iter.Next() { + dp, _, _ := iter.Current() + + // If this datapoint is before the current timestamp, add it to + // the accumulator + if !dp.Timestamp.After(stepTime) { + it.seriesPeek[i].started = false + it.accumulator.AddPointForIterator(dp, i) + } else { + // This point exists further than the current step; set peeked value + // to this point. + it.seriesPeek[i].point = dp + it.seriesPeek[i].started = true + return + } + } } func (it *encodedStepIterUnconsolidated) Next() bool { @@ -91,14 +133,27 @@ func (it *encodedStepIterUnconsolidated) Next() bool { } it.idx++ - if it.idx == 0 { - // decode the series on initial Next() call - if success := it.decodeSeries(); !success { - return success + next := it.idx < it.meta.Bounds.Steps() + + if !next { + return false + } + + stepTime, err := it.meta.Bounds.TimeForIndex(it.idx) + if err != nil { + it.err = err + return false + } + + for i, iter := range it.seriesIters { + it.nextForStep(i, stepTime) + if it.err = iter.Err(); it.err != nil { + return false } } - return it.idx < it.meta.Bounds.Steps() + it.stepTime = stepTime + return next } func (it *encodedStepIterUnconsolidated) StepCount() int { diff --git a/src/query/ts/m3db/types.go b/src/query/ts/m3db/types.go index 1e7f3453f3..6807983cd0 100644 --- a/src/query/ts/m3db/types.go +++ b/src/query/ts/m3db/types.go @@ -24,6 +24,7 @@ import ( "time" "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) @@ -65,3 +66,9 @@ type Options interface { // Validate ensures that the given block options are valid. Validate() error } + +type peekValue struct { + started bool + finished bool + point ts.Datapoint +} From d889a97a47f3c83aaf2f0551af237c93de7868c8 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 11 Jan 2019 20:07:57 -0500 Subject: [PATCH 06/17] Adding files. --- .../ts/m3db/consolidators/step_accumulator.go | 119 ++++++++++++++++++ .../consolidators/step_accumulator_test.go | 76 +++++++++++ 2 files changed, 195 insertions(+) create mode 100644 src/query/ts/m3db/consolidators/step_accumulator.go create mode 100644 src/query/ts/m3db/consolidators/step_accumulator_test.go diff --git a/src/query/ts/m3db/consolidators/step_accumulator.go b/src/query/ts/m3db/consolidators/step_accumulator.go new file mode 100644 index 0000000000..598f49ca54 --- /dev/null +++ b/src/query/ts/m3db/consolidators/step_accumulator.go @@ -0,0 +1,119 @@ +// Copyright (c) 2019 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package consolidators + +import ( + "time" + + "github.com/m3db/m3/src/dbnode/ts" + xts "github.com/m3db/m3/src/query/ts" +) + +func removeStaleAccumulated( + earliestLookback time.Time, + dps []xts.Datapoint, +) []xts.Datapoint { + for i, dp := range dps { + if !dp.Timestamp.Before(earliestLookback) { + return dps[i:] + } + } + + return dps[:0] +} + +// StepLookbackAccumulator is a helper for accumulating series in a step-wise +// fashion. It takes a 'step' of values, which represents a vertical +// slice of time across a list of series, and accumulates them when a +// valid step has been reached. +type StepLookbackAccumulator struct { + lookbackDuration time.Duration + stepSize time.Duration + earliestLookback time.Time + datapoints [][]xts.Datapoint + reset []bool +} + +// NewStepLookbackAccumulator creates an accumulator used for +// step iteration across a series list with a given lookback. +func NewStepLookbackAccumulator( + lookbackDuration, stepSize time.Duration, + startTime time.Time, + resultSize int, +) *StepLookbackAccumulator { + datapoints := make([][]xts.Datapoint, resultSize) + for i := range datapoints { + datapoints[i] = make([]xts.Datapoint, 0, initLength) + } + + reset := make([]bool, resultSize) + return &StepLookbackAccumulator{ + lookbackDuration: lookbackDuration, + stepSize: stepSize, + earliestLookback: startTime.Add(-1 * lookbackDuration), + datapoints: datapoints, + reset: reset, + } +} + +// AddPointForIterator adds a datapoint to a given step if it's within the valid +// time period; otherwise drops it silently, which is fine for accumulation. +func (c *StepLookbackAccumulator) AddPointForIterator( + dp ts.Datapoint, + i int, +) { + if dp.Timestamp.Before(c.earliestLookback) { + // this datapoint is too far in the past, it can be dropped. + return + } + + // TODO: the existing version of the step accumulator in Values.AlignToBounds + // resets incoming data points after accumulation; i.e. it will only keep + // points in the accumulation buffer if no other point comes in. This may not + // be the correct behaviour; investigate if it should be converted to keep + // the values instead. + if c.reset[i] { + c.datapoints[i] = c.datapoints[i][:0] + } + + c.reset[i] = false + c.datapoints[i] = append(c.datapoints[i], xts.Datapoint{ + Timestamp: dp.Timestamp, + Value: dp.Value, + }) +} + +// AccumulateAndMoveToNext consolidates the current values and moves the +// consolidator to the next given value, purging stale values. +func (c *StepLookbackAccumulator) AccumulateAndMoveToNext() []xts.Datapoints { + // Update earliest lookback then remove stale values for the next + // iteration of the datapoint set. + c.earliestLookback = c.earliestLookback.Add(c.stepSize) + accumulated := make([]xts.Datapoints, len(c.datapoints)) + for i, dps := range c.datapoints { + accumulated[i] = make(xts.Datapoints, len(dps)) + copy(accumulated[i], dps) + c.datapoints[i] = removeStaleAccumulated(c.earliestLookback, dps) + c.reset[i] = true + } + + return accumulated +} diff --git a/src/query/ts/m3db/consolidators/step_accumulator_test.go b/src/query/ts/m3db/consolidators/step_accumulator_test.go new file mode 100644 index 0000000000..b6c2671184 --- /dev/null +++ b/src/query/ts/m3db/consolidators/step_accumulator_test.go @@ -0,0 +1,76 @@ +// Copyright (c) 2019 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package consolidators + +import ( + "testing" + "time" + + "github.com/m3db/m3/src/dbnode/ts" + xts "github.com/m3db/m3/src/query/ts" + + "github.com/stretchr/testify/assert" +) + +func TestAccumulator(t *testing.T) { + lookback := time.Minute + start := time.Now().Truncate(time.Hour) + iterCount := 2 + + acc := NewStepLookbackAccumulator( + lookback, + lookback, + start, + iterCount, + ) + + // NB: lookback limit: start-1 + actual := acc.AccumulateAndMoveToNext() + expected := []xts.Datapoints{{}, {}} + assert.Equal(t, expected, actual) + + acc.AddPointForIterator(ts.Datapoint{Timestamp: start, Value: 1}, 0) + acc.AddPointForIterator(ts.Datapoint{Timestamp: start.Add(time.Minute), Value: 10}, 1) + + // NB: lookback limit: start + actual = acc.AccumulateAndMoveToNext() + expected[0] = xts.Datapoints{xts.Datapoint{Timestamp: start, Value: 1}} + expected[1] = xts.Datapoints{xts.Datapoint{Timestamp: start.Add(time.Minute), Value: 10}} + assert.Equal(t, expected, actual) + + // NB: lookback limit: start+1, point 1 is outside of the lookback period + actual = acc.AccumulateAndMoveToNext() + expected[0] = xts.Datapoints{} + assert.Equal(t, expected, actual) + + // NB: lookback limit: start+2 both points outside of the lookback period + actual = acc.AccumulateAndMoveToNext() + expected[1] = xts.Datapoints{} + assert.Equal(t, expected, actual) + + acc.AddPointForIterator(ts.Datapoint{Timestamp: start.Add(2*time.Minute + time.Second*30), Value: 2}, 0) + acc.AddPointForIterator(ts.Datapoint{Timestamp: start.Add(3*time.Minute + time.Second), Value: 3}, 0) + + // NB: lookback limit: start+3, both points in lookback period + actual = acc.AccumulateAndMoveToNext() + expected[0] = xts.Datapoints{xts.Datapoint{Timestamp: start.Add(3*time.Minute + time.Second), Value: 3}} + assert.Equal(t, expected, actual) +} From d4df67ddf6f66946765e2386e7016d29b3b2ad77 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Thu, 17 Jan 2019 15:54:01 -0500 Subject: [PATCH 07/17] Fixes to some boundary issues causing incorrect results. --- .../handler/prometheus/native/read_common.go | 31 +++-- src/query/models/bounds.go | 4 + src/query/ts/m3db/convert.go | 106 +++++++++++++++--- src/query/ts/m3db/convert_test.go | 93 +++++++++++++++ .../ts/m3db/encoded_block_iterator_builder.go | 11 +- .../ts/m3db/encoded_series_iterator_test.go | 5 +- .../ts/m3db/encoded_step_iterator_test.go | 5 +- 7 files changed, 214 insertions(+), 41 deletions(-) diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index d6bc964969..7ab6edfebb 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -143,11 +143,6 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { } firstBlock := blockList[0].block - firstStepIter, err := firstBlock.StepIter() - if err != nil { - return nil, err - } - firstSeriesIter, err := firstBlock.SeriesIter() if err != nil { return nil, err @@ -171,12 +166,21 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { seriesIters[i] = seriesIter } - numValues := firstStepIter.StepCount() * len(blockList) + numValues := 0 + for _, block := range blockList { + b, _ := block.block.StepIter() + numValues += b.StepCount() + } + for i := 0; i < numSeries; i++ { values := ts.NewFixedStepValues(bounds.StepSize, numValues, math.NaN(), bounds.Start) valIdx := 0 for idx, iter := range seriesIters { if !iter.Next() { + if err = iter.Err(); err != nil { + return nil, err + } + return nil, fmt.Errorf("invalid number of datapoints for series: %d, block: %d", i, idx) } @@ -224,21 +228,11 @@ func insertSortedBlock( "the block, wanted: %d, found: %d", seriesCount, blockSeriesCount) } - blockStepIter, err := b.StepIter() - if err != nil { - return nil, err - } - - blockStepCount := blockStepIter.StepCount() - if stepCount != blockStepCount { - return nil, fmt.Errorf("mismatch in number of steps for the"+ - "block, wanted: %d, found: %d", stepCount, blockStepCount) - } - // Binary search to keep the start times sorted index := sort.Search(len(blockList), func(i int) bool { - return blockList[i].meta.Bounds.Start.Before(blockMeta.Bounds.Start) + return blockList[i].meta.Bounds.Start.After(blockMeta.Bounds.Start) }) + // Append here ensures enough size in the slice blockList = append(blockList, blockWithMeta{}) copy(blockList[index+1:], blockList[index:]) @@ -246,5 +240,6 @@ func insertSortedBlock( block: b, meta: blockMeta, } + return blockList, nil } diff --git a/src/query/models/bounds.go b/src/query/models/bounds.go index a8e3902527..28b2e8720d 100644 --- a/src/query/models/bounds.go +++ b/src/query/models/bounds.go @@ -53,6 +53,10 @@ func (b Bounds) Steps() int { return 0 } + if b.StepSize > b.Duration { + return 1 + } + return int(b.Duration / b.StepSize) } diff --git a/src/query/ts/m3db/convert.go b/src/query/ts/m3db/convert.go index 63210ce295..852d17f3a0 100644 --- a/src/query/ts/m3db/convert.go +++ b/src/query/ts/m3db/convert.go @@ -21,6 +21,8 @@ package m3db import ( + "fmt" + "math" "sort" "time" @@ -35,15 +37,24 @@ const ( initBlockReplicaLength = 10 ) -// blockReplica contains the replicas for a single m3db block +// blockReplica contains the replicas for a single m3db block. type seriesBlock struct { - start time.Time + // internal start time for the block. + blockStart time.Time + // time at which the first point in the block will appear. + readStart time.Time blockSize time.Duration replicas []encoding.MultiReaderIterator } type seriesBlocks []seriesBlock +func (b seriesBlock) String() string { + return fmt.Sprint("BlockSize:", b.blockSize.Hours(), " blockStart:", + b.blockStart.Format("3:04:05PM"), " readStart:", b.readStart.Format("3:04:05PM"), + " num replicas", len(b.replicas)) +} + func (b seriesBlocks) Len() int { return len(b) } @@ -53,7 +64,7 @@ func (b seriesBlocks) Swap(i, j int) { } func (b seriesBlocks) Less(i, j int) bool { - return b[i].start.Before(b[j].start) + return b[i].blockStart.Before(b[j].blockStart) } func seriesIteratorsToEncodedBlockIterators( @@ -95,19 +106,36 @@ func convertM3DBSegmentedBlockIterators( opts Options, ) ([]block.Block, error) { defer iterators.Close() - blockBuilder := newEncodedBlockBuilder(opts.TagOptions(), opts.ConsolidationFunc()) + blockBuilder := newEncodedBlockBuilder(opts) var ( iterAlloc = opts.IterAlloc() pools = opts.IteratorPools() ) for _, seriesIterator := range iterators.Iters() { - blockReplicas, err := blockReplicasFromSeriesIterator(seriesIterator, iterAlloc, bounds, pools) + blockReplicas, err := blockReplicasFromSeriesIterator( + seriesIterator, + iterAlloc, + bounds, + pools, + ) if err != nil { return nil, err } - err = seriesBlocksFromBlockReplicas(blockBuilder, blockReplicas, bounds.StepSize, seriesIterator, pools) + blockReplicas = updateSeriesBlockStarts( + blockReplicas, + bounds.StepSize, + seriesIterator.Start(), + ) + + err = seriesBlocksFromBlockReplicas( + blockBuilder, + blockReplicas, + bounds.StepSize, + seriesIterator, + pools, + ) if err != nil { return nil, err } @@ -123,6 +151,11 @@ func blockReplicasFromSeriesIterator( pools encoding.IteratorPools, ) (seriesBlocks, error) { blocks := make(seriesBlocks, 0, bounds.Steps()) + var pool encoding.MultiReaderIteratorPool + if pools != nil { + pool = pools.MultiReaderIterator() + } + for _, replica := range seriesIterator.Replicas() { perBlockSliceReaders := replica.Readers() for next := true; next; next = perBlockSliceReaders.Next() { @@ -140,11 +173,11 @@ func blockReplicasFromSeriesIterator( readers[i] = clonedReader } - iter := encoding.NewMultiReaderIterator(iterAlloc, nil) + iter := encoding.NewMultiReaderIterator(iterAlloc, pool) iter.Reset(readers, start, bs) inserted := false for _, bl := range blocks { - if bl.start.Equal(start) { + if bl.blockStart.Equal(start) { inserted = true bl.replicas = append(bl.replicas, iter) break @@ -153,9 +186,9 @@ func blockReplicasFromSeriesIterator( if !inserted { blocks = append(blocks, seriesBlock{ - start: start, - blockSize: bs, - replicas: []encoding.MultiReaderIterator{iter}, + blockStart: start, + blockSize: bs, + replicas: []encoding.MultiReaderIterator{iter}, }) } } @@ -166,6 +199,40 @@ func blockReplicasFromSeriesIterator( return blocks, nil } +func blockDuration(blockSize, stepSize time.Duration) time.Duration { + numSteps := math.Ceil(float64(blockSize) / float64(stepSize)) + return stepSize * time.Duration(numSteps) +} + +// pads series blocks. +func updateSeriesBlockStarts( + blocks seriesBlocks, + stepSize time.Duration, + iterStart time.Time, +) seriesBlocks { + if len(blocks) == 0 { + return blocks + } + + firstStart := blocks[0].blockStart + if iterStart.Before(firstStart) { + fillSize := firstStart.Sub(iterStart) + numberToFill := int(fillSize/stepSize) + 1 + iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) + } + + // Update read starts for existing blocks. + for i, bl := range blocks { + blocks[i].readStart = iterStart + + fillSize := bl.blockStart.Add(bl.blockSize).Sub(iterStart) + numberToFill := int(fillSize/stepSize) + 1 + iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) + } + + return blocks +} + func seriesBlocksFromBlockReplicas( blockBuilder *encodedBlockBuilder, blockReplicas seriesBlocks, @@ -189,11 +256,11 @@ func seriesBlocksFromBlockReplicas( // TODO: use pooling for i, block := range blockReplicas { filterValuesStart := seriesIterator.Start() - if block.start.After(filterValuesStart) { - filterValuesStart = block.start + if block.blockStart.After(filterValuesStart) { + filterValuesStart = block.blockStart } - end := block.start.Add(block.blockSize) + end := block.blockStart.Add(block.blockSize) filterValuesEnd := seriesIterator.End() if end.Before(filterValuesEnd) { filterValuesEnd = end @@ -208,13 +275,20 @@ func seriesBlocksFromBlockReplicas( Replicas: block.replicas, }, nil) + // NB: if querying a small range, such that blockSize is greater than the + // iterator duration, use the smaller range instead. + duration := filterValuesEnd.Sub(filterValuesStart) + if duration > block.blockSize { + duration = block.blockSize + } + // NB(braskin): we should be careful when directly accessing the series iterators. // Instead, we should access them through the SeriesBlock. isLastBlock := i == replicaLength blockBuilder.add( models.Bounds{ - Start: filterValuesStart, - Duration: block.blockSize, + Start: block.readStart, + Duration: duration, StepSize: stepSize, }, iter, diff --git a/src/query/ts/m3db/convert_test.go b/src/query/ts/m3db/convert_test.go index 78b6f2d213..b51ce13f3c 100644 --- a/src/query/ts/m3db/convert_test.go +++ b/src/query/ts/m3db/convert_test.go @@ -182,3 +182,96 @@ func generateBlocks( require.NoError(t, err) return blocks, bounds } + +func TestUpdateTimeBySteps(t *testing.T) { + var tests = []struct { + stepSize, blockSize, expected time.Duration + }{ + {time.Minute * 15, time.Hour, time.Hour}, + {time.Minute * 14, time.Hour, time.Minute * 70}, + {time.Minute * 13, time.Hour, time.Minute * 65}, + {time.Minute * 59, time.Hour, time.Minute * 118}, + } + + for _, tt := range tests { + updateDuration := blockDuration(tt.blockSize, tt.stepSize) + assert.Equal(t, tt.expected/time.Minute, updateDuration/time.Minute) + } +} + +func TestPadSeriesBlocks(t *testing.T) { + blockSize := time.Hour + start := time.Now().Truncate(blockSize) + readOffset := time.Minute + itStart := start.Add(readOffset) + + var tests = []struct { + blockStart time.Time + stepSize time.Duration + expectedStart time.Time + expectedStartTwo time.Time + }{ + {start, time.Minute * 30, itStart, start.Add(61 * time.Minute)}, + { + start.Add(blockSize), + time.Minute * 30, + start.Add(61 * time.Minute), + start.Add(121 * time.Minute), + }, + // step 0: start + 1 , start + 37 + // step 1: start + 73 , start + 109 + // step 2: start + 145 + // step 3: start + 181, start + 217 + // step 4: start + 253, ... + { + start.Add(blockSize * 3), + time.Minute * 36, + start.Add(181 * time.Minute), + start.Add(253 * time.Minute), + }, + // step 0: start + 1 , start + 38 + // step 1: start + 75 , start + 112 + // step 2: start + 149 + // step 3: start + 186 + { + start.Add(blockSize * 2), + time.Minute * 37, + start.Add(149 * time.Minute), + start.Add(186 * time.Minute), + }, + // step 0: start + 1 , start + 12 , start + 23, + // start + 34 , start + 45 , start + 56 + // step 1: start + 67 , start + 78 , start + 89 + // start + 100, start + 111 + // step 2: start + 122 ... + { + start.Add(blockSize * 1), + time.Minute * 11, + start.Add(67 * time.Minute), + start.Add(122 * time.Minute), + }, + } + + for _, tt := range tests { + blocks := seriesBlocks{ + { + blockStart: tt.blockStart, + blockSize: blockSize, + replicas: []encoding.MultiReaderIterator{}, + }, + { + blockStart: tt.blockStart.Add(blockSize), + blockSize: blockSize, + replicas: []encoding.MultiReaderIterator{}, + }, + } + + updated := updateSeriesBlockStarts(blocks, tt.stepSize, itStart) + require.Equal(t, 2, len(updated)) + assert.Equal(t, tt.blockStart, updated[0].blockStart) + assert.Equal(t, tt.expectedStart, updated[0].readStart) + + assert.Equal(t, tt.blockStart.Add(blockSize), updated[1].blockStart) + assert.Equal(t, tt.expectedStartTwo, updated[1].readStart) + } +} diff --git a/src/query/ts/m3db/encoded_block_iterator_builder.go b/src/query/ts/m3db/encoded_block_iterator_builder.go index 00f243fe5e..e6a7a1d73f 100644 --- a/src/query/ts/m3db/encoded_block_iterator_builder.go +++ b/src/query/ts/m3db/encoded_block_iterator_builder.go @@ -42,16 +42,21 @@ type blockAtTime struct { type blocksAtTime []blockAtTime type encodedBlockBuilder struct { + lookback time.Duration blocksAtTime blocksAtTime tagOptions models.TagOptions consolidationFn consolidators.ConsolidationFunc } func newEncodedBlockBuilder( - tagOptions models.TagOptions, - consolidationFn consolidators.ConsolidationFunc, + opts Options, ) *encodedBlockBuilder { + lookback := opts.LookbackDuration() + tagOptions := opts.TagOptions() + consolidationFn := opts.ConsolidationFunc() + return &encodedBlockBuilder{ + lookback: lookback, blocksAtTime: make(blocksAtTime, 0, initBlockLength), tagOptions: tagOptions, consolidationFn: consolidationFn, @@ -83,7 +88,7 @@ func (b *encodedBlockBuilder) add( []encoding.SeriesIterator{}, b.tagOptions, consolidation, - time.Duration(0), + b.lookback, lastBlock, ) diff --git a/src/query/ts/m3db/encoded_series_iterator_test.go b/src/query/ts/m3db/encoded_series_iterator_test.go index dbb749e77b..ce89c7ecdd 100644 --- a/src/query/ts/m3db/encoded_series_iterator_test.go +++ b/src/query/ts/m3db/encoded_series_iterator_test.go @@ -234,11 +234,12 @@ func TestConsolidatedSeriesIteratorSplitByBlock(t *testing.T) { idx := verifyBoundsAndGetBlockIndex(t, bounds, iters.Meta().Bounds) verifyMetas(t, i, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - series, err := iters.Current() - require.NoError(t, err) + series := iters.Current() test.EqualsWithNans(t, tt.expected[idx][j], series.Values()) j++ } + + require.NoError(t, iters.Err()) } } } diff --git a/src/query/ts/m3db/encoded_step_iterator_test.go b/src/query/ts/m3db/encoded_step_iterator_test.go index e4ab0d9b70..11adb337f2 100644 --- a/src/query/ts/m3db/encoded_step_iterator_test.go +++ b/src/query/ts/m3db/encoded_step_iterator_test.go @@ -258,12 +258,13 @@ func TestConsolidatedStepIteratorSplitByBlock(t *testing.T) { idx := verifyBoundsAndGetBlockIndex(t, bounds, iters.Meta().Bounds) verifyMetas(t, i, iters.Meta(), iters.SeriesMeta()) for iters.Next() { - step, err := iters.Current() - require.NoError(t, err) + step := iters.Current() vals := step.Values() test.EqualsWithNans(t, tt.expected[idx][j], vals) j++ } + + require.NoError(t, iters.Err()) } } } From f60bd83e10da21bf807d1e1d1718fcfff89d5249 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Thu, 24 Jan 2019 10:13:24 -0500 Subject: [PATCH 08/17] wip --- .../api/v1/handler/prometheus/native/read_common.go | 13 +++++++++++-- src/query/block/column.go | 1 + src/query/models/bounds.go | 4 ++++ src/query/models/bounds_test.go | 8 +++++--- src/query/storage/m3/storage.go | 1 - src/query/ts/m3db/convert.go | 3 +-- src/query/ts/m3db/convert_test.go | 1 + .../m3db/encoded_unconsolidated_series_iterator.go | 3 +++ src/query/ts/m3db/options.go | 5 ----- src/query/ts/m3db/types.go | 3 --- 10 files changed, 26 insertions(+), 16 deletions(-) diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index 7ab6edfebb..9825e6f993 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -58,12 +58,13 @@ func read( // Results is closed by execute results := make(chan executor.Query) go engine.ExecuteExpr(ctx, parser, opts, params, results) - + fmt.Println("executing") // Block slices are sorted by start time // TODO: Pooling sortedBlockList := make([]blockWithMeta, 0, initialBlockAlloc) var processErr error for result := range results { + fmt.Println("got res") if result.Err != nil { processErr = result.Err break @@ -106,7 +107,7 @@ func read( } } } - + fmt.Println("got all") // Ensure that the blocks are closed. Can't do this above since sortedBlockList might change defer func() { for _, b := range sortedBlockList { @@ -116,6 +117,7 @@ func read( }() if processErr != nil { + fmt.Println("err not nil draining") // Drain anything remaining drainResultChan(results) return nil, processErr @@ -138,12 +140,15 @@ func drainResultChan(resultsChan chan executor.Query) { } func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { + fmt.Println("1") if len(blockList) == 0 { return emptySeriesList, nil } + fmt.Println("2") firstBlock := blockList[0].block firstSeriesIter, err := firstBlock.SeriesIter() + fmt.Println("3", firstSeriesIter, err) if err != nil { return nil, err } @@ -157,6 +162,7 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { seriesIters := make([]block.SeriesIter, len(blockList)) // To create individual series, we iterate over seriesIterators for each block in the block list. // For each iterator, the nth current() will be combined to give the nth series + fmt.Println("4") for i, b := range blockList { seriesIter, err := b.block.SeriesIter() if err != nil { @@ -165,12 +171,14 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { seriesIters[i] = seriesIter } + fmt.Println("5") numValues := 0 for _, block := range blockList { b, _ := block.block.StepIter() numValues += b.StepCount() } + fmt.Println("6") for i := 0; i < numSeries; i++ { values := ts.NewFixedStepValues(bounds.StepSize, numValues, math.NaN(), bounds.Start) @@ -178,6 +186,7 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { for idx, iter := range seriesIters { if !iter.Next() { if err = iter.Err(); err != nil { + fmt.Println("oops err") return nil, err } diff --git a/src/query/block/column.go b/src/query/block/column.go index 0289355229..336bd1830a 100644 --- a/src/query/block/column.go +++ b/src/query/block/column.go @@ -253,6 +253,7 @@ func (m *columnBlockSeriesIter) SeriesCount() int { } func (m *columnBlockSeriesIter) Err() error { + // no-op return nil } diff --git a/src/query/models/bounds.go b/src/query/models/bounds.go index 28b2e8720d..8b2a3de842 100644 --- a/src/query/models/bounds.go +++ b/src/query/models/bounds.go @@ -53,6 +53,10 @@ func (b Bounds) Steps() int { return 0 } + // NB: If the duration is larger than step size, there should still be a + // point in this block. This describes a case where the start of a series is + // at a block boundary; there is an assumption that step size is smaller than + // duration in general. if b.StepSize > b.Duration { return 1 } diff --git a/src/query/models/bounds_test.go b/src/query/models/bounds_test.go index 4df4400b5c..2772082b31 100644 --- a/src/query/models/bounds_test.go +++ b/src/query/models/bounds_test.go @@ -34,7 +34,8 @@ func TestBounds(t *testing.T) { Duration: 0, StepSize: time.Minute, } - assert.Equal(t, bounds.Steps(), 0) + + assert.Equal(t, bounds.Steps(), 1) _, err := bounds.TimeForIndex(0) assert.Error(t, err, "no valid index in this block") @@ -43,9 +44,10 @@ func TestBounds(t *testing.T) { Duration: time.Second, StepSize: time.Minute, } - assert.Equal(t, bounds.Steps(), 0) + + assert.Equal(t, bounds.Steps(), 1) _, err = bounds.TimeForIndex(0) - assert.Error(t, err, "no valid index in this block") + assert.NoError(t, err) bounds = Bounds{ Start: now, diff --git a/src/query/storage/m3/storage.go b/src/query/storage/m3/storage.go index c139bf2b10..8092c9b4fa 100644 --- a/src/query/storage/m3/storage.go +++ b/src/query/storage/m3/storage.go @@ -120,7 +120,6 @@ func (s *m3storage) FetchBlocks( // If using multiblock, update options to reflect this. if options.BlockType == models.TypeMultiBlock { opts = opts. - SetLookbackDuration(0). SetSplitSeriesByBlock(true) } diff --git a/src/query/ts/m3db/convert.go b/src/query/ts/m3db/convert.go index 852d17f3a0..5b0437e3bb 100644 --- a/src/query/ts/m3db/convert.go +++ b/src/query/ts/m3db/convert.go @@ -105,7 +105,7 @@ func convertM3DBSegmentedBlockIterators( bounds models.Bounds, opts Options, ) ([]block.Block, error) { - defer iterators.Close() + // defer iterators.Close() blockBuilder := newEncodedBlockBuilder(opts) var ( iterAlloc = opts.IterAlloc() @@ -224,7 +224,6 @@ func updateSeriesBlockStarts( // Update read starts for existing blocks. for i, bl := range blocks { blocks[i].readStart = iterStart - fillSize := bl.blockStart.Add(bl.blockSize).Sub(iterStart) numberToFill := int(fillSize/stepSize) + 1 iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) diff --git a/src/query/ts/m3db/convert_test.go b/src/query/ts/m3db/convert_test.go index b51ce13f3c..5a755039d5 100644 --- a/src/query/ts/m3db/convert_test.go +++ b/src/query/ts/m3db/convert_test.go @@ -140,6 +140,7 @@ func verifyBoundsAndGetBlockIndex(t *testing.T, bounds, sub models.Bounds) int { require.Equal(t, bounds.StepSize, sub.StepSize) require.Equal(t, blockSize, sub.Duration) diff := sub.Start.Sub(bounds.Start) + fmt.Println(diff, blockSize) require.Equal(t, 0, int(diff%blockSize)) return int(diff / blockSize) } diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index 9570903540..087de67146 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -21,6 +21,8 @@ package m3db import ( + "fmt" + "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" xts "github.com/m3db/m3/src/query/ts" @@ -44,6 +46,7 @@ func (it *encodedSeriesIterUnconsolidated) Err() error { } func (it *encodedSeriesIterUnconsolidated) Next() bool { + fmt.Println("Calling next") if it.err != nil { return false } diff --git a/src/query/ts/m3db/options.go b/src/query/ts/m3db/options.go index 20fc3f6a5c..d6f787de08 100644 --- a/src/query/ts/m3db/options.go +++ b/src/query/ts/m3db/options.go @@ -69,11 +69,6 @@ func (o *encodedBlockOptions) SetSplitSeriesByBlock(split bool) Options { } func (o *encodedBlockOptions) SplittingSeriesByBlock() bool { - // If any lookback duration has been set, cannot split series by block. - if o.lookbackDuration > 0 { - return false - } - return o.splitSeries } diff --git a/src/query/ts/m3db/types.go b/src/query/ts/m3db/types.go index 6807983cd0..87546dacf4 100644 --- a/src/query/ts/m3db/types.go +++ b/src/query/ts/m3db/types.go @@ -35,9 +35,6 @@ import ( type Options interface { // SetSplitSeriesByBlock determines if the converter will split the series // by blocks, or if it will instead treat the entire series as a single block. - // - // NB: if a lookback duration greater than 0 has been set, the series will - // always be treated as a single block. SetSplitSeriesByBlock(bool) Options // SplittingSeriesByBlock returns true iff lookback duration is 0, and the // options has not been forced to return a single block. From 7f4362769b6a209d0ab08c30f09e1a18016c2209 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 1 Feb 2019 16:50:48 -0500 Subject: [PATCH 09/17] Fix to temporal functions, cleanup. --- src/query/README.md | 6 +- .../handler/prometheus/native/read_common.go | 12 +- src/query/models/bounds.go | 8 - src/query/models/bounds_test.go | 6 +- src/query/storage/block.go | 33 +++- src/query/storage/unconsolidated_test.go | 30 ++-- src/query/storage/validator/storage.go | 17 +- .../ts/m3db/consolidators/step_accumulator.go | 29 +--- .../consolidators/step_accumulator_test.go | 6 +- src/query/ts/m3db/convert.go | 14 +- src/query/ts/m3db/convert_test.go | 1 - src/query/ts/m3db/encoded_series_iterator.go | 2 +- .../ts/m3db/encoded_series_iterator_test.go | 2 +- .../ts/m3db/encoded_step_iterator_test.go | 2 +- .../encoded_unconsolidated_iterator_test.go | 32 ++-- .../encoded_unconsolidated_series_iterator.go | 5 +- src/query/ts/values.go | 66 ++++++-- src/query/ts/values_test.go | 146 +++++++++--------- 18 files changed, 226 insertions(+), 191 deletions(-) diff --git a/src/query/README.md b/src/query/README.md index 534f55218b..5ac745f1a4 100644 --- a/src/query/README.md +++ b/src/query/README.md @@ -28,7 +28,7 @@ Finally, you can spin up the two containers using `docker-compose` within the `d > Note: The default local ports for Prometheus and m3coordinator are `9090` and `7201`, respectively, and the default `prometheus.yml` file is `docker/prometheus.yml` > ->If you want to override these, you can pass in the following environment variables to the `docker-compose` command: +> If you want to override these, you can pass in the following environment variables to the `docker-compose` command: > > `LOCAL_PROM_PORT` > @@ -89,6 +89,7 @@ Users can validate m3query's Prometheus results for a given query against input Query: `go_gc_duration_seconds{quantile="1"} * 2` Input JSON: + ``` { "input": { @@ -145,7 +146,8 @@ Input JSON: } } ``` -Full request: `curl -X POST 'localhost:7201/api/v1/debug/validate_query?start=1543431465&end=1543435045&step=14s&query=go_gc_duration_seconds%7Bquantile%3D%221%22%7D*2' -d @ --header "Content-Type: application/json"` + +Full request: `curl -X POST 'http://localhost:7201/api/v1/debug/validate_query?query=sum_over_time(process_cpu_seconds_total%7Binstance%3D~%22m3db_seed%3A7203%22%7D%5B60s%5D)&start=1549035092&end=1549035177&step=1' -d @lol.json --header "Content-Type: application/json"` [doc-img]: https://godoc.org/github.com/m3db/m3/src/query?status.svg [doc]: https://godoc.org/github.com/m3db/m3/src/query diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index 9825e6f993..4d328a1ae6 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -58,13 +58,11 @@ func read( // Results is closed by execute results := make(chan executor.Query) go engine.ExecuteExpr(ctx, parser, opts, params, results) - fmt.Println("executing") // Block slices are sorted by start time // TODO: Pooling sortedBlockList := make([]blockWithMeta, 0, initialBlockAlloc) var processErr error for result := range results { - fmt.Println("got res") if result.Err != nil { processErr = result.Err break @@ -107,7 +105,7 @@ func read( } } } - fmt.Println("got all") + // Ensure that the blocks are closed. Can't do this above since sortedBlockList might change defer func() { for _, b := range sortedBlockList { @@ -117,7 +115,6 @@ func read( }() if processErr != nil { - fmt.Println("err not nil draining") // Drain anything remaining drainResultChan(results) return nil, processErr @@ -140,15 +137,12 @@ func drainResultChan(resultsChan chan executor.Query) { } func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { - fmt.Println("1") if len(blockList) == 0 { return emptySeriesList, nil } - fmt.Println("2") firstBlock := blockList[0].block firstSeriesIter, err := firstBlock.SeriesIter() - fmt.Println("3", firstSeriesIter, err) if err != nil { return nil, err } @@ -162,7 +156,6 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { seriesIters := make([]block.SeriesIter, len(blockList)) // To create individual series, we iterate over seriesIterators for each block in the block list. // For each iterator, the nth current() will be combined to give the nth series - fmt.Println("4") for i, b := range blockList { seriesIter, err := b.block.SeriesIter() if err != nil { @@ -171,14 +164,12 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { seriesIters[i] = seriesIter } - fmt.Println("5") numValues := 0 for _, block := range blockList { b, _ := block.block.StepIter() numValues += b.StepCount() } - fmt.Println("6") for i := 0; i < numSeries; i++ { values := ts.NewFixedStepValues(bounds.StepSize, numValues, math.NaN(), bounds.Start) @@ -186,7 +177,6 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { for idx, iter := range seriesIters { if !iter.Next() { if err = iter.Err(); err != nil { - fmt.Println("oops err") return nil, err } diff --git a/src/query/models/bounds.go b/src/query/models/bounds.go index 8b2a3de842..a8e3902527 100644 --- a/src/query/models/bounds.go +++ b/src/query/models/bounds.go @@ -53,14 +53,6 @@ func (b Bounds) Steps() int { return 0 } - // NB: If the duration is larger than step size, there should still be a - // point in this block. This describes a case where the start of a series is - // at a block boundary; there is an assumption that step size is smaller than - // duration in general. - if b.StepSize > b.Duration { - return 1 - } - return int(b.Duration / b.StepSize) } diff --git a/src/query/models/bounds_test.go b/src/query/models/bounds_test.go index 2772082b31..8c8aa76012 100644 --- a/src/query/models/bounds_test.go +++ b/src/query/models/bounds_test.go @@ -35,7 +35,7 @@ func TestBounds(t *testing.T) { StepSize: time.Minute, } - assert.Equal(t, bounds.Steps(), 1) + assert.Equal(t, bounds.Steps(), 0) _, err := bounds.TimeForIndex(0) assert.Error(t, err, "no valid index in this block") @@ -45,9 +45,9 @@ func TestBounds(t *testing.T) { StepSize: time.Minute, } - assert.Equal(t, bounds.Steps(), 1) + assert.Equal(t, bounds.Steps(), 0) _, err = bounds.TimeForIndex(0) - assert.NoError(t, err) + assert.Error(t, err, "no valid index in this block") bounds = Bounds{ Start: now, diff --git a/src/query/storage/block.go b/src/query/storage/block.go index 86c6e89e37..9eab88fb43 100644 --- a/src/query/storage/block.go +++ b/src/query/storage/block.go @@ -137,8 +137,9 @@ func (m *multiBlockWrapper) Close() error { } type multiSeriesBlock struct { - seriesList ts.SeriesList - meta block.Metadata + seriesList ts.SeriesList + meta block.Metadata + consolidated bool } // NewMultiSeriesBlock returns a new unconsolidated block @@ -202,6 +203,7 @@ func (m multiSeriesBlock) SeriesMeta() []block.SeriesMeta { } func (m multiSeriesBlock) Consolidate() (block.Block, error) { + m.consolidated = true return &consolidatedBlock{ unconsolidated: m, consolidationFunc: block.TakeLast, @@ -240,8 +242,12 @@ type multiSeriesBlockStepIter struct { func newMultiSeriesBlockStepIter(block multiSeriesBlock) block.UnconsolidatedStepIter { values := make([][]ts.Datapoints, len(block.seriesList)) bounds := block.meta.Bounds - for i, series := range block.seriesList { - values[i] = series.Values().AlignToBounds(bounds) + for i, s := range block.seriesList { + if block.consolidated { + values[i] = s.Values().AlignToBounds(bounds) + } else { + values[i] = s.Values().AlignToBoundsNoWriteForward(bounds) + } } return &multiSeriesBlockStepIter{ @@ -293,8 +299,9 @@ func (m *multiSeriesBlockStepIter) Close() { } type multiSeriesBlockSeriesIter struct { - block multiSeriesBlock - index int + block multiSeriesBlock + index int + consolidated bool } func (m *multiSeriesBlockSeriesIter) Meta() block.Metadata { @@ -308,7 +315,11 @@ func (m *multiSeriesBlockSeriesIter) SeriesMeta() []block.SeriesMeta { func newMultiSeriesBlockSeriesIter( block multiSeriesBlock, ) block.UnconsolidatedSeriesIter { - return &multiSeriesBlockSeriesIter{block: block, index: -1} + return &multiSeriesBlockSeriesIter{ + block: block, + index: -1, + consolidated: block.consolidated, + } } func (m *multiSeriesBlockSeriesIter) SeriesCount() int { @@ -323,7 +334,13 @@ func (m *multiSeriesBlockSeriesIter) Next() bool { func (m *multiSeriesBlockSeriesIter) Current() block.UnconsolidatedSeries { s := m.block.seriesList[m.index] values := make([]ts.Datapoints, m.block.StepCount()) - seriesValues := s.Values().AlignToBounds(m.block.meta.Bounds) + var seriesValues []ts.Datapoints + if m.consolidated { + seriesValues = s.Values().AlignToBounds(m.block.meta.Bounds) + } else { + seriesValues = s.Values().AlignToBoundsNoWriteForward(m.block.meta.Bounds) + } + seriesLen := len(seriesValues) for i := 0; i < m.block.StepCount(); i++ { if i < seriesLen { diff --git a/src/query/storage/unconsolidated_test.go b/src/query/storage/unconsolidated_test.go index 151fe9e836..d3b313c3a8 100644 --- a/src/query/storage/unconsolidated_test.go +++ b/src/query/storage/unconsolidated_test.go @@ -40,7 +40,7 @@ func buildUnconsolidatedBlock(t *testing.T) block.UnconsolidatedBlock { seriesList := make(ts.SeriesList, len(datapoints)) for i, dp := range datapoints { seriesList[i] = ts.NewSeries( - fmt.Sprintf("name_%d", i), + []byte(fmt.Sprintf("name_%d", i)), dp, models.Tags{ Opts: models.NewTagOptions(), @@ -78,28 +78,28 @@ func TestUnconsolidatedStep(t *testing.T) { {{}, {}, {}}, {{}, {10}, {}}, {{}, {20}, {100}}, - {{}, {20}, {100}}, + {{}, {}, {}}, {{}, {}, {}}, {{1}, {}, {}}, - {{1}, {}, {}}, + {{}, {}, {}}, {{2, 3}, {}, {}}, {{4}, {}, {200}}, - {{5}, {}, {200}}, + {{5}, {}, {}}, {{6}, {}, {}}, {{7}, {}, {}}, - {{7}, {30}, {}}, {{}, {30}, {}}, + {{}, {}, {}}, {{}, {}, {300}}, - {{}, {}, {300}}, - {{8}, {}, {}}, + {{}, {}, {}}, {{8}, {}, {}}, + {{}, {}, {}}, {{}, {40}, {}}, {{}, {}, {}}, {{}, {}, {400}}, - {{9}, {}, {400}}, {{9}, {}, {}}, + {{}, {}, {}}, {{}, {}, {500}}, - {{}, {}, {500}}, + {{}, {}, {}}, {{}, {}, {}}, {{}, {}, {}}, {{}, {}, {}}, @@ -125,16 +125,16 @@ func TestUnconsolidatedStep(t *testing.T) { func TestUnconsolidatedSeries(t *testing.T) { expected := [][][]float64{ { - {}, {}, {}, {}, {}, {}, {1}, {1}, {2, 3}, {4}, {5}, {6}, {7}, {7}, - {}, {}, {}, {8}, {8}, {}, {}, {}, {9}, {9}, {}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {}, {}, {1}, {}, {2, 3}, {4}, {5}, {6}, {7}, {}, + {}, {}, {}, {8}, {}, {}, {}, {}, {9}, {}, {}, {}, {}, {}, {}, {}, }, { - {}, {}, {10}, {20}, {20}, {}, {}, {}, {}, {}, {}, {}, {}, {30}, - {30}, {}, {}, {}, {}, {40}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, + {}, {}, {10}, {20}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {30}, + {}, {}, {}, {}, {}, {40}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, }, { - {}, {}, {}, {100}, {100}, {}, {}, {}, {}, {200}, {200}, {}, {}, {}, {}, - {300}, {300}, {}, {}, {}, {}, {400}, {400}, {}, {500}, {500}, {}, {}, {}, {}, + {}, {}, {}, {100}, {}, {}, {}, {}, {}, {200}, {}, {}, {}, {}, {}, + {300}, {}, {}, {}, {}, {}, {400}, {}, {}, {500}, {}, {}, {}, {}, {}, }, } diff --git a/src/query/storage/validator/storage.go b/src/query/storage/validator/storage.go index a73cc5d213..1cdcbec7e9 100644 --- a/src/query/storage/validator/storage.go +++ b/src/query/storage/validator/storage.go @@ -81,6 +81,7 @@ func PromResultToSeriesList(promReadResp prometheus.PromResp, tagOptions models. } results := promReadResp.Data.Result + seriesList := make([]*ts.Series, len(results)) for i, result := range results { @@ -113,17 +114,23 @@ func PromResultToSeriesList(promReadResp prometheus.PromResp, tagOptions models. } } + metricName := string(tagOptions.MetricName()) tags := models.NewTags(len(result.Metric), tagOptions) for name, val := range result.Metric { - tags = tags.AddTag(models.Tag{ - Name: []byte(name), - Value: []byte(val), - }) + if name == metricName { + tags = tags.SetName([]byte(val)) + } else { + tags = tags.AddTag(models.Tag{ + Name: []byte(name), + Value: []byte(val), + }) + } } name, exists := tags.Name() if !exists { - return nil, errors.New("metric name does not exist") + // return nil, errors.New("metric name does not exist at all ever") + name = []byte("") } seriesList[i] = ts.NewSeries( diff --git a/src/query/ts/m3db/consolidators/step_accumulator.go b/src/query/ts/m3db/consolidators/step_accumulator.go index 598f49ca54..7ba2051187 100644 --- a/src/query/ts/m3db/consolidators/step_accumulator.go +++ b/src/query/ts/m3db/consolidators/step_accumulator.go @@ -27,19 +27,6 @@ import ( xts "github.com/m3db/m3/src/query/ts" ) -func removeStaleAccumulated( - earliestLookback time.Time, - dps []xts.Datapoint, -) []xts.Datapoint { - for i, dp := range dps { - if !dp.Timestamp.Before(earliestLookback) { - return dps[i:] - } - } - - return dps[:0] -} - // StepLookbackAccumulator is a helper for accumulating series in a step-wise // fashion. It takes a 'step' of values, which represents a vertical // slice of time across a list of series, and accumulates them when a @@ -49,7 +36,6 @@ type StepLookbackAccumulator struct { stepSize time.Duration earliestLookback time.Time datapoints [][]xts.Datapoint - reset []bool } // NewStepLookbackAccumulator creates an accumulator used for @@ -64,13 +50,11 @@ func NewStepLookbackAccumulator( datapoints[i] = make([]xts.Datapoint, 0, initLength) } - reset := make([]bool, resultSize) return &StepLookbackAccumulator{ lookbackDuration: lookbackDuration, stepSize: stepSize, earliestLookback: startTime.Add(-1 * lookbackDuration), datapoints: datapoints, - reset: reset, } } @@ -85,16 +69,6 @@ func (c *StepLookbackAccumulator) AddPointForIterator( return } - // TODO: the existing version of the step accumulator in Values.AlignToBounds - // resets incoming data points after accumulation; i.e. it will only keep - // points in the accumulation buffer if no other point comes in. This may not - // be the correct behaviour; investigate if it should be converted to keep - // the values instead. - if c.reset[i] { - c.datapoints[i] = c.datapoints[i][:0] - } - - c.reset[i] = false c.datapoints[i] = append(c.datapoints[i], xts.Datapoint{ Timestamp: dp.Timestamp, Value: dp.Value, @@ -111,8 +85,7 @@ func (c *StepLookbackAccumulator) AccumulateAndMoveToNext() []xts.Datapoints { for i, dps := range c.datapoints { accumulated[i] = make(xts.Datapoints, len(dps)) copy(accumulated[i], dps) - c.datapoints[i] = removeStaleAccumulated(c.earliestLookback, dps) - c.reset[i] = true + c.datapoints[i] = c.datapoints[i][:0] } return accumulated diff --git a/src/query/ts/m3db/consolidators/step_accumulator_test.go b/src/query/ts/m3db/consolidators/step_accumulator_test.go index b6c2671184..796ca06601 100644 --- a/src/query/ts/m3db/consolidators/step_accumulator_test.go +++ b/src/query/ts/m3db/consolidators/step_accumulator_test.go @@ -56,14 +56,14 @@ func TestAccumulator(t *testing.T) { expected[1] = xts.Datapoints{xts.Datapoint{Timestamp: start.Add(time.Minute), Value: 10}} assert.Equal(t, expected, actual) - // NB: lookback limit: start+1, point 1 is outside of the lookback period + // NB: lookback limit: start+1, should be reset actual = acc.AccumulateAndMoveToNext() + expected[1] = xts.Datapoints{} expected[0] = xts.Datapoints{} assert.Equal(t, expected, actual) - // NB: lookback limit: start+2 both points outside of the lookback period + // NB: lookback limit: start+2, should be reset actual = acc.AccumulateAndMoveToNext() - expected[1] = xts.Datapoints{} assert.Equal(t, expected, actual) acc.AddPointForIterator(ts.Datapoint{Timestamp: start.Add(2*time.Minute + time.Second*30), Value: 2}, 0) diff --git a/src/query/ts/m3db/convert.go b/src/query/ts/m3db/convert.go index 5b0437e3bb..a02ec012b8 100644 --- a/src/query/ts/m3db/convert.go +++ b/src/query/ts/m3db/convert.go @@ -105,7 +105,7 @@ func convertM3DBSegmentedBlockIterators( bounds models.Bounds, opts Options, ) ([]block.Block, error) { - // defer iterators.Close() + defer iterators.Close() blockBuilder := newEncodedBlockBuilder(opts) var ( iterAlloc = opts.IterAlloc() @@ -217,7 +217,11 @@ func updateSeriesBlockStarts( firstStart := blocks[0].blockStart if iterStart.Before(firstStart) { fillSize := firstStart.Sub(iterStart) - numberToFill := int(fillSize/stepSize) + 1 + numberToFill := int(fillSize / stepSize) + if fillSize%stepSize != 0 { + numberToFill++ + } + iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) } @@ -225,7 +229,11 @@ func updateSeriesBlockStarts( for i, bl := range blocks { blocks[i].readStart = iterStart fillSize := bl.blockStart.Add(bl.blockSize).Sub(iterStart) - numberToFill := int(fillSize/stepSize) + 1 + numberToFill := int(fillSize / stepSize) + if fillSize%stepSize != 0 { + numberToFill++ + } + iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) } diff --git a/src/query/ts/m3db/convert_test.go b/src/query/ts/m3db/convert_test.go index 51d8f45cd3..287cda3d55 100644 --- a/src/query/ts/m3db/convert_test.go +++ b/src/query/ts/m3db/convert_test.go @@ -140,7 +140,6 @@ func verifyBoundsAndGetBlockIndex(t *testing.T, bounds, sub models.Bounds) int { require.Equal(t, bounds.StepSize, sub.StepSize) require.Equal(t, blockSize, sub.Duration) diff := sub.Start.Sub(bounds.Start) - fmt.Println(diff, blockSize) require.Equal(t, 0, int(diff%blockSize)) return int(diff / blockSize) } diff --git a/src/query/ts/m3db/encoded_series_iterator.go b/src/query/ts/m3db/encoded_series_iterator.go index 4fef40cbc4..481cbd6d3f 100644 --- a/src/query/ts/m3db/encoded_series_iterator.go +++ b/src/query/ts/m3db/encoded_series_iterator.go @@ -94,7 +94,7 @@ func (it *encodedSeriesIter) Next() bool { continue } - for { + for i < len(values) { values[i] = it.consolidator.ConsolidateAndMoveToNext() i++ currentTime = currentTime.Add(it.bounds.StepSize) diff --git a/src/query/ts/m3db/encoded_series_iterator_test.go b/src/query/ts/m3db/encoded_series_iterator_test.go index ce89c7ecdd..42894db858 100644 --- a/src/query/ts/m3db/encoded_series_iterator_test.go +++ b/src/query/ts/m3db/encoded_series_iterator_test.go @@ -141,7 +141,7 @@ var consolidatedSeriesIteratorTestsSplitByBlock = []struct { }, { {7, nan, nan, nan, nan, 8}, - {nan, 30, nan, nan, nan, nan}, + {nan, nan, nan, nan, nan, nan}, {nan, nan, nan, 300, nan, nan}, }, { diff --git a/src/query/ts/m3db/encoded_step_iterator_test.go b/src/query/ts/m3db/encoded_step_iterator_test.go index 11adb337f2..e27a813554 100644 --- a/src/query/ts/m3db/encoded_step_iterator_test.go +++ b/src/query/ts/m3db/encoded_step_iterator_test.go @@ -161,7 +161,7 @@ var consolidatedStepIteratorTestsSplitByBlock = []struct { }, { {7, nan, nan}, - {nan, 30, nan}, + {nan, nan, nan}, {nan, nan, nan}, {nan, nan, 300}, {nan, nan, nan}, diff --git a/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go b/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go index 080570ce89..aae7d7d7e0 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go +++ b/src/query/ts/m3db/encoded_unconsolidated_iterator_test.go @@ -45,28 +45,28 @@ func TestUnconsolidatedStepIterator(t *testing.T) { {{}, {}, {}}, {{}, {10}, {}}, {{}, {20}, {100}}, - {{}, {20}, {100}}, + {{}, {}, {}}, {{}, {}, {}}, {{1}, {}, {}}, - {{1}, {}, {}}, + {{}, {}, {}}, {{2, 3}, {}, {}}, {{4}, {}, {200}}, - {{5}, {}, {200}}, + {{5}, {}, {}}, {{6}, {}, {}}, {{7}, {}, {}}, - {{7}, {30}, {}}, {{}, {30}, {}}, + {{}, {}, {}}, {{}, {}, {300}}, - {{}, {}, {300}}, - {{8}, {}, {}}, + {{}, {}, {}}, {{8}, {}, {}}, + {{}, {}, {}}, {{}, {40}, {}}, {{}, {}, {}}, {{}, {}, {400}}, - {{9}, {}, {400}}, {{9}, {}, {}}, + {{}, {}, {}}, {{}, {}, {500}}, - {{}, {}, {500}}, + {{}, {}, {}}, {{}, {}, {}}, {{}, {}, {}}, {{}, {}, {}}, @@ -76,7 +76,8 @@ func TestUnconsolidatedStepIterator(t *testing.T) { j := 0 opts := NewOptions(). SetLookbackDuration(1 * time.Minute). - SetSplitSeriesByBlock(true) + SetSplitSeriesByBlock(false) + blocks, bounds := generateBlocks(t, time.Minute, opts) for i, block := range blocks { unconsolidated, err := block.Unconsolidated() @@ -91,7 +92,6 @@ func TestUnconsolidatedStepIterator(t *testing.T) { step := iters.Current() vals := step.Values() actual := datapointsToFloatSlices(t, vals) - test.EqualsWithNans(t, expected[j], actual) j++ } @@ -104,16 +104,16 @@ func TestUnconsolidatedStepIterator(t *testing.T) { func TestUnconsolidatedSeriesIterator(t *testing.T) { expected := [][][]float64{ { - {}, {}, {}, {}, {}, {}, {1}, {1}, {2, 3}, {4}, {5}, {6}, {7}, {7}, - {}, {}, {}, {8}, {8}, {}, {}, {}, {9}, {9}, {}, {}, {}, {}, {}, {}, + {}, {}, {}, {}, {}, {}, {1}, {}, {2, 3}, {4}, {5}, {6}, {7}, {}, + {}, {}, {}, {8}, {}, {}, {}, {}, {9}, {}, {}, {}, {}, {}, {}, {}, }, { - {}, {}, {10}, {20}, {20}, {}, {}, {}, {}, {}, {}, {}, {}, {30}, - {30}, {}, {}, {}, {}, {40}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, + {}, {}, {10}, {20}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {30}, + {}, {}, {}, {}, {}, {40}, {}, {}, {}, {}, {}, {}, {}, {}, {}, {}, }, { - {}, {}, {}, {100}, {100}, {}, {}, {}, {}, {200}, {200}, {}, {}, {}, {}, - {300}, {300}, {}, {}, {}, {}, {400}, {400}, {}, {500}, {500}, {}, {}, {}, {}, + {}, {}, {}, {100}, {}, {}, {}, {}, {}, {200}, {}, {}, {}, {}, {}, + {300}, {}, {}, {}, {}, {}, {400}, {}, {}, {500}, {}, {}, {}, {}, {}, }, } diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index 087de67146..6f5b00a854 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -21,8 +21,6 @@ package m3db import ( - "fmt" - "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" xts "github.com/m3db/m3/src/query/ts" @@ -46,7 +44,6 @@ func (it *encodedSeriesIterUnconsolidated) Err() error { } func (it *encodedSeriesIterUnconsolidated) Next() bool { - fmt.Println("Calling next") if it.err != nil { return false } @@ -72,7 +69,7 @@ func (it *encodedSeriesIterUnconsolidated) Next() bool { return false } - alignedValues := values.AlignToBounds(it.meta.Bounds) + alignedValues := values.AlignToBoundsNoWriteForward(it.meta.Bounds) it.series = block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) return next diff --git a/src/query/ts/values.go b/src/query/ts/values.go index bf54ee7de3..2f94dd822c 100644 --- a/src/query/ts/values.go +++ b/src/query/ts/values.go @@ -46,8 +46,15 @@ type Values interface { // Datapoints returns all the datapoints Datapoints() []Datapoint - // AlignToBounds returns values aligned to the start time and duration + // AlignToBounds returns values aligned to given bounds. To belong to a step, + // values should be <= stepTime and not stale. AlignToBounds(bounds models.Bounds) []Datapoints + + // AlignToBoundsNoWriteForward returns values aligned to the start time + // and duration, but does not write points forward after aligning them. This + // differs from AlignToBounds which will write points forwards if no additional + // values are found in the values, adding an empty point instead. + AlignToBoundsNoWriteForward(bounds models.Bounds) []Datapoints } // A Datapoint is a single data value reported at a given time @@ -81,8 +88,10 @@ func (d Datapoints) Values() []float64 { return values } -// AlignToBounds returns values aligned to given bounds. To belong to a step, values should be <= stepTime and not stale -func (d Datapoints) AlignToBounds(bounds models.Bounds) []Datapoints { +func (d Datapoints) alignToBounds( + bounds models.Bounds, + writeForward bool, +) []Datapoints { numDatapoints := d.Len() steps := bounds.Steps() stepValues := make([]Datapoints, steps) @@ -103,11 +112,14 @@ func (d Datapoints) AlignToBounds(bounds models.Bounds) []Datapoints { singleStepValues = append(singleStepValues, point) } - // If no point found for this interval, reuse the last point as long as its not stale - if len(singleStepValues) == 0 && dpIdx > 0 { - prevPoint := d[dpIdx-1] - if t.Sub(prevPoint.Timestamp) <= models.LookbackDelta { - singleStepValues = Datapoints{prevPoint} + if writeForward { + // If writeForward is enabled and there is no point found for this + // interval, reuse the last point as long as its not stale + if len(singleStepValues) == 0 && dpIdx > 0 { + prevPoint := d[dpIdx-1] + if t.Sub(prevPoint.Timestamp) <= models.LookbackDelta { + singleStepValues = Datapoints{prevPoint} + } } } @@ -118,6 +130,22 @@ func (d Datapoints) AlignToBounds(bounds models.Bounds) []Datapoints { return stepValues } +// AlignToBoundsNoWriteForward returns values aligned to the start time +// and duration, but does not write points forward after aligning them. This +// differs from AlignToBounds which will write points forwards if no additional +// values are found in the values, adding an empty point instead. +func (d Datapoints) AlignToBoundsNoWriteForward( + bounds models.Bounds, +) []Datapoints { + return d.alignToBounds(bounds, false) +} + +// AlignToBounds returns values aligned to given bounds. To belong to a step, +// values should be <= stepTime and not stale. +func (d Datapoints) AlignToBounds(bounds models.Bounds) []Datapoints { + return d.alignToBounds(bounds, true) +} + // MutableValues is the interface for values that can be updated type MutableValues interface { Values @@ -159,8 +187,6 @@ func (b *fixedResolutionValues) Datapoints() []Datapoint { return datapoints } -// AlignToBounds returns values aligned to given bounds. -// TODO: Consider bounds as well func (b *fixedResolutionValues) AlignToBounds(_ models.Bounds) []Datapoints { values := make([]Datapoints, len(b.values)) for i := 0; i < b.Len(); i++ { @@ -170,6 +196,12 @@ func (b *fixedResolutionValues) AlignToBounds(_ models.Bounds) []Datapoints { return values } +func (b *fixedResolutionValues) AlignToBoundsNoWriteForward( + bb models.Bounds, +) []Datapoints { + return b.AlignToBounds(bb) +} + // StartTime returns the time the values start func (b *fixedResolutionValues) StartTime() time.Time { return b.startTime @@ -196,11 +228,21 @@ func (b *fixedResolutionValues) SetValueAt(n int, v float64) { } // NewFixedStepValues returns mutable values with fixed resolution -func NewFixedStepValues(resolution time.Duration, numSteps int, initialValue float64, startTime time.Time) FixedResolutionMutableValues { +func NewFixedStepValues( + resolution time.Duration, + numSteps int, + initialValue float64, + startTime time.Time, +) FixedResolutionMutableValues { return newFixedStepValues(resolution, numSteps, initialValue, startTime) } -func newFixedStepValues(resolution time.Duration, numSteps int, initialValue float64, startTime time.Time) *fixedResolutionValues { +func newFixedStepValues( + resolution time.Duration, + numSteps int, + initialValue float64, + startTime time.Time, +) *fixedResolutionValues { values := make([]float64, numSteps) // Faster way to initialize an array instead of a loop Memset(values, initialValue) diff --git a/src/query/ts/values_test.go b/src/query/ts/values_test.go index b1552bd12b..69068bd460 100644 --- a/src/query/ts/values_test.go +++ b/src/query/ts/values_test.go @@ -30,99 +30,107 @@ import ( "github.com/stretchr/testify/require" ) -type testSample struct { - input Datapoints - expected [][]float64 - bounds models.Bounds - description string -} - -func generateDatapoints(now time.Time, times []int) Datapoints { +func generateDatapoints(t time.Time, times []int) Datapoints { num := len(times) dps := make(Datapoints, len(times)) for i := 0; i < num; i++ { dps[i] = Datapoint{ - Timestamp: now.Add(time.Duration(times[i]) * time.Second), + Timestamp: t.Add(time.Duration(times[i]) * time.Second), Value: float64(i), } } return dps } -func createExamples() []testSample { - now := time.Time{} - samples := []testSample{ - { - input: generateDatapoints(now, []int{-1, 0, 10, 18, 28, 38}), - expected: [][]float64{{0, 1}, {2}, {3}, {4}}, - bounds: models.Bounds{ - Start: now, - Duration: 40 * time.Second, - StepSize: 10 * time.Second, - }, - description: "some points line up and others before", +var now = time.Now() +var samples = []struct { + input Datapoints + expected [][]float64 + expectedNoWriteForward [][]float64 + bounds models.Bounds + description string +}{ + { + input: generateDatapoints(now, []int{-1, 0, 10, 18, 28, 38}), + expected: [][]float64{{0, 1}, {2}, {3}, {4}}, + expectedNoWriteForward: [][]float64{{0, 1}, {2}, {3}, {4}}, + bounds: models.Bounds{ + Start: now, + Duration: 40 * time.Second, + StepSize: 10 * time.Second, }, - { - input: generateDatapoints(now, []int{1, 10, 18, 28}), - expected: [][]float64{{}, {0, 1}, {2}, {3}}, - bounds: models.Bounds{ - Start: now, - Duration: 40 * time.Second, - StepSize: 10 * time.Second, - }, - description: "only points after start", + description: "some points line up and others before", + }, + { + input: generateDatapoints(now, []int{1, 10, 18, 28}), + expected: [][]float64{{}, {0, 1}, {2}, {3}}, + expectedNoWriteForward: [][]float64{{}, {0, 1}, {2}, {3}}, + bounds: models.Bounds{ + Start: now, + Duration: 40 * time.Second, + StepSize: 10 * time.Second, }, - { - input: generateDatapoints(now, []int{0, 10, 18, 28}), - expected: [][]float64{{0}, {0}, {1}, {1}, {2}, {2}, {3}, {3}}, - bounds: models.Bounds{ - Start: now, - Duration: 40 * time.Second, - StepSize: 5 * time.Second, - }, - description: "half resolution so datapoints repeated", + description: "only points after start", + }, + { + input: generateDatapoints(now, []int{0, 10, 18, 28}), + expected: [][]float64{{0}, {0}, {1}, {1}, {2}, {2}, {3}, {3}}, + expectedNoWriteForward: [][]float64{{0}, {}, {1}, {}, {2}, {}, {3}, {}}, + bounds: models.Bounds{ + Start: now, + Duration: 40 * time.Second, + StepSize: 5 * time.Second, }, - { - input: generateDatapoints(now, []int{0, 10, 18, 28}), - expected: [][]float64{{0}, {1, 2}}, - bounds: models.Bounds{ - Start: now, - Duration: 40 * time.Second, - StepSize: 20 * time.Second, - }, - description: "double resolution so multiple dps in the same interval", + description: "half resolution so datapoints repeated", + }, + { + input: generateDatapoints(now, []int{0, 10, 18, 28}), + expected: [][]float64{{0}, {1, 2}}, + expectedNoWriteForward: [][]float64{{0}, {1, 2}}, + bounds: models.Bounds{ + Start: now, + Duration: 40 * time.Second, + StepSize: 20 * time.Second, }, - { - input: generateDatapoints(now, []int{0, 4, 5, 14}), - expected: [][]float64{{0}, {1, 2}, {2}, {3}}, - bounds: models.Bounds{ - Start: now, - Duration: 20 * time.Second, - StepSize: 5 * time.Second, - }, - description: "third interval has repeated datapoint", + description: "double resolution so multiple dps in the same interval", + }, + { + input: generateDatapoints(now, []int{0, 4, 5, 14}), + expected: [][]float64{{0}, {1, 2}, {2}, {3}}, + expectedNoWriteForward: [][]float64{{0}, {1, 2}, {}, {3}}, + bounds: models.Bounds{ + Start: now, + Duration: 20 * time.Second, + StepSize: 5 * time.Second, }, - { - input: generateDatapoints(now.Add(-10*time.Minute), []int{-1, 0, 10, 18, 28, 38}), - expected: [][]float64{{}, {}, {}, {}}, - bounds: models.Bounds{ - Start: now, - Duration: 40 * time.Second, - StepSize: 10 * time.Second, - }, - description: "skip really old datapoints", + description: "third interval has repeated datapoint", + }, + { + input: generateDatapoints(now.Add(-10*time.Minute), + []int{-1, 0, 10, 18, 28, 38}), + expected: [][]float64{{}, {}, {}, {}}, + expectedNoWriteForward: [][]float64{{}, {}, {}, {}}, + bounds: models.Bounds{ + Start: now, + Duration: 40 * time.Second, + StepSize: 10 * time.Second, }, - } - return samples + description: "skip really old datapoints", + }, } func TestDPAlign(t *testing.T) { - samples := createExamples() for _, sample := range samples { dpSlice := sample.input.AlignToBounds(sample.bounds) require.Len(t, dpSlice, len(sample.expected), sample.description) for i, dps := range dpSlice { assert.Equal(t, sample.expected[i], dps.Values()) } + + dpSlice = sample.input.AlignToBoundsNoWriteForward(sample.bounds) + require.Len(t, dpSlice, len(sample.expected), sample.description) + for i, dps := range dpSlice { + require.Equal(t, sample.expectedNoWriteForward[i], dps.Values()) + } } } From 3ecb3305ff2a4c4327240f66184c473c86a040f1 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Fri, 1 Feb 2019 17:22:24 -0500 Subject: [PATCH 10/17] Fixing merge issues --- src/query/storage/block.go | 2 +- src/query/storage/unconsolidated_test.go | 2 +- .../encoded_unconsolidated_series_iterator.go | 16 ++++++---------- src/query/ts/values.go | 10 ++++++---- src/query/ts/values_test.go | 2 +- 5 files changed, 15 insertions(+), 17 deletions(-) diff --git a/src/query/storage/block.go b/src/query/storage/block.go index 2dfcfa72b2..140e483275 100644 --- a/src/query/storage/block.go +++ b/src/query/storage/block.go @@ -241,7 +241,7 @@ type multiSeriesBlockStepIter struct { } func newMultiSeriesBlockStepIter(b multiSeriesBlock) block.UnconsolidatedStepIter { - values := make([][]ts.Datapoints, len(block.seriesList)) + values := make([][]ts.Datapoints, len(b.seriesList)) bounds := b.meta.Bounds for i, s := range b.seriesList { if b.consolidated { diff --git a/src/query/storage/unconsolidated_test.go b/src/query/storage/unconsolidated_test.go index d3b313c3a8..16a4f8da79 100644 --- a/src/query/storage/unconsolidated_test.go +++ b/src/query/storage/unconsolidated_test.go @@ -58,7 +58,7 @@ func buildUnconsolidatedBlock(t *testing.T) block.UnconsolidatedBlock { Interval: time.Minute, } - unconsolidated, err := NewMultiSeriesBlock(seriesList, fetchQuery) + unconsolidated, err := NewMultiSeriesBlock(seriesList, fetchQuery, time.Minute) require.NoError(t, err) return unconsolidated } diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index 98aab69301..2fdc4c540e 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -21,7 +21,6 @@ package m3db import ( - "sync" "time" "github.com/m3db/m3/src/dbnode/encoding" @@ -30,13 +29,13 @@ import ( ) type encodedSeriesIterUnconsolidated struct { - idx int + idx int lookbackDuration time.Duration - err error - meta block.Metadata - series block.UnconsolidatedSeries - seriesMeta []block.SeriesMeta - seriesIters []encoding.SeriesIterator + err error + meta block.Metadata + series block.UnconsolidatedSeries + seriesMeta []block.SeriesMeta + seriesIters []encoding.SeriesIterator } func (it *encodedSeriesIterUnconsolidated) Current() block.UnconsolidatedSeries { @@ -75,9 +74,6 @@ func (it *encodedSeriesIterUnconsolidated) Next() bool { alignedValues := values.AlignToBoundsNoWriteForward(it.meta.Bounds, it.lookbackDuration) it.series = block.NewUnconsolidatedSeries(alignedValues, it.seriesMeta[it.idx]) - it.mu.RUnlock() - return series, nil -} return next } diff --git a/src/query/ts/values.go b/src/query/ts/values.go index 6a6e33adc2..3eac0a5770 100644 --- a/src/query/ts/values.go +++ b/src/query/ts/values.go @@ -121,10 +121,12 @@ func (d Datapoints) alignToBounds( // If writeForward is enabled and there is no point found for this // interval, reuse the last point as long as its not stale - if len(singleStepValues) == 0 && dpIdx > 0 { - prevPoint := d[dpIdx-1] - if t.Sub(prevPoint.Timestamp) <= lookbackDuration { - singleStepValues = Datapoints{prevPoint} + if writeForward { + if len(singleStepValues) == 0 && dpIdx > 0 { + prevPoint := d[dpIdx-1] + if t.Sub(prevPoint.Timestamp) <= lookbackDuration { + singleStepValues = Datapoints{prevPoint} + } } } diff --git a/src/query/ts/values_test.go b/src/query/ts/values_test.go index be45d526cf..2bffca9c24 100644 --- a/src/query/ts/values_test.go +++ b/src/query/ts/values_test.go @@ -127,7 +127,7 @@ func TestDPAlign(t *testing.T) { assert.Equal(t, sample.expected[i], dps.Values()) } - dpSlice = sample.input.AlignToBoundsNoWriteForward(sample.bounds) + dpSlice = sample.input.AlignToBoundsNoWriteForward(sample.bounds, time.Minute) require.Len(t, dpSlice, len(sample.expected), sample.description) for i, dps := range dpSlice { require.Equal(t, sample.expectedNoWriteForward[i], dps.Values()) From 2aa13a3a8eabd568cdc5373614b1467634a3f824 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Mon, 4 Feb 2019 12:07:19 -0500 Subject: [PATCH 11/17] PR responses --- src/query/README.md | 4 ++-- .../v1/handler/prometheus/native/read_common.go | 6 +++++- src/query/executor/transform/lazy.go | 14 ++++++++++---- src/query/storage/validator/storage.go | 10 ++++++++-- 4 files changed, 25 insertions(+), 9 deletions(-) diff --git a/src/query/README.md b/src/query/README.md index 5ac745f1a4..af08513c70 100644 --- a/src/query/README.md +++ b/src/query/README.md @@ -28,7 +28,7 @@ Finally, you can spin up the two containers using `docker-compose` within the `d > Note: The default local ports for Prometheus and m3coordinator are `9090` and `7201`, respectively, and the default `prometheus.yml` file is `docker/prometheus.yml` > -> If you want to override these, you can pass in the following environment variables to the `docker-compose` command: +>If you want to override these, you can pass in the following environment variables to the `docker-compose` command: > > `LOCAL_PROM_PORT` > @@ -147,7 +147,7 @@ Input JSON: } ``` -Full request: `curl -X POST 'http://localhost:7201/api/v1/debug/validate_query?query=sum_over_time(process_cpu_seconds_total%7Binstance%3D~%22m3db_seed%3A7203%22%7D%5B60s%5D)&start=1549035092&end=1549035177&step=1' -d @lol.json --header "Content-Type: application/json"` +Full request: `curl -X POST 'localhost:7201/api/v1/debug/validate_query?start=1543431465&end=1543435045&step=14s&query=go_gc_duration_seconds%7Bquantile%3D%221%22%7D*2' -d @ --header "Content-Type: application/json"` [doc-img]: https://godoc.org/github.com/m3db/m3/src/query?status.svg [doc]: https://godoc.org/github.com/m3db/m3/src/query diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index 4d328a1ae6..b72c2a9fa2 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -167,7 +167,11 @@ func sortedBlocksToSeriesList(blockList []blockWithMeta) ([]*ts.Series, error) { numValues := 0 for _, block := range blockList { - b, _ := block.block.StepIter() + b, err := block.block.StepIter() + if err != nil { + return nil, err + } + numValues += b.StepCount() } diff --git a/src/query/executor/transform/lazy.go b/src/query/executor/transform/lazy.go index d53e21c62a..53ebe29ae5 100644 --- a/src/query/executor/transform/lazy.go +++ b/src/query/executor/transform/lazy.go @@ -94,7 +94,6 @@ func (s *stepIter) Next() bool { next := s.iter.Next() if !next { - s.err = s.iter.Err() return false } @@ -112,7 +111,11 @@ func (s *stepIter) Close() { } func (s *stepIter) Err() error { - return s.err + if s.err != nil { + return s.err + } + + return s.iter.Err() } func (s *stepIter) Current() block.Step { @@ -143,7 +146,11 @@ func (s *seriesIter) Close() { } func (s *seriesIter) Err() error { - return s.err + if s.err != nil { + return s.err + } + + return s.iter.Err() } func (s *seriesIter) Current() block.Series { @@ -157,7 +164,6 @@ func (s *seriesIter) Next() bool { next := s.iter.Next() if !next { - s.err = s.iter.Err() return false } diff --git a/src/query/storage/validator/storage.go b/src/query/storage/validator/storage.go index 2dc6825a61..d994e058db 100644 --- a/src/query/storage/validator/storage.go +++ b/src/query/storage/validator/storage.go @@ -129,10 +129,16 @@ func PromResultToSeriesList(promReadResp prometheus.PromResp, tagOptions models. } } + // NB: if there is no tag for series name here, the input is a Prometheus + // query result for a function that mutates the output tags and drops `name` + // which is a valid case. + // + // It's safe to set ts.Series.Name() here to a default value, as this field + // is used as a minor optimization for presenting grafana output, and as + // such, series names are not validated for equality. name, exists := tags.Name() if !exists { - // return nil, errors.New("metric name does not exist at all ever") - name = []byte("") + name = []byte("default") } seriesList[i] = ts.NewSeries( From 2f78a9951cbc051d049d2bec528e00a61052d7e4 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Mon, 4 Feb 2019 16:24:22 -0500 Subject: [PATCH 12/17] Refactoring encoded step iterators --- .../ts/m3db/consolidators/step_accumulator.go | 3 + .../m3db/consolidators/step_consolidator.go | 3 + src/query/ts/m3db/consolidators/types.go | 6 + ...ded_block_iterator.go => encoded_block.go} | 0 ...or_builder.go => encoded_block_builder.go} | 0 .../ts/m3db/encoded_block_unconsolidated.go | 13 -- .../ts/m3db/encoded_step_iterator_generic.go | 120 ++++++++++++++++++ .../encoded_unconsolidated_series_iterator.go | 13 ++ .../encoded_unconsolidated_step_iterator.go | 105 ++++----------- 9 files changed, 168 insertions(+), 95 deletions(-) rename src/query/ts/m3db/{encoded_block_iterator.go => encoded_block.go} (100%) rename src/query/ts/m3db/{encoded_block_iterator_builder.go => encoded_block_builder.go} (100%) create mode 100644 src/query/ts/m3db/encoded_step_iterator_generic.go diff --git a/src/query/ts/m3db/consolidators/step_accumulator.go b/src/query/ts/m3db/consolidators/step_accumulator.go index 7ba2051187..cc8a49a71b 100644 --- a/src/query/ts/m3db/consolidators/step_accumulator.go +++ b/src/query/ts/m3db/consolidators/step_accumulator.go @@ -38,6 +38,9 @@ type StepLookbackAccumulator struct { datapoints [][]xts.Datapoint } +// Ensure StepLookbackAccumulator satisfies StepCollector +var _ StepCollector = (*StepLookbackAccumulator)(nil) + // NewStepLookbackAccumulator creates an accumulator used for // step iteration across a series list with a given lookback. func NewStepLookbackAccumulator( diff --git a/src/query/ts/m3db/consolidators/step_consolidator.go b/src/query/ts/m3db/consolidators/step_consolidator.go index f435f7d431..83fd2f27dc 100644 --- a/src/query/ts/m3db/consolidators/step_consolidator.go +++ b/src/query/ts/m3db/consolidators/step_consolidator.go @@ -54,6 +54,9 @@ type StepLookbackConsolidator struct { fn ConsolidationFunc } +// Ensure StepLookbackConsolidator satisfies StepCollector +var _ StepCollector = (*StepLookbackConsolidator)(nil) + // NewStepLookbackConsolidator creates a multivalue consolidator used for // step iteration across a series list with a given lookback. func NewStepLookbackConsolidator( diff --git a/src/query/ts/m3db/consolidators/types.go b/src/query/ts/m3db/consolidators/types.go index a797769e8a..0e2180e7f2 100644 --- a/src/query/ts/m3db/consolidators/types.go +++ b/src/query/ts/m3db/consolidators/types.go @@ -26,6 +26,12 @@ import ( "github.com/m3db/m3/src/dbnode/ts" ) +// StepCollector is implemented by any accumulators or consolidators working on +// stepwise iteration. +type StepCollector interface { + AddPointForIterator(ts.Datapoint, int) +} + // ConsolidationFunc consolidates a bunch of datapoints into a single float value type ConsolidationFunc func(datapoints []ts.Datapoint) float64 diff --git a/src/query/ts/m3db/encoded_block_iterator.go b/src/query/ts/m3db/encoded_block.go similarity index 100% rename from src/query/ts/m3db/encoded_block_iterator.go rename to src/query/ts/m3db/encoded_block.go diff --git a/src/query/ts/m3db/encoded_block_iterator_builder.go b/src/query/ts/m3db/encoded_block_builder.go similarity index 100% rename from src/query/ts/m3db/encoded_block_iterator_builder.go rename to src/query/ts/m3db/encoded_block_builder.go diff --git a/src/query/ts/m3db/encoded_block_unconsolidated.go b/src/query/ts/m3db/encoded_block_unconsolidated.go index b30b7f910a..e87f31a1cb 100644 --- a/src/query/ts/m3db/encoded_block_unconsolidated.go +++ b/src/query/ts/m3db/encoded_block_unconsolidated.go @@ -72,16 +72,3 @@ func (b *encodedBlockUnconsolidated) WithMetadata( seriesMetas: seriesMetas, }, nil } - -func (b *encodedBlockUnconsolidated) SeriesIter() ( - block.UnconsolidatedSeriesIter, - error, -) { - return &encodedSeriesIterUnconsolidated{ - idx: -1, - meta: b.meta, - seriesMeta: b.seriesMetas, - seriesIters: b.seriesBlockIterators, - lookbackDuration: b.lookback, - }, nil -} diff --git a/src/query/ts/m3db/encoded_step_iterator_generic.go b/src/query/ts/m3db/encoded_step_iterator_generic.go new file mode 100644 index 0000000000..de11eb744c --- /dev/null +++ b/src/query/ts/m3db/encoded_step_iterator_generic.go @@ -0,0 +1,120 @@ +// Copyright (c) 2019 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package m3db + +import ( + "time" + + "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/query/ts/m3db/consolidators" +) + +type encodedStepIterWithCollector struct { + lastBlock bool + err error + collector consolidators.StepCollector + seriesPeek []peekValue + seriesIters []encoding.SeriesIterator +} + +func newEncodedStepIterWithCollector( + lastBlock bool, + collector consolidators.StepCollector, + seriesIters []encoding.SeriesIterator, +) *encodedStepIterWithCollector { + return &encodedStepIterWithCollector{ + lastBlock: lastBlock, + collector: collector, + seriesIters: seriesIters, + // TODO: pool + seriesPeek: make([]peekValue, len(seriesIters)), + } +} + +// Moves to the next step for the i-th series in the block, populating +// the collector for that step. Will keep reading values until either +// hitting the next step boundary and returning, or until encountering +// a point beyond the step boundary. This point is then added to a stored +// peeked value that is consumed on the next pass. +func (it *encodedStepIterWithCollector) nextForStep( + i int, + stepTime time.Time, +) { + peek := it.seriesPeek[i] + if peek.finished { + // No next value in this iterator. + return + } + + if peek.started { + point := peek.point + if point.Timestamp.After(stepTime) { + // This point exists further than the current step + // There are next values, but this point should be NaN. + return + } + + // Currently at a potentially viable data point. + // Record previously peeked value, and all potentially valid + // values, then apply consolidation function to them to get the + // consolidated point. + it.collector.AddPointForIterator(point, i) + // clear peeked point. + it.seriesPeek[i].started = false + // If this point is currently at the boundary, finish here as there is no + // need to check any additional points in the enclosed iterator. + if point.Timestamp.Equal(stepTime) { + return + } + } + + iter := it.seriesIters[i] + // Read through iterator until finding a data point outside of the + // range of this consolidated step; then consolidate those points into + // a value, set the next peek value. + for iter.Next() { + dp, _, _ := iter.Current() + + // If this datapoint is before the current timestamp, add it as a + // consolidation candidate. + if !dp.Timestamp.After(stepTime) { + it.seriesPeek[i].started = false + it.collector.AddPointForIterator(dp, i) + } else { + // This point exists further than the current step. + // Set peeked value to this point, then consolidate the retrieved + // series. + it.seriesPeek[i].point = dp + it.seriesPeek[i].started = true + return + } + } + + if err := iter.Err(); err != nil { + it.err = err + } +} + +func (it *encodedStepIterWithCollector) nextAtTime(stepTime time.Time) { + for i := range it.seriesIters { + it.nextForStep(i, stepTime) + } +} diff --git a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go index 2fdc4c540e..e6bd8ef8f6 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_series_iterator.go @@ -38,6 +38,19 @@ type encodedSeriesIterUnconsolidated struct { seriesIters []encoding.SeriesIterator } +func (b *encodedBlockUnconsolidated) SeriesIter() ( + block.UnconsolidatedSeriesIter, + error, +) { + return &encodedSeriesIterUnconsolidated{ + idx: -1, + meta: b.meta, + seriesMeta: b.seriesMetas, + seriesIters: b.seriesBlockIterators, + lookbackDuration: b.lookback, + }, nil +} + func (it *encodedSeriesIterUnconsolidated) Current() block.UnconsolidatedSeries { return it.series } diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index 9074366dbc..493c5fd04b 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -23,26 +23,19 @@ package m3db import ( "time" - "github.com/m3db/m3/src/dbnode/encoding" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/storage" - xts "github.com/m3db/m3/src/query/ts" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIterUnconsolidated struct { - lastBlock bool - idx int - err error - meta block.Metadata - seriesMeta []block.SeriesMeta - seriesIters []encoding.SeriesIterator - expandedSeries [][]xts.Datapoints - - stepTime time.Time - lookback time.Duration - accumulator *consolidators.StepLookbackAccumulator - seriesPeek []peekValue + idx int + err error + stepTime time.Time + meta block.Metadata + seriesMeta []block.SeriesMeta + accumulator *consolidators.StepLookbackAccumulator + collectorIterator *encodedStepIterWithCollector } func (b *encodedBlockUnconsolidated) StepIter() ( @@ -57,17 +50,15 @@ func (b *encodedBlockUnconsolidated) StepIter() ( len(b.seriesBlockIterators), ) + collectorIterator := newEncodedStepIterWithCollector( + b.lastBlock, accumulator, b.seriesBlockIterators) + return &encodedStepIterUnconsolidated{ - idx: -1, - meta: b.meta, - seriesMeta: b.seriesMetas, - seriesIters: b.seriesBlockIterators, - lastBlock: b.lastBlock, - expandedSeries: make([][]xts.Datapoints, len(b.seriesBlockIterators)), - - lookback: b.lookback, - accumulator: accumulator, - seriesPeek: make([]peekValue, len(b.seriesBlockIterators)), + idx: -1, + meta: b.meta, + seriesMeta: b.seriesMetas, + accumulator: accumulator, + collectorIterator: collectorIterator, }, nil } @@ -76,57 +67,6 @@ func (it *encodedStepIterUnconsolidated) Current() block.UnconsolidatedStep { return storage.NewUnconsolidatedStep(it.stepTime, points) } -func (it *encodedStepIterUnconsolidated) nextForStep( - i int, - stepTime time.Time, -) { - peek := it.seriesPeek[i] - if peek.finished { - // No next value in this iterator - return - } - - if peek.started { - point := peek.point - if point.Timestamp.After(stepTime) { - // This point exists further than the current step - // There are next values, but current step is empty. - return - } - - // clear peeked point. - it.seriesPeek[i].started = false - // Currently at a potentially viable data point. - // Record previously peeked value, and all potentially valid values. - it.accumulator.AddPointForIterator(point, i) - - // If at boundary, add the point as the current value. - if point.Timestamp.Equal(stepTime) { - return - } - } - - iter := it.seriesIters[i] - // Read through iterator until finding a data point outside of the - // range of this step; then set the next peek value. - for iter.Next() { - dp, _, _ := iter.Current() - - // If this datapoint is before the current timestamp, add it to - // the accumulator - if !dp.Timestamp.After(stepTime) { - it.seriesPeek[i].started = false - it.accumulator.AddPointForIterator(dp, i) - } else { - // This point exists further than the current step; set peeked value - // to this point. - it.seriesPeek[i].point = dp - it.seriesPeek[i].started = true - return - } - } -} - func (it *encodedStepIterUnconsolidated) Next() bool { if it.err != nil { return false @@ -134,7 +74,6 @@ func (it *encodedStepIterUnconsolidated) Next() bool { it.idx++ next := it.idx < it.meta.Bounds.Steps() - if !next { return false } @@ -145,11 +84,9 @@ func (it *encodedStepIterUnconsolidated) Next() bool { return false } - for i, iter := range it.seriesIters { - it.nextForStep(i, stepTime) - if it.err = iter.Err(); it.err != nil { - return false - } + it.collectorIterator.nextAtTime(stepTime) + if it.err = it.collectorIterator.err; it.err != nil { + return false } it.stepTime = stepTime @@ -162,7 +99,11 @@ func (it *encodedStepIterUnconsolidated) StepCount() int { } func (it *encodedStepIterUnconsolidated) Err() error { - return it.err + if it.err != nil { + return it.err + } + + return it.collectorIterator.err } func (it *encodedStepIterUnconsolidated) SeriesMeta() []block.SeriesMeta { From df4dcd6599707e3aa12678f82200d7f3c934529d Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Mon, 4 Feb 2019 20:02:12 -0500 Subject: [PATCH 13/17] Refactored encoded step iterator. --- .../ts/m3db/consolidators/step_accumulator.go | 2 +- .../m3db/consolidators/step_consolidator.go | 2 +- src/query/ts/m3db/consolidators/types.go | 4 +- src/query/ts/m3db/encoded_step_iterator.go | 165 ++++-------------- .../encoded_unconsolidated_step_iterator.go | 5 +- 5 files changed, 40 insertions(+), 138 deletions(-) diff --git a/src/query/ts/m3db/consolidators/step_accumulator.go b/src/query/ts/m3db/consolidators/step_accumulator.go index cc8a49a71b..fdef656a0d 100644 --- a/src/query/ts/m3db/consolidators/step_accumulator.go +++ b/src/query/ts/m3db/consolidators/step_accumulator.go @@ -38,7 +38,7 @@ type StepLookbackAccumulator struct { datapoints [][]xts.Datapoint } -// Ensure StepLookbackAccumulator satisfies StepCollector +// Ensure StepLookbackAccumulator satisfies StepCollector. var _ StepCollector = (*StepLookbackAccumulator)(nil) // NewStepLookbackAccumulator creates an accumulator used for diff --git a/src/query/ts/m3db/consolidators/step_consolidator.go b/src/query/ts/m3db/consolidators/step_consolidator.go index 83fd2f27dc..db9f1dcea4 100644 --- a/src/query/ts/m3db/consolidators/step_consolidator.go +++ b/src/query/ts/m3db/consolidators/step_consolidator.go @@ -54,7 +54,7 @@ type StepLookbackConsolidator struct { fn ConsolidationFunc } -// Ensure StepLookbackConsolidator satisfies StepCollector +// Ensure StepLookbackConsolidator satisfies StepCollector. var _ StepCollector = (*StepLookbackConsolidator)(nil) // NewStepLookbackConsolidator creates a multivalue consolidator used for diff --git a/src/query/ts/m3db/consolidators/types.go b/src/query/ts/m3db/consolidators/types.go index 0e2180e7f2..042fac7180 100644 --- a/src/query/ts/m3db/consolidators/types.go +++ b/src/query/ts/m3db/consolidators/types.go @@ -32,10 +32,10 @@ type StepCollector interface { AddPointForIterator(ts.Datapoint, int) } -// ConsolidationFunc consolidates a bunch of datapoints into a single float value +// ConsolidationFunc consolidates a bunch of datapoints into a single float value. type ConsolidationFunc func(datapoints []ts.Datapoint) float64 -// TakeLast is a consolidation function which takes the last datapoint +// TakeLast is a consolidation function which takes the last datapoint. func TakeLast(values []ts.Datapoint) float64 { for i := len(values) - 1; i >= 0; i-- { value := values[i].Value diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index d23e98d9bd..bafe7f3a6e 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -23,24 +23,18 @@ package m3db import ( "time" - "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/query/block" - "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIter struct { - lastBlock bool - started bool - currentTime time.Time - err error - bounds models.Bounds - meta block.Metadata - seriesMeta []block.SeriesMeta - seriesIters []encoding.SeriesIterator - seriesPeek []peekValue - consolidator *consolidators.StepLookbackConsolidator + lastBlock bool + err error + stepTime time.Time + meta block.Metadata + seriesMeta []block.SeriesMeta + consolidator *consolidators.StepLookbackConsolidator + collectorIterator *encodedStepIterWithCollector } func (b *encodedBlock) stepIter() block.StepIter { @@ -52,14 +46,17 @@ func (b *encodedBlock) stepIter() block.StepIter { len(b.seriesBlockIterators), cs.consolidationFn, ) + + collectorIterator := newEncodedStepIterWithCollector(b.lastBlock, + consolidator, b.seriesBlockIterators) + return &encodedStepIter{ - lastBlock: b.lastBlock, - currentTime: cs.currentTime, - bounds: cs.bounds, - meta: b.meta, - seriesMeta: b.seriesMetas, - seriesIters: b.seriesBlockIterators, - consolidator: consolidator, + lastBlock: b.lastBlock, + stepTime: cs.currentTime, + meta: b.meta, + seriesMeta: b.seriesMetas, + consolidator: consolidator, + collectorIterator: collectorIterator, } } @@ -73,135 +70,35 @@ func (s *encodedStep) Values() []float64 { return s.values } func (it *encodedStepIter) Current() block.Step { return &encodedStep{ - time: it.currentTime, + time: it.stepTime, values: it.consolidator.ConsolidateAndMoveToNext(), } } -// Moves to the next consolidated step for the i-th series in the block, -// populating the consolidator for that step. Will keep reading values -// until either hitting the next step boundary and returning, or until -// encountering a value beyond the boundary, at which point it adds it -// to a stored peeked value that is consumed on the next pass. -func (it *encodedStepIter) nextConsolidatedForStep(i int) { - peek := it.seriesPeek[i] - if peek.finished { - // No next value in this iterator - return - } - - if peek.started { - point := peek.point - if point.Timestamp.After(it.currentTime) { - // This point exists further than the current step - // There are next values, but this point should be NaN - return - } - - // Currently at a potentially viable data point. - // Record previously peeked value, and all potentially valid - // values, then apply consolidation function to them to get the - // consolidated point. - it.consolidator.AddPointForIterator(point, i) - // clear peeked point. - it.seriesPeek[i].started = false - // If at boundary, add the point as the current value. - if point.Timestamp.Equal(it.currentTime) { - return - } - } - - iter := it.seriesIters[i] - // Read through iterator until finding a data point outside of the - // range of this consolidated step; then consolidate those points into - // a value, set the next peek value. - for iter.Next() { - dp, _, _ := iter.Current() - - // If this datapoint is before the current timestamp, add it as a - // consolidation candidate. - if !dp.Timestamp.After(it.currentTime) { - it.seriesPeek[i].started = false - it.consolidator.AddPointForIterator(dp, i) - } else { - // This point exists further than the current step. - // Set peeked value to this point, then consolidate the retrieved - // series. - it.seriesPeek[i].point = dp - it.seriesPeek[i].started = true - return - } - } - - if err := iter.Err(); err != nil { - it.err = err - } -} - -func (it *encodedStepIter) nextConsolidated() { - end := it.bounds.End() - // Check that current time is not before end since end is exclusive - if it.currentTime.After(end) { - return - } - - for i := range it.seriesIters { - it.nextConsolidatedForStep(i) - } -} - -// Need to run an initial step; if there are any values -// that appear at exactly the start, they must be added. -func (it *encodedStepIter) initialStep() { - it.seriesPeek = make([]peekValue, len(it.seriesIters)) - for i, iter := range it.seriesIters { - if iter.Next() { - dp, _, _ := iter.Current() - if dp.Timestamp.Equal(it.bounds.Start) { - it.consolidator.AddPointForIterator(dp, i) - } else { - it.seriesPeek[i] = peekValue{ - point: ts.Datapoint{ - Timestamp: dp.Timestamp, - Value: dp.Value, - }, - started: true, - } - } - } - - if err := iter.Err(); err != nil { - it.err = err - return - } - } -} - func (it *encodedStepIter) Next() bool { if it.err != nil { return false } - checkNextTime := it.currentTime.Add(it.bounds.StepSize * 2) - if it.bounds.End().Before(checkNextTime) { + bounds := it.meta.Bounds + checkNextTime := it.stepTime.Add(bounds.StepSize * 2) + if bounds.End().Before(checkNextTime) { return false } - if !it.started { - it.initialStep() - it.started = true - } else { - it.currentTime = it.currentTime.Add(it.bounds.StepSize) - it.nextConsolidated() + it.collectorIterator.nextAtTime(it.stepTime) + it.stepTime = it.stepTime.Add(bounds.StepSize) + if it.err = it.collectorIterator.err; it.err != nil { + return false } - nextTime := it.currentTime.Add(it.bounds.StepSize) + nextTime := it.stepTime.Add(bounds.StepSize) // Has next values if the next step is before end boundary. - return !it.bounds.End().Before(nextTime) + return !bounds.End().Before(nextTime) } func (it *encodedStepIter) StepCount() int { - return it.bounds.Steps() + return it.meta.Bounds.Steps() } func (it *encodedStepIter) SeriesMeta() []block.SeriesMeta { @@ -213,7 +110,11 @@ func (it *encodedStepIter) Meta() block.Metadata { } func (it *encodedStepIter) Err() error { - return it.err + if it.err != nil { + return it.err + } + + return it.collectorIterator.err } func (it *encodedStepIter) Close() { diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index 493c5fd04b..e3924a531d 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -72,13 +72,14 @@ func (it *encodedStepIterUnconsolidated) Next() bool { return false } + bounds := it.meta.Bounds it.idx++ - next := it.idx < it.meta.Bounds.Steps() + next := it.idx < bounds.Steps() if !next { return false } - stepTime, err := it.meta.Bounds.TimeForIndex(it.idx) + stepTime, err := bounds.TimeForIndex(it.idx) if err != nil { it.err = err return false From 9beda82da50172b30dd1d048d15345a12676b135 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Wed, 6 Feb 2019 18:44:48 -0500 Subject: [PATCH 14/17] Refactoring step iterators --- .../api/v1/handler/prometheus/native/read.go | 5 ++++- src/query/ts/m3db/convert.go | 2 ++ src/query/ts/m3db/encoded_step_iterator.go | 7 ++----- .../encoded_unconsolidated_step_iterator.go | 19 +++++-------------- 4 files changed, 13 insertions(+), 20 deletions(-) diff --git a/src/query/api/v1/handler/prometheus/native/read.go b/src/query/api/v1/handler/prometheus/native/read.go index bd157ffe94..900711e864 100644 --- a/src/query/api/v1/handler/prometheus/native/read.go +++ b/src/query/api/v1/handler/prometheus/native/read.go @@ -143,7 +143,10 @@ func (h *PromReadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } // ServeHTTPWithEngine returns query results from the storage -func (h *PromReadHandler) ServeHTTPWithEngine(w http.ResponseWriter, r *http.Request, engine *executor.Engine) ([]*ts.Series, models.RequestParams, *RespError) { +func (h *PromReadHandler) ServeHTTPWithEngine( + w http.ResponseWriter, + r *http.Request, engine *executor.Engine, +) ([]*ts.Series, models.RequestParams, *RespError) { ctx := context.WithValue(r.Context(), handler.HeaderKey, r.Header) logger := logging.WithContext(ctx) diff --git a/src/query/ts/m3db/convert.go b/src/query/ts/m3db/convert.go index 8abc4f8b46..028f81cb61 100644 --- a/src/query/ts/m3db/convert.go +++ b/src/query/ts/m3db/convert.go @@ -119,6 +119,7 @@ func convertM3DBSegmentedBlockIterators( seriesIterator, iterAlloc, bounds, + pools, checkedPools, ) @@ -151,6 +152,7 @@ func blockReplicasFromSeriesIterator( seriesIterator encoding.SeriesIterator, iterAlloc encoding.ReaderIteratorAllocate, bounds models.Bounds, + pools encoding.IteratorPools, checkedPools pool.CheckedBytesPool, ) (seriesBlocks, error) { blocks := make(seriesBlocks, 0, bounds.Steps()) diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index bafe7f3a6e..3e5a6f61a0 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -81,8 +81,7 @@ func (it *encodedStepIter) Next() bool { } bounds := it.meta.Bounds - checkNextTime := it.stepTime.Add(bounds.StepSize * 2) - if bounds.End().Before(checkNextTime) { + if bounds.End().Before(it.stepTime) { return false } @@ -92,9 +91,7 @@ func (it *encodedStepIter) Next() bool { return false } - nextTime := it.stepTime.Add(bounds.StepSize) - // Has next values if the next step is before end boundary. - return !bounds.End().Before(nextTime) + return !bounds.End().Before(it.stepTime) } func (it *encodedStepIter) StepCount() int { diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index e3924a531d..bd2238f341 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -29,7 +29,6 @@ import ( ) type encodedStepIterUnconsolidated struct { - idx int err error stepTime time.Time meta block.Metadata @@ -54,7 +53,7 @@ func (b *encodedBlockUnconsolidated) StepIter() ( b.lastBlock, accumulator, b.seriesBlockIterators) return &encodedStepIterUnconsolidated{ - idx: -1, + stepTime: cs.currentTime, meta: b.meta, seriesMeta: b.seriesMetas, accumulator: accumulator, @@ -73,25 +72,17 @@ func (it *encodedStepIterUnconsolidated) Next() bool { } bounds := it.meta.Bounds - it.idx++ - next := it.idx < bounds.Steps() - if !next { + if bounds.End().Before(it.stepTime) { return false } - stepTime, err := bounds.TimeForIndex(it.idx) - if err != nil { - it.err = err - return false - } - - it.collectorIterator.nextAtTime(stepTime) + it.collectorIterator.nextAtTime(it.stepTime) + it.stepTime = it.stepTime.Add(bounds.StepSize) if it.err = it.collectorIterator.err; it.err != nil { return false } - it.stepTime = stepTime - return next + return !bounds.End().Before(it.stepTime) } func (it *encodedStepIterUnconsolidated) StepCount() int { From 402e21e3a6216b2e202daafca27529f708153a0c Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Thu, 7 Feb 2019 10:52:05 -0500 Subject: [PATCH 15/17] Finish refactoring encoded step iterators --- src/query/ts/m3db/encoded_block.go | 10 +- src/query/ts/m3db/encoded_series_iterator.go | 8 +- src/query/ts/m3db/encoded_step_iterator.go | 102 +++++------------- .../ts/m3db/encoded_step_iterator_generic.go | 57 +++++++++- .../encoded_unconsolidated_step_iterator.go | 76 +++---------- 5 files changed, 100 insertions(+), 153 deletions(-) diff --git a/src/query/ts/m3db/encoded_block.go b/src/query/ts/m3db/encoded_block.go index 2e3e3e53ec..a94b7556af 100644 --- a/src/query/ts/m3db/encoded_block.go +++ b/src/query/ts/m3db/encoded_block.go @@ -159,16 +159,8 @@ func (b *encodedBlock) WithMetadata( b.lookback, b.lastBlock, ) + bl.meta = meta bl.seriesMetas = seriesMetas - return &bl, nil } - -func (b *encodedBlock) StepIter() (block.StepIter, error) { - return b.stepIter(), nil -} - -func (b *encodedBlock) SeriesIter() (block.SeriesIter, error) { - return b.seriesIter(), nil -} diff --git a/src/query/ts/m3db/encoded_series_iterator.go b/src/query/ts/m3db/encoded_series_iterator.go index 481cbd6d3f..b42119f9e0 100644 --- a/src/query/ts/m3db/encoded_series_iterator.go +++ b/src/query/ts/m3db/encoded_series_iterator.go @@ -41,7 +41,10 @@ type encodedSeriesIter struct { consolidator *consolidators.SeriesLookbackConsolidator } -func (b *encodedBlock) seriesIter() block.SeriesIter { +func (b *encodedBlock) SeriesIter() ( + block.SeriesIter, + error, +) { cs := b.consolidation bounds := cs.bounds consolidator := consolidators.NewSeriesLookbackConsolidator( @@ -50,6 +53,7 @@ func (b *encodedBlock) seriesIter() block.SeriesIter { cs.currentTime, cs.consolidationFn, ) + return &encodedSeriesIter{ idx: -1, meta: b.meta, @@ -57,7 +61,7 @@ func (b *encodedBlock) seriesIter() block.SeriesIter { seriesMeta: b.seriesMetas, seriesIters: b.seriesBlockIterators, consolidator: consolidator, - } + }, nil } func (it *encodedSeriesIter) Err() error { diff --git a/src/query/ts/m3db/encoded_step_iterator.go b/src/query/ts/m3db/encoded_step_iterator.go index 3e5a6f61a0..0316434465 100644 --- a/src/query/ts/m3db/encoded_step_iterator.go +++ b/src/query/ts/m3db/encoded_step_iterator.go @@ -21,100 +21,48 @@ package m3db import ( - "time" - "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIter struct { - lastBlock bool - err error - stepTime time.Time - meta block.Metadata - seriesMeta []block.SeriesMeta - consolidator *consolidators.StepLookbackConsolidator - collectorIterator *encodedStepIterWithCollector + consolidator *consolidators.StepLookbackConsolidator + encodedStepIterWithCollector } -func (b *encodedBlock) stepIter() block.StepIter { +func (b *encodedBlock) StepIter() ( + block.StepIter, + error, +) { cs := b.consolidation + iters := b.seriesBlockIterators consolidator := consolidators.NewStepLookbackConsolidator( b.lookback, cs.bounds.StepSize, cs.currentTime, - len(b.seriesBlockIterators), + len(iters), cs.consolidationFn, ) - collectorIterator := newEncodedStepIterWithCollector(b.lastBlock, - consolidator, b.seriesBlockIterators) - return &encodedStepIter{ - lastBlock: b.lastBlock, - stepTime: cs.currentTime, - meta: b.meta, - seriesMeta: b.seriesMetas, - consolidator: consolidator, - collectorIterator: collectorIterator, - } -} - -type encodedStep struct { - time time.Time - values []float64 + consolidator: consolidator, + encodedStepIterWithCollector: encodedStepIterWithCollector{ + lastBlock: b.lastBlock, + + stepTime: cs.currentTime, + meta: b.meta, + seriesMeta: b.seriesMetas, + + collector: consolidator, + seriesPeek: make([]peekValue, len(iters)), + seriesIters: iters, + }, + }, nil } -func (s *encodedStep) Time() time.Time { return s.time } -func (s *encodedStep) Values() []float64 { return s.values } - func (it *encodedStepIter) Current() block.Step { - return &encodedStep{ - time: it.stepTime, - values: it.consolidator.ConsolidateAndMoveToNext(), - } -} - -func (it *encodedStepIter) Next() bool { - if it.err != nil { - return false - } - - bounds := it.meta.Bounds - if bounds.End().Before(it.stepTime) { - return false - } - - it.collectorIterator.nextAtTime(it.stepTime) - it.stepTime = it.stepTime.Add(bounds.StepSize) - if it.err = it.collectorIterator.err; it.err != nil { - return false - } - - return !bounds.End().Before(it.stepTime) -} - -func (it *encodedStepIter) StepCount() int { - return it.meta.Bounds.Steps() -} - -func (it *encodedStepIter) SeriesMeta() []block.SeriesMeta { - return it.seriesMeta -} - -func (it *encodedStepIter) Meta() block.Metadata { - return it.meta -} - -func (it *encodedStepIter) Err() error { - if it.err != nil { - return it.err - } - - return it.collectorIterator.err -} - -func (it *encodedStepIter) Close() { - // noop, as the resources at the step may still be in use; - // instead call Close() on the encodedBlock that generated this + return block.NewColStep( + it.stepTime, + it.consolidator.ConsolidateAndMoveToNext(), + ) } diff --git a/src/query/ts/m3db/encoded_step_iterator_generic.go b/src/query/ts/m3db/encoded_step_iterator_generic.go index de11eb744c..7a0e48c6d2 100644 --- a/src/query/ts/m3db/encoded_step_iterator_generic.go +++ b/src/query/ts/m3db/encoded_step_iterator_generic.go @@ -24,12 +24,18 @@ import ( "time" "github.com/m3db/m3/src/dbnode/encoding" + "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIterWithCollector struct { - lastBlock bool - err error + lastBlock bool + err error + + stepTime time.Time + meta block.Metadata + seriesMeta []block.SeriesMeta + collector consolidators.StepCollector seriesPeek []peekValue seriesIters []encoding.SeriesIterator @@ -113,8 +119,51 @@ func (it *encodedStepIterWithCollector) nextForStep( } } -func (it *encodedStepIterWithCollector) nextAtTime(stepTime time.Time) { +func (it *encodedStepIterWithCollector) nextAtTime(time time.Time) { + for i := range it.seriesIters { + it.nextForStep(i, it.stepTime) + } +} + +func (it *encodedStepIterWithCollector) Next() bool { + if it.err != nil { + return false + } + + bounds := it.meta.Bounds + if bounds.End().Before(it.stepTime) { + return false + } + for i := range it.seriesIters { - it.nextForStep(i, stepTime) + it.nextForStep(i, it.stepTime) + } + + if it.err != nil { + return false } + + it.stepTime = it.stepTime.Add(bounds.StepSize) + return !bounds.End().Before(it.stepTime) +} + +func (it *encodedStepIterWithCollector) StepCount() int { + return it.meta.Bounds.Steps() +} + +func (it *encodedStepIterWithCollector) SeriesMeta() []block.SeriesMeta { + return it.seriesMeta +} + +func (it *encodedStepIterWithCollector) Meta() block.Metadata { + return it.meta +} + +func (it *encodedStepIterWithCollector) Err() error { + return it.err +} + +func (it *encodedStepIterWithCollector) Close() { + // noop, as the resources at the step may still be in use; + // instead call Close() on the encodedBlock that generated this } diff --git a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go index bd2238f341..2ce8f534e9 100644 --- a/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go +++ b/src/query/ts/m3db/encoded_unconsolidated_step_iterator.go @@ -21,20 +21,14 @@ package m3db import ( - "time" - "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/ts/m3db/consolidators" ) type encodedStepIterUnconsolidated struct { - err error - stepTime time.Time - meta block.Metadata - seriesMeta []block.SeriesMeta - accumulator *consolidators.StepLookbackAccumulator - collectorIterator *encodedStepIterWithCollector + accumulator *consolidators.StepLookbackAccumulator + encodedStepIterWithCollector } func (b *encodedBlockUnconsolidated) StepIter() ( @@ -42,6 +36,7 @@ func (b *encodedBlockUnconsolidated) StepIter() ( error, ) { cs := b.consolidation + iters := b.seriesBlockIterators accumulator := consolidators.NewStepLookbackAccumulator( b.lookback, cs.bounds.StepSize, @@ -49,15 +44,19 @@ func (b *encodedBlockUnconsolidated) StepIter() ( len(b.seriesBlockIterators), ) - collectorIterator := newEncodedStepIterWithCollector( - b.lastBlock, accumulator, b.seriesBlockIterators) - return &encodedStepIterUnconsolidated{ - stepTime: cs.currentTime, - meta: b.meta, - seriesMeta: b.seriesMetas, - accumulator: accumulator, - collectorIterator: collectorIterator, + accumulator: accumulator, + encodedStepIterWithCollector: encodedStepIterWithCollector{ + lastBlock: b.lastBlock, + + stepTime: cs.currentTime, + meta: b.meta, + seriesMeta: b.seriesMetas, + + collector: accumulator, + seriesPeek: make([]peekValue, len(iters)), + seriesIters: iters, + }, }, nil } @@ -65,48 +64,3 @@ func (it *encodedStepIterUnconsolidated) Current() block.UnconsolidatedStep { points := it.accumulator.AccumulateAndMoveToNext() return storage.NewUnconsolidatedStep(it.stepTime, points) } - -func (it *encodedStepIterUnconsolidated) Next() bool { - if it.err != nil { - return false - } - - bounds := it.meta.Bounds - if bounds.End().Before(it.stepTime) { - return false - } - - it.collectorIterator.nextAtTime(it.stepTime) - it.stepTime = it.stepTime.Add(bounds.StepSize) - if it.err = it.collectorIterator.err; it.err != nil { - return false - } - - return !bounds.End().Before(it.stepTime) -} - -func (it *encodedStepIterUnconsolidated) StepCount() int { - // Returns the projected step count, post-consolidation - return it.meta.Bounds.Steps() -} - -func (it *encodedStepIterUnconsolidated) Err() error { - if it.err != nil { - return it.err - } - - return it.collectorIterator.err -} - -func (it *encodedStepIterUnconsolidated) SeriesMeta() []block.SeriesMeta { - return it.seriesMeta -} - -func (it *encodedStepIterUnconsolidated) Meta() block.Metadata { - return it.meta -} - -func (it *encodedStepIterUnconsolidated) Close() { - // noop, as the resources at the step may still be in use; - // instead call Close() on the encodedBlock that generated this -} From ff21f56bbb6fd972bc12ed11dc4b6b20f4f7e407 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Thu, 7 Feb 2019 10:54:47 -0500 Subject: [PATCH 16/17] Metalint fix --- .../ts/m3db/encoded_step_iterator_generic.go | 20 ------------------- 1 file changed, 20 deletions(-) diff --git a/src/query/ts/m3db/encoded_step_iterator_generic.go b/src/query/ts/m3db/encoded_step_iterator_generic.go index 7a0e48c6d2..c0bc0c75c7 100644 --- a/src/query/ts/m3db/encoded_step_iterator_generic.go +++ b/src/query/ts/m3db/encoded_step_iterator_generic.go @@ -41,20 +41,6 @@ type encodedStepIterWithCollector struct { seriesIters []encoding.SeriesIterator } -func newEncodedStepIterWithCollector( - lastBlock bool, - collector consolidators.StepCollector, - seriesIters []encoding.SeriesIterator, -) *encodedStepIterWithCollector { - return &encodedStepIterWithCollector{ - lastBlock: lastBlock, - collector: collector, - seriesIters: seriesIters, - // TODO: pool - seriesPeek: make([]peekValue, len(seriesIters)), - } -} - // Moves to the next step for the i-th series in the block, populating // the collector for that step. Will keep reading values until either // hitting the next step boundary and returning, or until encountering @@ -119,12 +105,6 @@ func (it *encodedStepIterWithCollector) nextForStep( } } -func (it *encodedStepIterWithCollector) nextAtTime(time time.Time) { - for i := range it.seriesIters { - it.nextForStep(i, it.stepTime) - } -} - func (it *encodedStepIterWithCollector) Next() bool { if it.err != nil { return false From c93e14c2dc97da55e2ddd725663e1021807c1b01 Mon Sep 17 00:00:00 2001 From: Artem Nikolayevsky Date: Thu, 7 Feb 2019 17:35:42 -0500 Subject: [PATCH 17/17] PR responses --- src/query/storage/unconsolidated_test.go | 2 +- src/query/ts/m3db/convert.go | 31 +++++++++++++----------- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/query/storage/unconsolidated_test.go b/src/query/storage/unconsolidated_test.go index 16a4f8da79..658d6a9311 100644 --- a/src/query/storage/unconsolidated_test.go +++ b/src/query/storage/unconsolidated_test.go @@ -1,4 +1,4 @@ -// Copyright (c) 2018 Uber Technologies, Inc. +// Copyright (c) 2019 Uber Technologies, Inc. // // Permission is hereby granted, free of charge, to any person obtaining a copy // of this software and associated documentation files (the "Software"), to deal diff --git a/src/query/ts/m3db/convert.go b/src/query/ts/m3db/convert.go index 028f81cb61..ee7dd79877 100644 --- a/src/query/ts/m3db/convert.go +++ b/src/query/ts/m3db/convert.go @@ -36,6 +36,8 @@ import ( const ( initBlockReplicaLength = 10 + // This outputs time as 11:12:03AM + blockTimeFormat = "3:04:05PM" ) // blockReplica contains the replicas for a single m3db block. @@ -52,8 +54,8 @@ type seriesBlocks []seriesBlock func (b seriesBlock) String() string { return fmt.Sprint("BlockSize:", b.blockSize.Hours(), " blockStart:", - b.blockStart.Format("3:04:05PM"), " readStart:", b.readStart.Format("3:04:05PM"), - " num replicas", len(b.replicas)) + b.blockStart.Format(blockTimeFormat), " readStart:", + b.readStart.Format(blockTimeFormat), " num replicas", len(b.replicas)) } func (b seriesBlocks) Len() int { @@ -209,6 +211,17 @@ func blockDuration(blockSize, stepSize time.Duration) time.Duration { return stepSize * time.Duration(numSteps) } +// calculates duration required to fill the gap of fillSize in stepSize sized +// increments. +func calculateFillDuration(fillSize, stepSize time.Duration) time.Duration { + numberToFill := int(fillSize / stepSize) + if fillSize%stepSize != 0 { + numberToFill++ + } + + return stepSize * time.Duration(numberToFill) +} + // pads series blocks. func updateSeriesBlockStarts( blocks seriesBlocks, @@ -222,24 +235,14 @@ func updateSeriesBlockStarts( firstStart := blocks[0].blockStart if iterStart.Before(firstStart) { fillSize := firstStart.Sub(iterStart) - numberToFill := int(fillSize / stepSize) - if fillSize%stepSize != 0 { - numberToFill++ - } - - iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) + iterStart = iterStart.Add(calculateFillDuration(fillSize, stepSize)) } // Update read starts for existing blocks. for i, bl := range blocks { blocks[i].readStart = iterStart fillSize := bl.blockStart.Add(bl.blockSize).Sub(iterStart) - numberToFill := int(fillSize / stepSize) - if fillSize%stepSize != 0 { - numberToFill++ - } - - iterStart = iterStart.Add(stepSize * time.Duration(numberToFill)) + iterStart = iterStart.Add(calculateFillDuration(fillSize, stepSize)) } return blocks