diff --git a/CHANGES.md b/CHANGES.md index 38415e5f1..85b36cba8 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -9,6 +9,7 @@ Release Notes. - Add the `bydbctl analyze series` command to analyze the series data. - Index: Remove sortable field from the stored field. If a field is sortable only, it won't be stored. - Index: Support InsertIfAbsent functionality which ensures documents are only inserted if their docIDs are not already present in the current index. There is a exception for the documents with extra index fields more than the entity's index fields. +- Measure: Introduce "index_mode" to save data exclusively in the series index, ideal for non-timeseries measures. ### Bug Fixes diff --git a/api/proto/banyandb/database/v1/schema.proto b/api/proto/banyandb/database/v1/schema.proto index db7f2d503..75876e3ab 100644 --- a/api/proto/banyandb/database/v1/schema.proto +++ b/api/proto/banyandb/database/v1/schema.proto @@ -115,6 +115,9 @@ message Measure { string interval = 5; // updated_at indicates when the measure is updated google.protobuf.Timestamp updated_at = 6; + // index_mode specifies whether the data should be stored exclusively in the index, + // meaning it will not be stored in the data storage system. + bool index_mode = 7; } message MeasureAggregateFunction { diff --git a/api/validate/validate.go b/api/validate/validate.go index 1c4481aa2..9c404762a 100644 --- a/api/validate/validate.go +++ b/api/validate/validate.go @@ -132,6 +132,9 @@ func Measure(measure *databasev1.Measure) error { if len(measure.TagFamilies) == 0 { return errors.New("measure tag families is empty") } + if measure.IndexMode && len(measure.Fields) > 0 { + return errors.New("index mode is enabled, but fields are not empty") + } return tagFamily(measure.TagFamilies) } diff --git a/banyand/internal/storage/index.go b/banyand/internal/storage/index.go index ef9639838..e25ad7aec 100644 --- a/banyand/internal/storage/index.go +++ b/banyand/internal/storage/index.go @@ -31,6 +31,7 @@ import ( "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" "github.com/apache/skywalking-banyandb/pkg/query" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) func (s *segment[T, O]) IndexDB() IndexDB { @@ -38,7 +39,7 @@ func (s *segment[T, O]) IndexDB() IndexDB { } func (s *segment[T, O]) Lookup(ctx context.Context, series []*pbv1.Series) (pbv1.SeriesList, error) { - sl, _, err := s.index.filter(ctx, series, nil, nil) + sl, _, _, err := s.index.filter(ctx, series, nil, nil, nil) return sl, err } @@ -76,21 +77,19 @@ func (s *seriesIndex) Write(docs index.Documents) error { }) } -var rangeOpts = index.RangeOpts{} - func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, - projection []index.FieldKey, secondaryQuery index.Query, -) (sl pbv1.SeriesList, fields FieldResultList, err error) { + projection []index.FieldKey, secondaryQuery index.Query, timeRange *timestamp.TimeRange, +) (sl pbv1.SeriesList, fields FieldResultList, tss []int64, err error) { seriesMatchers := make([]index.SeriesMatcher, len(series)) for i := range series { seriesMatchers[i], err = convertEntityValuesToSeriesMatcher(series[i]) if err != nil { - return nil, nil, err + return nil, nil, nil, err } } - indexQuery, err := s.store.BuildQuery(seriesMatchers, secondaryQuery) + indexQuery, err := s.store.BuildQuery(seriesMatchers, secondaryQuery, timeRange) if err != nil { - return nil, nil, err + return nil, nil, nil, err } tracer := query.GetTracer(ctx) if tracer != nil { @@ -109,13 +108,13 @@ func (s *seriesIndex) filter(ctx context.Context, series []*pbv1.Series, } ss, err := s.store.Search(ctx, projection, indexQuery) if err != nil { - return nil, nil, err + return nil, nil, nil, err } - sl, fields, err = convertIndexSeriesToSeriesList(ss, len(projection) > 0) + sl, fields, tss, err = convertIndexSeriesToSeriesList(ss, len(projection) > 0) if err != nil { - return nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(ss)) + return nil, nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(ss)) } - return sl, fields, nil + return sl, fields, tss, nil } var emptySeriesMatcher = index.SeriesMatcher{} @@ -173,31 +172,38 @@ func convertEntityValuesToSeriesMatcher(series *pbv1.Series) (index.SeriesMatche }, nil } -func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasFields bool) (pbv1.SeriesList, FieldResultList, error) { +func convertIndexSeriesToSeriesList(indexSeries []index.SeriesDocument, hasFields bool) (pbv1.SeriesList, FieldResultList, []int64, error) { seriesList := make(pbv1.SeriesList, 0, len(indexSeries)) var fields FieldResultList if hasFields { fields = make(FieldResultList, 0, len(indexSeries)) } + var timestamps []int64 for _, s := range indexSeries { var series pbv1.Series - series.ID = s.Key.ID if err := series.Unmarshal(s.Key.EntityValues); err != nil { - return nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", s.Key.EntityValues) + return nil, nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", s.Key.EntityValues) } seriesList = append(seriesList, &series) if fields != nil { fields = append(fields, s.Fields) } + if s.Timestamp > 0 { + timestamps = append(timestamps, s.Timestamp) + } } - return seriesList, fields, nil + return seriesList, fields, timestamps, nil } -func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (sl pbv1.SeriesList, frl FieldResultList, err error) { +func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts, +) (sl pbv1.SeriesList, frl FieldResultList, tss []int64, sortedValues [][]byte, err error) { tracer := query.GetTracer(ctx) if tracer != nil { var span *query.Span span, ctx = tracer.StartSpan(ctx, "seriesIndex.Search") + if opts.Query != nil { + span.Tagf("secondary_query", "%s", opts.Query.String()) + } defer func() { if err != nil { span.Error(err) @@ -207,21 +213,11 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In } if opts.Order == nil || opts.Order.Index == nil { - var seriesList pbv1.SeriesList - var fieldResultList FieldResultList - if opts.Query != nil { - seriesList, fieldResultList, err = s.filter(ctx, series, opts.Projection, opts.Query) - } else { - seriesList, fieldResultList, err = s.filter(ctx, series, opts.Projection, nil) - } + sl, frl, tss, err = s.filter(ctx, series, opts.Projection, opts.Query, opts.TimeRange) if err != nil { - return nil, nil, err + return nil, nil, nil, nil, err } - return seriesList, fieldResultList, nil - } - - fieldKey := index.FieldKey{ - IndexRuleID: opts.Order.Index.GetMetadata().Id, + return sl, frl, tss, nil, nil } var span *query.Span if tracer != nil { @@ -238,43 +234,43 @@ func (s *seriesIndex) Search(ctx context.Context, series []*pbv1.Series, opts In for i := range series { seriesMatchers[i], err = convertEntityValuesToSeriesMatcher(series[i]) if err != nil { - return nil, nil, err + return nil, nil, nil, nil, err } } - query, err := s.store.BuildQuery(seriesMatchers, opts.Query) + query, err := s.store.BuildQuery(seriesMatchers, opts.Query, opts.TimeRange) if err != nil { - return nil, nil, err + return nil, nil, nil, nil, err } - iter, err := s.store.SeriesSort(ctx, fieldKey, rangeOpts, - opts.Order.Sort, opts.PreloadSize, query, opts.Projection) + iter, err := s.store.SeriesSort(ctx, query, opts.Order, + opts.PreloadSize, opts.Projection) if err != nil { - return nil, nil, err + return nil, nil, nil, nil, err } defer func() { err = multierr.Append(err, iter.Close()) }() var r int - result := make([]index.SeriesDocument, 0, 10) for iter.Next() { r++ val := iter.Val() - var doc index.SeriesDocument - doc.Fields = maps.Clone(val.Values) - doc.Key.ID = common.SeriesID(val.DocID) - doc.Key.EntityValues = val.EntityValues - result = append(result, doc) - } - sortedSeriesList, sortedFieldResultList, err := convertIndexSeriesToSeriesList(result, len(opts.Projection) > 0) - if err != nil { - return nil, nil, errors.WithMessagef(err, "failed to convert index series to series list, matchers: %v, matched: %d", seriesMatchers, len(result)) + var series pbv1.Series + if err = series.Unmarshal(val.EntityValues); err != nil { + return nil, nil, nil, nil, errors.WithMessagef(err, "failed to unmarshal series: %s", val.EntityValues) + } + sl = append(sl, &series) + tss = append(tss, val.Timestamp) + if len(opts.Projection) > 0 { + frl = append(frl, maps.Clone(val.Values)) + } + sortedValues = append(sortedValues, val.SortedValue) } if span != nil { span.Tagf("query", "%s", iter.Query().String()) span.Tagf("rounds", "%d", r) - span.Tagf("size", "%d", len(sortedSeriesList)) + span.Tagf("size", "%d", len(sl)) } - return sortedSeriesList, sortedFieldResultList, err + return sl, frl, tss, sortedValues, err } func (s *seriesIndex) Close() error { diff --git a/banyand/internal/storage/index_test.go b/banyand/internal/storage/index_test.go index 304bcff9d..3fd39aed4 100644 --- a/banyand/internal/storage/index_test.go +++ b/banyand/internal/storage/index_test.go @@ -25,6 +25,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "github.com/apache/skywalking-banyandb/api/common" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" @@ -157,11 +158,12 @@ func TestSeriesIndex_Primary(t *testing.T) { seriesQuery.EntityValues = tt.entityValues[i] seriesQueries = append(seriesQueries, &seriesQuery) } - sl, _, err := si.filter(ctx, seriesQueries, nil, nil) + sl, _, _, err := si.filter(ctx, seriesQueries, nil, nil, nil) require.NoError(t, err) require.Equal(t, len(tt.entityValues), len(sl)) assert.Equal(t, tt.subject, sl[0].Subject) for i := range tt.expected { + assert.Greater(t, sl[i].ID, common.SeriesID(0)) assert.Equal(t, tt.expected[i][0].GetStr().GetValue(), sl[i].EntityValues[0].GetStr().GetValue()) assert.Equal(t, tt.expected[i][1].GetStr().GetValue(), sl[i].EntityValues[1].GetStr().GetValue()) assert.True(t, sl[0].ID > 0) diff --git a/banyand/internal/storage/segment.go b/banyand/internal/storage/segment.go index 0f7d816b9..02f595814 100644 --- a/banyand/internal/storage/segment.go +++ b/banyand/internal/storage/segment.go @@ -234,6 +234,9 @@ func (sc *segmentController[T, O]) selectSegments(timeRange timestamp.TimeRange) last := len(sc.lst) - 1 for i := range sc.lst { s := sc.lst[last-i] + if s.GetTimeRange().End.Before(timeRange.Start) { + break + } if s.Overlapping(timeRange) { s.incRef() tt = append(tt, s) diff --git a/banyand/internal/storage/storage.go b/banyand/internal/storage/storage.go index 4bcc43d8f..aed7a40cd 100644 --- a/banyand/internal/storage/storage.go +++ b/banyand/internal/storage/storage.go @@ -36,7 +36,6 @@ import ( "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" - "github.com/apache/skywalking-banyandb/pkg/query/model" "github.com/apache/skywalking-banyandb/pkg/timestamp" ) @@ -67,7 +66,8 @@ type SupplyTSDB[T TSTable] func() T // IndexSearchOpts is the options for searching index. type IndexSearchOpts struct { Query index.Query - Order *model.OrderBy + Order *index.OrderBy + TimeRange *timestamp.TimeRange Projection []index.FieldKey PreloadSize int } @@ -81,7 +81,7 @@ type FieldResultList []FieldResult // IndexDB is the interface of index database. type IndexDB interface { Write(docs index.Documents) error - Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, error) + Search(ctx context.Context, series []*pbv1.Series, opts IndexSearchOpts) (pbv1.SeriesList, FieldResultList, []int64, [][]byte, error) } // TSDB allows listing and getting shard details. diff --git a/banyand/measure/measure.go b/banyand/measure/measure.go index f278735e4..40a108982 100644 --- a/banyand/measure/measure.go +++ b/banyand/measure/measure.go @@ -103,7 +103,7 @@ func (s *measure) parseSpec() (err error) { if s.schema.Interval != "" { s.interval, err = timestamp.ParseDuration(s.schema.Interval) } - s.indexRuleLocators, s.fieldIndexLocation = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules) + s.indexRuleLocators, s.fieldIndexLocation = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules, s.schema.IndexMode) return err } diff --git a/banyand/measure/query.go b/banyand/measure/query.go index 299402fe1..e0d074944 100644 --- a/banyand/measure/query.go +++ b/banyand/measure/query.go @@ -18,6 +18,7 @@ package measure import ( + "bytes" "container/heap" "context" "fmt" @@ -86,29 +87,39 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr EntityValues: mqo.Entities[i], } } - var result queryResult - result.ctx = ctx + tsdb := db.(storage.TSDB[*tsTable, option]) - result.segments = tsdb.SelectSegments(*mqo.TimeRange) - if len(result.segments) < 1 { - return &result, nil + segments := tsdb.SelectSegments(*mqo.TimeRange) + if len(segments) < 1 { + return nil, nil } - defer func() { - if err != nil { - result.Release() - } - }() - sids, tables, storedIndexValue, newTagProjection, err := s.searchSeriesList(ctx, series, mqo, result.segments) + if s.schema.IndexMode { + return s.buildIndexQueryResult(ctx, series, mqo, segments) + } + + sids, tables, storedIndexValue, newTagProjection, err := s.searchSeriesList(ctx, series, mqo, segments) if err != nil { return nil, err } if len(sids) < 1 { - return &result, nil + for i := range segments { + segments[i].DecRef() + } + return nil, nil } - result.tagProjection = mqo.TagProjection + result := queryResult{ + ctx: ctx, + segments: segments, + tagProjection: mqo.TagProjection, + storedIndexValue: storedIndexValue, + } + defer func() { + if err != nil { + result.Release() + } + }() mqo.TagProjection = newTagProjection - result.storedIndexValue = storedIndexValue var parts []*part qo := queryOptions{ MeasureQueryOptions: mqo, @@ -135,23 +146,27 @@ func (s *measure) Query(ctx context.Context, mqo model.MeasureQueryOptions) (mqr if mqo.Order == nil { result.ascTS = true - } else if mqo.Order.Sort == modelv1.Sort_SORT_ASC || mqo.Order.Sort == modelv1.Sort_SORT_UNSPECIFIED { - result.ascTS = true - } - switch mqo.OrderByType { - case model.OrderByTypeTime: result.orderByTS = true - case model.OrderByTypeIndex: - result.orderByTS = false - case model.OrderByTypeSeries: - result.orderByTS = false + } else { + if mqo.Order.Sort == modelv1.Sort_SORT_ASC || mqo.Order.Sort == modelv1.Sort_SORT_UNSPECIFIED { + result.ascTS = true + } + switch mqo.Order.Type { + case index.OrderByTypeTime: + result.orderByTS = true + case index.OrderByTypeIndex: + result.orderByTS = false + case index.OrderByTypeSeries: + result.orderByTS = false + } } + return &result, nil } type tagNameWithType struct { - name string - typ pbv1.ValueType + fieldName string + typ pbv1.ValueType } func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, mqo model.MeasureQueryOptions, @@ -180,8 +195,8 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m if field, ok := fields[n]; ok { indexProjection = append(indexProjection, field.Key) fieldToValueType[field.Key.Marshal()] = tagNameWithType{ - name: n, - typ: field.Type, + fieldName: n, + typ: field.Type, } continue TAG } @@ -195,7 +210,7 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m } seriesFilter := roaring.NewPostingList() for i := range segments { - sll, fieldResultList, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ + sll, fieldResultList, _, _, err := segments[i].IndexDB().Search(ctx, series, storage.IndexSearchOpts{ Query: mqo.Query, Order: mqo.Order, PreloadSize: preloadSize, @@ -229,7 +244,7 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m } for f, v := range fieldResultList[j] { if tnt, ok := fieldToValueType[f]; ok { - tagValues[tnt.name] = mustDecodeTagValue(tnt.typ, v) + tagValues[tnt.fieldName] = mustDecodeTagValue(tnt.typ, v) } else { logger.Panicf("unknown field %s not found in fieldToValueType", f) } @@ -239,6 +254,70 @@ func (s *measure) searchSeriesList(ctx context.Context, series []*pbv1.Series, m return sl, tables, storedIndexValue, newTagProjection, nil } +func (s *measure) buildIndexQueryResult(ctx context.Context, series []*pbv1.Series, mqo model.MeasureQueryOptions, + segments []storage.Segment[*tsTable, option], +) (*indexSortResult, error) { + defer func() { + for i := range segments { + segments[i].DecRef() + } + }() + r := &indexSortResult{} + var indexProjection []index.FieldKey + for _, tp := range mqo.TagProjection { + tagFamilyLocation := tagFamilyLocation{ + name: tp.Family, + fieldToValueType: make(map[string]tagNameWithType), + projectedEntityOffsets: make(map[string]int), + } + TAG: + for _, n := range tp.Names { + tagFamilyLocation.tagNames = append(tagFamilyLocation.tagNames, n) + for i := range s.schema.GetEntity().GetTagNames() { + if n == s.schema.GetEntity().GetTagNames()[i] { + tagFamilyLocation.projectedEntityOffsets[n] = i + continue TAG + } + } + if fields, ok := s.fieldIndexLocation[tp.Family]; ok { + if field, ok := fields[n]; ok { + indexProjection = append(indexProjection, field.Key) + tagFamilyLocation.fieldToValueType[n] = tagNameWithType{ + fieldName: field.Key.Marshal(), + typ: field.Type, + } + continue TAG + } + } + return nil, fmt.Errorf("tag %s not found in schema", n) + } + r.tfl = append(r.tfl, tagFamilyLocation) + } + var err error + opts := storage.IndexSearchOpts{ + Query: mqo.Query, + Order: mqo.Order, + PreloadSize: preloadSize, + Projection: indexProjection, + } + + for i := range segments { + if mqo.TimeRange.Include(segments[i].GetTimeRange()) { + opts.TimeRange = nil + } else { + opts.TimeRange = mqo.TimeRange + } + sr := &segResult{} + sr.sll, sr.frl, sr.timestamps, sr.sortedValues, err = segments[i].IndexDB().Search(ctx, series, opts) + if err != nil { + return nil, err + } + r.segResults = append(r.segResults, sr) + } + heap.Init(&r.segResults) + return r, nil +} + func (s *measure) searchBlocks(ctx context.Context, result *queryResult, sids []common.SeriesID, parts []*part, qo queryOptions) error { bma := generateBlockMetadataArray() defer releaseBlockMetadataArray(bma) @@ -634,3 +713,113 @@ func (qr *queryResult) merge(storedIndexValue map[common.SeriesID]map[string]*mo return result } + +var bypassVersions = []int64{1} + +type indexSortResult struct { + tfl []tagFamilyLocation + segResults segResultHeap +} + +// Pull implements model.MeasureQueryResult. +func (iqr *indexSortResult) Pull() *model.MeasureResult { + if len(iqr.segResults) < 1 { + return nil + } + if len(iqr.segResults) == 1 { + if iqr.segResults[0].i >= len(iqr.segResults[0].sll) { + return nil + } + sr := iqr.segResults[0] + r := iqr.copyTo(sr) + sr.i++ + if sr.i >= len(sr.sll) { + iqr.segResults = iqr.segResults[:0] + } + return r + } + top := heap.Pop(&iqr.segResults) + sr := top.(*segResult) + r := iqr.copyTo(sr) + sr.i++ + if sr.i < len(sr.sll) { + heap.Push(&iqr.segResults, sr) + } + return r +} + +func (iqr *indexSortResult) Release() {} + +func (iqr *indexSortResult) copyTo(src *segResult) (dest *model.MeasureResult) { + index := src.i + dest = &model.MeasureResult{ + SID: src.sll[index].ID, + Timestamps: []int64{src.timestamps[index]}, + Versions: bypassVersions, + } + for i := range iqr.tfl { + tagFamily := model.TagFamily{Name: iqr.tfl[i].name} + peo := iqr.tfl[i].projectedEntityOffsets + var fr storage.FieldResult + if src.frl != nil { + fr = src.frl[index] + } + for _, n := range iqr.tfl[i].tagNames { + if offset, ok := peo[n]; ok { + tagFamily.Tags = append(tagFamily.Tags, model.Tag{ + Name: n, + Values: []*modelv1.TagValue{src.sll[index].EntityValues[offset]}, + }) + continue + } + if fr == nil { + continue + } + if tnt, ok := iqr.tfl[i].fieldToValueType[n]; ok { + tagFamily.Tags = append(tagFamily.Tags, model.Tag{ + Name: n, + Values: []*modelv1.TagValue{mustDecodeTagValue(tnt.typ, fr[tnt.fieldName])}, + }) + } else { + logger.Panicf("unknown field %s not found in fieldToValueType", n) + } + } + dest.TagFamilies = append(dest.TagFamilies, tagFamily) + } + return dest +} + +type tagFamilyLocation struct { + fieldToValueType map[string]tagNameWithType + projectedEntityOffsets map[string]int + name string + tagNames []string +} + +type segResult struct { + sll pbv1.SeriesList + frl storage.FieldResultList + timestamps []int64 + sortedValues [][]byte + i int +} + +type segResultHeap []*segResult + +func (h segResultHeap) Len() int { return len(h) } +func (h segResultHeap) Less(i, j int) bool { + return bytes.Compare(h[i].sortedValues[h[i].i], h[j].sortedValues[h[j].i]) < 0 +} +func (h segResultHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *segResultHeap) Push(x interface{}) { + *h = append(*h, x.(*segResult)) +} + +func (h *segResultHeap) Pop() interface{} { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} diff --git a/banyand/measure/write.go b/banyand/measure/write.go index bb6165d38..ab0879a22 100644 --- a/banyand/measure/write.go +++ b/banyand/measure/write.go @@ -125,6 +125,61 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me )) } dpt.dataPoints.fields = append(dpt.dataPoints.fields, field) + tagFamily, fields := w.handleTagFamily(stm, req) + dpt.dataPoints.tagFamilies = append(dpt.dataPoints.tagFamilies, tagFamily) + + if stm.processorManager != nil { + stm.processorManager.onMeasureWrite(uint64(series.ID), &measurev1.InternalWriteRequest{ + Request: &measurev1.WriteRequest{ + Metadata: stm.GetSchema().Metadata, + DataPoint: req.DataPoint, + MessageId: uint64(time.Now().UnixNano()), + }, + EntityValues: writeEvent.EntityValues, + }) + } + + doc := index.Document{ + DocID: uint64(series.ID), + EntityValues: series.Buffer, + Fields: fields, + } + if stm.schema.IndexMode { + doc.Timestamp = ts + } + dpg.docs = append(dpg.docs, doc) + + return dst, nil +} + +func (w *writeCallback) newDpt(tsdb storage.TSDB[*tsTable, option], dpg *dataPointsInGroup, t time.Time, ts int64, shardID common.ShardID) (*dataPointsInTable, error) { + var segment storage.Segment[*tsTable, option] + for _, seg := range dpg.segments { + if seg.GetTimeRange().Contains(ts) { + segment = seg + } + } + if segment == nil { + var err error + segment, err = tsdb.CreateSegmentIfNotExist(t) + if err != nil { + return nil, fmt.Errorf("cannot create segment: %w", err) + } + dpg.segments = append(dpg.segments, segment) + } + tstb, err := segment.CreateTSTableIfNotExist(shardID) + if err != nil { + return nil, fmt.Errorf("cannot create ts table: %w", err) + } + dpt := &dataPointsInTable{ + timeRange: segment.GetTimeRange(), + tsTable: tstb, + } + dpg.tables = append(dpg.tables, dpt) + return dpt, nil +} + +func (w *writeCallback) handleTagFamily(stm *measure, req *measurev1.WriteRequest) ([]nameValues, []index.Field) { tagFamilies := make([]nameValues, 0, len(stm.schema.TagFamilies)) if len(stm.indexRuleLocators.TagFamilyTRule) != len(stm.GetSchema().GetTagFamilies()) { logger.Panicf("metadata crashed, tag family rule length %d, tag family length %d", @@ -156,26 +211,34 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me t.Name, t.Type, tagValue) - if r, ok := tfr[t.Name]; ok { + r, ok := tfr[t.Name] + if ok || stm.schema.IndexMode { + fieldKey := index.FieldKey{} + switch { + case ok: + fieldKey.IndexRuleID = r.GetMetadata().GetId() + fieldKey.Analyzer = r.Analyzer + case stm.schema.IndexMode: + fieldKey.TagName = t.Name + default: + logger.Panicf("metadata crashed, tag family rule %s not found", t.Name) + } + toIndex := ok || !stm.schema.IndexMode if encodeTagValue.value != nil { fields = append(fields, index.Field{ - Key: index.FieldKey{ - IndexRuleID: r.GetMetadata().GetId(), - Analyzer: r.Analyzer, - }, + Key: fieldKey, Term: encodeTagValue.value, Store: true, + Index: toIndex, NoSort: r.GetNoSort(), }) } else { for _, val := range encodeTagValue.valueArr { fields = append(fields, index.Field{ - Key: index.FieldKey{ - IndexRuleID: r.GetMetadata().GetId(), - Analyzer: r.Analyzer, - }, + Key: fieldKey, Term: val, Store: true, + Index: toIndex, NoSort: r.GetNoSort(), }) } @@ -192,52 +255,7 @@ func (w *writeCallback) handle(dst map[string]*dataPointsInGroup, writeEvent *me tagFamilies = append(tagFamilies, tf) } } - dpt.dataPoints.tagFamilies = append(dpt.dataPoints.tagFamilies, tagFamilies) - - if stm.processorManager != nil { - stm.processorManager.onMeasureWrite(uint64(series.ID), &measurev1.InternalWriteRequest{ - Request: &measurev1.WriteRequest{ - Metadata: stm.GetSchema().Metadata, - DataPoint: req.DataPoint, - MessageId: uint64(time.Now().UnixNano()), - }, - EntityValues: writeEvent.EntityValues, - }) - } - - dpg.docs = append(dpg.docs, index.Document{ - DocID: uint64(series.ID), - EntityValues: series.Buffer, - Fields: fields, - }) - return dst, nil -} - -func (w *writeCallback) newDpt(tsdb storage.TSDB[*tsTable, option], dpg *dataPointsInGroup, t time.Time, ts int64, shardID common.ShardID) (*dataPointsInTable, error) { - var segment storage.Segment[*tsTable, option] - for _, seg := range dpg.segments { - if seg.GetTimeRange().Contains(ts) { - segment = seg - } - } - if segment == nil { - var err error - segment, err = tsdb.CreateSegmentIfNotExist(t) - if err != nil { - return nil, fmt.Errorf("cannot create segment: %w", err) - } - dpg.segments = append(dpg.segments, segment) - } - tstb, err := segment.CreateTSTableIfNotExist(shardID) - if err != nil { - return nil, fmt.Errorf("cannot create ts table: %w", err) - } - dpt := &dataPointsInTable{ - timeRange: segment.GetTimeRange(), - tsTable: tstb, - } - dpg.tables = append(dpg.tables, dpt) - return dpt, nil + return tagFamilies, fields } func (w *writeCallback) Rev(_ context.Context, message bus.Message) (resp bus.Message) { diff --git a/banyand/stream/benchmark_test.go b/banyand/stream/benchmark_test.go index 0bb418e50..5aa0e10e9 100644 --- a/banyand/stream/benchmark_test.go +++ b/banyand/stream/benchmark_test.go @@ -264,7 +264,7 @@ func generateStream(db storage.TSDB[*tsTable, option]) *stream { } } -func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryOptions { +func generateStreamQueryOptions(p parameter, midx mockIndex) model.StreamQueryOptions { timeRange := timestamp.TimeRange{ Start: time.Unix(int64(p.startTimestamp), 0), End: time.Unix(int64(p.endTimestamp), 0), @@ -287,7 +287,7 @@ func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryO num := generateRandomNumber(int64(p.tagCardinality)) value := filterTagValuePrefix + strconv.Itoa(num) filter := mockFilter{ - index: index, + index: midx, value: value, } indexRule := &databasev1.IndexRule{ @@ -297,7 +297,7 @@ func generateStreamQueryOptions(p parameter, index mockIndex) model.StreamQueryO Tags: []string{"filter-tag"}, Type: databasev1.IndexRule_TYPE_INVERTED, } - order := &model.OrderBy{ + order := &index.OrderBy{ Index: indexRule, Sort: modelv1.Sort_SORT_ASC, } diff --git a/banyand/stream/stream.go b/banyand/stream/stream.go index 60596f57b..1243a38e4 100644 --- a/banyand/stream/stream.go +++ b/banyand/stream/stream.go @@ -89,7 +89,7 @@ func (s *stream) Close() error { func (s *stream) parseSpec() { s.name, s.group = s.schema.GetMetadata().GetName(), s.schema.GetMetadata().GetGroup() - s.indexRuleLocators, _ = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules) + s.indexRuleLocators, _ = partition.ParseIndexRuleLocators(s.schema.GetEntity(), s.schema.GetTagFamilies(), s.indexRules, false) } type streamSpec struct { diff --git a/dist/LICENSE b/dist/LICENSE index a6503207c..f90ac0ca4 100644 --- a/dist/LICENSE +++ b/dist/LICENSE @@ -326,7 +326,7 @@ MIT licenses github.com/go-ole/go-ole v1.3.0 MIT github.com/go-resty/resty/v2 v2.14.0 MIT github.com/go-task/slim-sprig/v3 v3.0.0 MIT - github.com/golang-jwt/jwt/v4 v4.5.0 MIT + github.com/golang-jwt/jwt/v4 v4.5.1 MIT github.com/json-iterator/go v1.1.12 MIT github.com/mattn/go-colorable v0.1.13 MIT github.com/mattn/go-isatty v0.0.20 MIT diff --git a/docs/api-reference.md b/docs/api-reference.md index 852c7f667..4954ce1f6 100644 --- a/docs/api-reference.md +++ b/docs/api-reference.md @@ -1076,6 +1076,7 @@ Measure intends to store data point | entity | [Entity](#banyandb-database-v1-Entity) | | entity indicates which tags will be to generate a series and shard a measure | | interval | [string](#string) | | interval indicates how frequently to send a data point valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h", "d". | | updated_at | [google.protobuf.Timestamp](#google-protobuf-Timestamp) | | updated_at indicates when the measure is updated | +| index_mode | [bool](#bool) | | index_mode specifies whether the data should be stored exclusively in the index, meaning it will not be stored in the data storage system. | diff --git a/docs/concept/data-model.md b/docs/concept/data-model.md index 06726eb3e..e81c0640b 100644 --- a/docs/concept/data-model.md +++ b/docs/concept/data-model.md @@ -76,6 +76,7 @@ fields: entity: tag_names: - entity_id +index_mode: false interval: 1m ``` @@ -115,6 +116,25 @@ functions to them. Another option named `interval` plays a critical role in encoding. It indicates the time range between two adjacent data points in a time series and implies that all data points belonging to the same time series are distributed based on a fixed interval. A better practice for the naming measure is to append the interval literal to the tail, for example, `service_cpm_minute`. It's a parameter of `GORILLA` encoding method. +`index_mode` is a flag to enable the series index as the storage engine. All the tags will be stored in the inverted index and no field is allowed in the measure. This mode is suitable for the non-time series data model but needs TTL to be set. In this mode, the tags defined in the `entity` is the unique key of the data point. `timestamp` and `version` are the common tags in the inverted index. + +There is an example of a measure with the index mode enabled: + +```yaml +metadata: + name: service_traffic + group: sw_metric +tag_families: +- name: default + tags: + - name: id + type: TAG_TYPE_STRING + - name: service_name + type: TAG_TYPE_STRING +index_mode: true +entity: ["id"] +``` + [Measure Registration Operations](../api-reference.md#measureregistryservice) #### TopNAggregation diff --git a/go.mod b/go.mod index 56646ccf0..57454f710 100644 --- a/go.mod +++ b/go.mod @@ -83,7 +83,7 @@ require ( github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.3.0 // indirect github.com/gogo/protobuf v1.3.2 // indirect - github.com/golang-jwt/jwt/v4 v4.5.0 // indirect + github.com/golang-jwt/jwt/v4 v4.5.1 // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/google/btree v1.1.3 // indirect github.com/google/pprof v0.0.0-20240910150728-a0b0bb1d4134 // indirect diff --git a/go.sum b/go.sum index 0d9c71dbc..5c140a3bf 100644 --- a/go.sum +++ b/go.sum @@ -102,8 +102,8 @@ github.com/go-task/slim-sprig/v3 v3.0.0/go.mod h1:W848ghGpv3Qj3dhTPRyJypKRiqCdHZ github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= -github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= -github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.1 h1:JdqV9zKUdtaa9gdPlywC3aeoEsR681PlKC+4F5gQgeo= +github.com/golang-jwt/jwt/v4 v4.5.1/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= diff --git a/pkg/index/index.go b/pkg/index/index.go index fdcb2c927..88485605d 100644 --- a/pkg/index/index.go +++ b/pkg/index/index.go @@ -49,12 +49,16 @@ const ( // FieldKey is the key of field in a document. type FieldKey struct { Analyzer string + TagName string SeriesID common.SeriesID IndexRuleID uint32 } // Marshal encodes f to string. func (f FieldKey) Marshal() string { + if len(f.TagName) > 0 { + return f.TagName + } return string(convert.Uint32ToBytes(f.IndexRuleID)) } @@ -64,6 +68,7 @@ type Field struct { Key FieldKey NoSort bool Store bool + Index bool } // RangeOpts contains options to performance a continuous scan. @@ -169,7 +174,7 @@ type Writer interface { // FieldIterable allows building a FieldIterator. type FieldIterable interface { - BuildQuery(seriesMatchers []SeriesMatcher, secondaryQuery Query) (Query, error) + BuildQuery(seriesMatchers []SeriesMatcher, secondaryQuery Query, timeRange *timestamp.TimeRange) (Query, error) Iterator(ctx context.Context, fieldKey FieldKey, termRange RangeOpts, order modelv1.Sort, preLoadSize int) (iter FieldIterator[*DocumentResult], err error) Sort(ctx context.Context, sids []common.SeriesID, fieldKey FieldKey, @@ -201,11 +206,10 @@ type Store interface { // Series represents a series in an index. type Series struct { EntityValues []byte - ID common.SeriesID } func (s Series) String() string { - return fmt.Sprintf("%s:%d", s.EntityValues, s.ID) + return convert.BytesToString(s.EntityValues) } // SortedField returns the value of the sorted field. @@ -215,8 +219,28 @@ func (s Series) SortedField() []byte { // SeriesDocument represents a series document in an index. type SeriesDocument struct { - Fields map[string][]byte - Key Series + Fields map[string][]byte + Key Series + Timestamp int64 +} + +// OrderByType is the type of order by. +type OrderByType int + +const ( + // OrderByTypeTime is the order by time. + OrderByTypeTime OrderByType = iota + // OrderByTypeIndex is the order by index. + OrderByTypeIndex + // OrderByTypeSeries is the order by series. + OrderByTypeSeries +) + +// OrderBy is the order by rule. +type OrderBy struct { + Index *databasev1.IndexRule + Sort modelv1.Sort + Type OrderByType } // SeriesStore is an abstract of a series repository. @@ -225,8 +249,8 @@ type SeriesStore interface { // Search returns a list of series that match the given matchers. Search(context.Context, []FieldKey, Query) ([]SeriesDocument, error) SeriesIterator(context.Context) (FieldIterator[Series], error) - SeriesSort(ctx context.Context, fieldKey FieldKey, termRange RangeOpts, order modelv1.Sort, - preLoadSize int, query Query, fieldKeys []FieldKey) (iter FieldIterator[*DocumentResult], err error) + SeriesSort(ctx context.Context, indexQuery Query, orderBy *OrderBy, + preLoadSize int, fieldKeys []FieldKey) (iter FieldIterator[*DocumentResult], err error) } // SeriesMatcherType represents the type of series matcher. diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go index 084ef7290..4772f8b89 100644 --- a/pkg/index/inverted/inverted.go +++ b/pkg/index/inverted/inverted.go @@ -48,10 +48,8 @@ import ( const ( docIDField = "_id" batchSize = 1024 - seriesIDField = "series_id" - entityField = "entity" - idField = "id" - timestampField = "timestamp" + seriesIDField = "_series_id" + timestampField = "_timestamp" ) var ( @@ -317,22 +315,24 @@ func (s *store) Range(fieldKey index.FieldKey, opts index.RangeOpts) (list posti } type blugeMatchIterator struct { - delegated search.DocumentMatchIterator - err error - closer io.Closer - ctx *search.Context - current index.DocumentResult - hit int + delegated search.DocumentMatchIterator + err error + closer io.Closer + ctx *search.Context + loadDocValues []string + current index.DocumentResult + hit int } func newBlugeMatchIterator(delegated search.DocumentMatchIterator, closer io.Closer, - _ []string, + loadDocValues []string, ) blugeIterator { bmi := &blugeMatchIterator{ - delegated: delegated, - closer: closer, - current: index.DocumentResult{}, - ctx: search.NewSearchContext(1, 0), + delegated: delegated, + closer: closer, + current: index.DocumentResult{}, + ctx: search.NewSearchContext(1, 0), + loadDocValues: loadDocValues, } return bmi } @@ -359,9 +359,23 @@ func (bmi *blugeMatchIterator) Next() bool { if len(match.SortValue) > 0 { bmi.current.SortedValue = match.SortValue[0] } - err := match.VisitStoredFields(bmi.setVal) - bmi.err = multierr.Combine(bmi.err, err) - return bmi.err == nil + if len(bmi.loadDocValues) == 0 { + err := match.VisitStoredFields(bmi.setVal) + bmi.err = multierr.Combine(bmi.err, err) + return bmi.err == nil + } + if err := match.LoadDocumentValues(bmi.ctx, bmi.loadDocValues); err != nil { + bmi.err = multierr.Combine(bmi.err, err) + return false + } + for _, dv := range bmi.loadDocValues { + vv := match.DocValues(dv) + if len(vv) == 0 { + continue + } + bmi.setVal(dv, vv[0]) + } + return true } func (bmi *blugeMatchIterator) setVal(field string, value []byte) bool { diff --git a/pkg/index/inverted/inverted_series.go b/pkg/index/inverted/inverted_series.go index 9d0b15edb..59feb2c50 100644 --- a/pkg/index/inverted/inverted_series.go +++ b/pkg/index/inverted/inverted_series.go @@ -30,10 +30,10 @@ import ( "github.com/pkg/errors" "go.uber.org/multierr" - "github.com/apache/skywalking-banyandb/api/common" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var emptySeries = make([]index.SeriesDocument, 0) @@ -49,12 +49,15 @@ func (s *store) SeriesBatch(batch index.Batch) error { b := generateBatch() defer releaseBatch(b) for _, d := range batch.Documents { - doc := bluge.NewDocument(convert.BytesToString(convert.Uint64ToBytes(d.DocID))) + doc := bluge.NewDocument(convert.BytesToString(d.EntityValues)) for _, f := range d.Fields { tf := bluge.NewKeywordFieldBytes(f.Key.Marshal(), f.Term) - if !f.NoSort { + if !f.Index { + tf.FieldOptions = 0 + } else if !f.NoSort { tf.Sortable() } + if f.Store { tf.StoreValue() } @@ -64,7 +67,6 @@ func (s *store) SeriesBatch(batch index.Batch) error { doc.AddField(tf) } - doc.AddField(bluge.NewKeywordFieldBytes(entityField, d.EntityValues).StoreValue()) if d.Timestamp > 0 { doc.AddField(bluge.NewDateTimeField(timestampField, time.Unix(0, d.Timestamp)).StoreValue()) } @@ -78,7 +80,7 @@ func (s *store) SeriesBatch(batch index.Batch) error { } // BuildQuery implements index.SeriesStore. -func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery index.Query) (index.Query, error) { +func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery index.Query, timeRange *timestamp.TimeRange) (index.Query, error) { if len(seriesMatchers) == 0 { return secondaryQuery, nil } @@ -90,19 +92,19 @@ func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery case index.SeriesMatcherTypeExact: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewTermQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newTermNode(match, nil)) case index.SeriesMatcherTypePrefix: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewPrefixQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newPrefixNode(match)) case index.SeriesMatcherTypeWildcard: match := convert.BytesToString(seriesMatchers[i].Match) q := bluge.NewWildcardQuery(match) - q.SetField(entityField) + q.SetField(docIDField) qs[i] = q nodes = append(nodes, newWildcardNode(match)) default: @@ -132,6 +134,12 @@ func (s *store) BuildQuery(seriesMatchers []index.SeriesMatcher, secondaryQuery query.AddMust(secondaryQuery.(*queryNode).query) node.Append(secondaryQuery.(*queryNode).node) } + if timeRange != nil { + q := bluge.NewDateRangeInclusiveQuery(timeRange.Start, timeRange.End, timeRange.IncludeStart, timeRange.IncludeEnd) + q.SetField(timestampField) + query.AddMust(q) + node.Append(newTimeRangeNode(timeRange)) + } return &queryNode{query, node}, nil } @@ -164,7 +172,6 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey if err != nil { return nil, errors.WithMessage(err, "iterate document match iterator") } - docIDMap := make(map[uint64]struct{}) fields := make([]string, 0, len(loadedFields)) for i := range loadedFields { fields = append(fields, loadedFields[i].Marshal()) @@ -179,16 +186,19 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey doc.Fields[fields[i]] = nil } } + var errTime error err = next.VisitStoredFields(func(field string, value []byte) bool { switch field { case docIDField: - id := convert.BytesToUint64(value) - if _, ok := docIDMap[id]; !ok { - doc.Key.ID = common.SeriesID(convert.BytesToUint64(value)) - docIDMap[id] = struct{}{} - } - case entityField: doc.Key.EntityValues = value + case timestampField: + var ts time.Time + ts, errTime = bluge.DecodeDateTime(value) + if errTime != nil { + err = errTime + return false + } + doc.Timestamp = ts.UnixNano() default: if _, ok := doc.Fields[field]; ok { doc.Fields[field] = bytes.Clone(value) @@ -196,10 +206,10 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey } return true }) - if err != nil { + if err = multierr.Combine(err, errTime); err != nil { return nil, errors.WithMessagef(err, "visit stored fields, hit: %d", hitNumber) } - if doc.Key.ID > 0 { + if len(doc.Key.EntityValues) > 0 { result = append(result, doc) } next, err = dmi.Next() @@ -210,58 +220,39 @@ func parseResult(dmi search.DocumentMatchIterator, loadedFields []index.FieldKey return result, nil } -func (s *store) SeriesSort(ctx context.Context, fieldKey index.FieldKey, termRange index.RangeOpts, order modelv1.Sort, - preLoadSize int, indexQuery index.Query, fieldKeys []index.FieldKey, +func (s *store) SeriesSort(ctx context.Context, indexQuery index.Query, orderBy *index.OrderBy, + preLoadSize int, fieldKeys []index.FieldKey, ) (iter index.FieldIterator[*index.DocumentResult], err error) { - if termRange.Lower != nil && - termRange.Upper != nil && - bytes.Compare(termRange.Lower, termRange.Upper) > 0 { - return index.DummyFieldIterator, nil + var sortedKey string + switch orderBy.Type { + case index.OrderByTypeTime: + sortedKey = timestampField + case index.OrderByTypeIndex: + fieldKey := index.FieldKey{ + IndexRuleID: orderBy.Index.Metadata.Id, + } + sortedKey = fieldKey.Marshal() + default: + return nil, errors.Errorf("unsupported order by type: %v", orderBy.Type) + } + if orderBy.Sort == modelv1.Sort_SORT_DESC { + sortedKey = "-" + sortedKey } + fields := make([]string, 0, len(fieldKeys)) + for i := range fieldKeys { + fields = append(fields, fieldKeys[i].Marshal()) + } + if !s.closer.AddRunning() { return nil, nil } - reader, err := s.writer.Reader() if err != nil { return nil, err } - fk := fieldKey.Marshal() - if termRange.Upper == nil { - termRange.Upper = defaultUpper - } - if termRange.Lower == nil { - termRange.Lower = defaultLower - } - rangeQuery := bluge.NewBooleanQuery() - rangeQuery.AddMust(bluge.NewTermRangeInclusiveQuery( - string(termRange.Lower), - string(termRange.Upper), - termRange.IncludesLower, - termRange.IncludesUpper, - ). - SetField(fk)) - rangeNode := newMustNode() - rangeNode.Append(newTermRangeInclusiveNode(string(termRange.Lower), string(termRange.Upper), termRange.IncludesLower, termRange.IncludesUpper, nil)) - - sortedKey := fk - if order == modelv1.Sort_SORT_DESC { - sortedKey = "-" + sortedKey - } - query := bluge.NewBooleanQuery().AddMust(rangeQuery) - node := newMustNode() - node.Append(rangeNode) - if indexQuery != nil && indexQuery.(*queryNode).query != nil { - query.AddMust(indexQuery.(*queryNode).query) - node.Append(indexQuery.(*queryNode).node) - } - fields := make([]string, 0, len(fieldKeys)) - for i := range fieldKeys { - fields = append(fields, fieldKeys[i].Marshal()) - } - result := &sortIterator{ - query: &queryNode{query, node}, + return &sortIterator{ + query: indexQuery, fields: fields, reader: reader, sortedKey: sortedKey, @@ -269,13 +260,11 @@ func (s *store) SeriesSort(ctx context.Context, fieldKey index.FieldKey, termRan closer: s.closer, ctx: ctx, newIterator: newSeriesIterator, - } - return result, nil + }, nil } type seriesIterator struct { *blugeMatchIterator - needToLoadFields []string } func newSeriesIterator(delegated search.DocumentMatchIterator, closer io.Closer, @@ -288,7 +277,6 @@ func newSeriesIterator(delegated search.DocumentMatchIterator, closer io.Closer, ctx: search.NewSearchContext(1, 0), current: index.DocumentResult{Values: make(map[string][]byte, len(needToLoadFields))}, }, - needToLoadFields: append(needToLoadFields, entityField, docIDField, seriesIDField, timestampField), } for _, f := range needToLoadFields { si.current.Values[f] = nil @@ -312,7 +300,6 @@ func (si *seriesIterator) Next() bool { si.current.Values[i] = nil } si.current.DocID = 0 - si.current.SeriesID = 0 si.current.Timestamp = 0 si.current.SortedValue = nil if len(match.SortValue) > 0 { @@ -329,10 +316,8 @@ func (si *seriesIterator) Next() bool { func (si *seriesIterator) setVal(field string, value []byte) bool { switch field { - case entityField: - si.current.EntityValues = value case docIDField: - si.current.DocID = convert.BytesToUint64(value) + si.current.EntityValues = value case timestampField: ts, errTime := bluge.DecodeDateTime(value) if errTime != nil { @@ -356,7 +341,7 @@ func (s *store) SeriesIterator(ctx context.Context) (index.FieldIterator[index.S defer func() { _ = reader.Close() }() - dict, err := reader.DictionaryIterator(entityField, nil, nil, nil) + dict, err := reader.DictionaryIterator(docIDField, nil, nil, nil) if err != nil { return nil, err } diff --git a/pkg/index/inverted/inverted_series_test.go b/pkg/index/inverted/inverted_series_test.go index cd0a3bd46..ac546ca9f 100644 --- a/pkg/index/inverted/inverted_series_test.go +++ b/pkg/index/inverted/inverted_series_test.go @@ -19,15 +19,22 @@ package inverted import ( "context" + "maps" "testing" + "time" + "github.com/blugelabs/bluge" + "github.com/blugelabs/bluge/numeric" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/apache/skywalking-banyandb/api/common" + commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1" + databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1" + modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var ( @@ -37,6 +44,9 @@ var ( fieldKeyServiceName = index.FieldKey{ IndexRuleID: 6, } + fieldKeyStartTime = index.FieldKey{ + IndexRuleID: 21, + } ) func TestStore_Search(t *testing.T) { @@ -66,7 +76,6 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, @@ -78,33 +87,32 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), fieldKeyServiceName.Marshal(): []byte("svc2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), + fieldKeyDuration.Marshal(): nil, fieldKeyServiceName.Marshal(): nil, }, }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): nil, + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), fieldKeyServiceName.Marshal(): nil, }, + Timestamp: int64(1001), }, }, }, @@ -114,30 +122,30 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), }, + Timestamp: int64(101), }, + { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), + fieldKeyDuration.Marshal(): nil, }, }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ - fieldKeyDuration.Marshal(): nil, + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(500)), }, + Timestamp: int64(1001), }, }, }, @@ -147,17 +155,16 @@ func TestStore_Search(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): []byte("svc2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), - EntityValues: []byte("test3"), + EntityValues: []byte("test1"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): nil, @@ -165,12 +172,12 @@ func TestStore_Search(t *testing.T) { }, { Key: index.Series{ - ID: common.SeriesID(1), - EntityValues: []byte("test1"), + EntityValues: []byte("test3"), }, Fields: map[string][]byte{ fieldKeyServiceName.Marshal(): nil, }, + Timestamp: int64(1001), }, }, }, @@ -191,7 +198,7 @@ func TestStore_Search(t *testing.T) { name += string(term) + "-" } t.Run(name, func(t *testing.T) { - query, err := s.BuildQuery(matchers, nil) + query, err := s.BuildQuery(matchers, nil, nil) require.NotEmpty(t, query.String()) require.NoError(t, err) got, err := s.Search(context.Background(), tt.projection, query) @@ -228,21 +235,26 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), EntityValues: []byte("test3"), }, + Timestamp: int64(1001), + }, + { + Key: index.Series{ + EntityValues: []byte("test4"), + }, + Timestamp: int64(2001), }, }, }, @@ -251,9 +263,9 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, }, }, @@ -262,7 +274,6 @@ func TestStore_SearchWildcard(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, @@ -281,7 +292,7 @@ func TestStore_SearchWildcard(t *testing.T) { Type: index.SeriesMatcherTypeWildcard, Match: tt.wildcard, }, - }, nil) + }, nil, nil) require.NoError(t, err) require.NotEmpty(t, query.String()) got, err := s.Search(context.Background(), tt.projection, query) @@ -318,21 +329,26 @@ func TestStore_SearchPrefix(t *testing.T) { want: []index.SeriesDocument{ { Key: index.Series{ - ID: common.SeriesID(1), EntityValues: []byte("test1"), }, }, { Key: index.Series{ - ID: common.SeriesID(2), EntityValues: []byte("test2"), }, + Timestamp: int64(101), }, { Key: index.Series{ - ID: common.SeriesID(3), EntityValues: []byte("test3"), }, + Timestamp: int64(1001), + }, + { + Key: index.Series{ + EntityValues: []byte("test4"), + }, + Timestamp: int64(2001), }, }, }, @@ -349,7 +365,7 @@ func TestStore_SearchPrefix(t *testing.T) { Type: index.SeriesMatcherTypePrefix, Match: tt.prefix, }, - }, nil) + }, nil, nil) require.NoError(t, err) require.NotEmpty(t, query.String()) got, err := s.Search(context.Background(), tt.projection, query) @@ -359,41 +375,600 @@ func TestStore_SearchPrefix(t *testing.T) { } } +func TestStore_SearchWithSecondaryQuery(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the secondary query + secondaryQuery := &queryNode{ + query: bluge.NewTermQuery("svc2").SetField(fieldKeyServiceName.Marshal()), + node: newTermNode("svc2", nil), + } + + // Test cases + tests := []struct { + term [][]byte + want []index.SeriesDocument + projection []index.FieldKey + }{ + { + term: [][]byte{[]byte("test2")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration, {TagName: "short_name"}}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + "short_name": []byte("t2"), + }, + Timestamp: int64(101), + }, + }, + }, + { + term: [][]byte{[]byte("test3")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{}, + }, + { + term: [][]byte{[]byte("test1")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{}, + }, + { + term: [][]byte{[]byte("test2"), []byte("test3")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration, {TagName: "short_name"}}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + "short_name": []byte("t2"), + }, + Timestamp: int64(101), + }, + }, + }, + { + term: [][]byte{[]byte("test1"), []byte("test2")}, + projection: []index.FieldKey{fieldKeyServiceName, fieldKeyDuration}, + want: []index.SeriesDocument{ + { + Key: index.Series{ + EntityValues: []byte("test2"), + }, + Fields: map[string][]byte{ + fieldKeyDuration.Marshal(): convert.Int64ToBytes(int64(100)), + fieldKeyServiceName.Marshal(): []byte("svc2"), + }, + Timestamp: int64(101), + }, + }, + }, + } + + for _, tt := range tests { + var matchers []index.SeriesMatcher + var name string + for _, term := range tt.term { + matchers = append(matchers, index.SeriesMatcher{ + Type: index.SeriesMatcherTypeExact, + Match: term, + }) + name += string(term) + "-" + } + t.Run(name, func(t *testing.T) { + query, err := s.BuildQuery(matchers, secondaryQuery, nil) + require.NotEmpty(t, query.String()) + require.NoError(t, err) + got, err := s.Search(context.Background(), tt.projection, query) + require.NoError(t, err) + assert.Equal(t, tt.want, got) + }) + } +} + +func TestStore_SeriesSort(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the order by field + orderBy := &index.OrderBy{ + Index: &databasev1.IndexRule{ + Metadata: &commonv1.Metadata{ + Id: fieldKeyStartTime.IndexRuleID, + }, + }, + Sort: modelv1.Sort_SORT_ASC, + Type: index.OrderByTypeIndex, + } + + // Test cases + tests := []struct { + name string + orderBy *index.OrderBy + timeRange *timestamp.TimeRange + want []index.DocumentResult + fieldKeys []index.FieldKey + }{ + { + name: "Sort by start_time ascending", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + { + EntityValues: []byte("test1"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): nil, + }, + SortedValue: []byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, + }, + }, + }, + { + name: "Sort by start_time descending", + orderBy: &index.OrderBy{ + Index: &databasev1.IndexRule{ + Metadata: &commonv1.Metadata{ + Id: fieldKeyStartTime.IndexRuleID, + }, + }, + Sort: modelv1.Sort_SORT_DESC, + Type: index.OrderByTypeIndex, + }, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + want: []index.DocumentResult{ + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test1"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): nil, + }, + SortedValue: []byte{0x00}, + }, + }, + }, + { + name: "Sort by start_time ascending with time range 100 to 1000", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 100), time.Unix(0, 1000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + }, + }, + { + name: "Sort by start_time ascending with time range 0 to 2000", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 0), time.Unix(0, 2000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(100)), + }, + SortedValue: convert.Int64ToBytes(int64(100)), + }, + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + { + EntityValues: []byte("test4"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(2000)), + }, + SortedValue: convert.Int64ToBytes(int64(2000)), + }, + }, + }, + { + name: "Sort by start_time ascending with time range 500 to 1500", + orderBy: orderBy, + fieldKeys: []index.FieldKey{fieldKeyStartTime}, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 500), time.Unix(0, 1500)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test3"), + Values: map[string][]byte{ + fieldKeyStartTime.Marshal(): convert.Int64ToBytes(int64(1000)), + }, + SortedValue: convert.Int64ToBytes(int64(1000)), + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + var secondaryQuery index.Query + if tt.timeRange != nil { + secondaryQuery = &queryNode{ + query: bluge.NewTermRangeInclusiveQuery( + string(convert.Int64ToBytes(tt.timeRange.Start.Local().UnixNano())), + string(convert.Int64ToBytes(tt.timeRange.End.Local().UnixNano())), + tt.timeRange.IncludeStart, + tt.timeRange.IncludeEnd, + ).SetField(fieldKeyStartTime.Marshal()), + } + } + query, err := s.BuildQuery([]index.SeriesMatcher{ + { + Type: index.SeriesMatcherTypePrefix, + Match: []byte("test"), + }, + }, secondaryQuery, nil) + require.NoError(t, err) + iter, err := s.SeriesSort(context.Background(), query, tt.orderBy, 10, tt.fieldKeys) + require.NoError(t, err) + defer iter.Close() + + var got []index.DocumentResult + for iter.Next() { + var g index.DocumentResult + val := iter.Val() + g.DocID = val.DocID + g.EntityValues = val.EntityValues + g.Values = maps.Clone(val.Values) + g.SortedValue = val.SortedValue + got = append(got, g) + } + assert.Equal(t, tt.want, got) + }) + } +} + +func TestStore_TimestampSort(t *testing.T) { + tester := require.New(t) + path, fn := setUp(tester) + s, err := NewStore(StoreOpts{ + Path: path, + Logger: logger.GetLogger("test"), + }) + tester.NoError(err) + defer func() { + tester.NoError(s.Close()) + fn() + }() + + // Setup some data + setupData(tester, s) + + // Define the order by field + orderBy := &index.OrderBy{ + Type: index.OrderByTypeTime, + Sort: modelv1.Sort_SORT_ASC, + } + + // Test cases + tests := []struct { + name string + orderBy *index.OrderBy + timeRange *timestamp.TimeRange + want []index.DocumentResult + fieldKeys []index.FieldKey + }{ + { + name: "Sort by timestamp ascending", + orderBy: orderBy, + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + { + EntityValues: []byte("test4"), + Timestamp: int64(2001), + SortedValue: numeric.MustNewPrefixCodedInt64(2001, 0), + }, + { + EntityValues: []byte("test1"), + Timestamp: 0, + SortedValue: []byte{0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}, + }, + }, + }, + { + name: "Sort by timestamp descending", + orderBy: &index.OrderBy{ + Type: index.OrderByTypeTime, + Sort: modelv1.Sort_SORT_DESC, + }, + want: []index.DocumentResult{ + { + EntityValues: []byte("test4"), + Timestamp: int64(2001), + SortedValue: numeric.MustNewPrefixCodedInt64(2001, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test1"), + Timestamp: 0, + SortedValue: []byte{0x00}, + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 100 to 1000", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 100), time.Unix(0, 1000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 0 to 2000", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 0), time.Unix(0, 2000)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test2"), + Timestamp: int64(101), + SortedValue: numeric.MustNewPrefixCodedInt64(101, 0), + }, + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + }, + }, + { + name: "Sort by timestamp ascending with time range 500 to 1500", + orderBy: orderBy, + timeRange: func() *timestamp.TimeRange { + tr := timestamp.NewInclusiveTimeRange(time.Unix(0, 500), time.Unix(0, 1500)) + return &tr + }(), + want: []index.DocumentResult{ + { + EntityValues: []byte("test3"), + Timestamp: int64(1001), + SortedValue: numeric.MustNewPrefixCodedInt64(1001, 0), + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + query, err := s.BuildQuery([]index.SeriesMatcher{ + { + Type: index.SeriesMatcherTypePrefix, + Match: []byte("test"), + }, + }, nil, tt.timeRange) + require.NoError(t, err) + iter, err := s.SeriesSort(context.Background(), query, tt.orderBy, 10, tt.fieldKeys) + require.NoError(t, err) + defer iter.Close() + + var got []index.DocumentResult + for iter.Next() { + var g index.DocumentResult + val := iter.Val() + g.DocID = val.DocID + g.EntityValues = val.EntityValues + if len(val.Values) > 0 { + g.Values = maps.Clone(val.Values) + } + g.SortedValue = val.SortedValue + g.Timestamp = val.Timestamp + got = append(got, g) + } + assert.Equal(t, tt.want, got) + }) + } +} + func setupData(tester *require.Assertions, s index.SeriesStore) { series1 := index.Document{ - DocID: 1, EntityValues: []byte("test1"), } series2 := index.Document{ - DocID: 2, EntityValues: []byte("test2"), Fields: []index.Field{ { Key: fieldKeyDuration, Term: convert.Int64ToBytes(int64(100)), Store: true, + Index: true, }, { Key: fieldKeyServiceName, Term: []byte("svc2"), Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(100)), + Store: true, + Index: true, + }, + { + Key: index.FieldKey{ + TagName: "short_name", + }, + Term: []byte("t2"), + Store: true, + Index: false, }, }, + Timestamp: int64(101), } series3 := index.Document{ - DocID: 3, EntityValues: []byte("test3"), Fields: []index.Field{ { Key: fieldKeyDuration, Term: convert.Int64ToBytes(int64(500)), Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(1000)), + Store: true, + Index: true, + }, + { + Key: index.FieldKey{ + TagName: "short_name", + }, + Term: []byte("t3"), + Store: true, + Index: false, }, }, + Timestamp: int64(1001), } + series4 := index.Document{ + EntityValues: []byte("test4"), + Fields: []index.Field{ + { + Key: fieldKeyDuration, + Term: convert.Int64ToBytes(int64(500)), + Store: true, + Index: true, + }, + { + Key: fieldKeyStartTime, + Term: convert.Int64ToBytes(int64(2000)), + Store: true, + Index: true, + }, + }, + Timestamp: int64(2001), + } + tester.NoError(s.SeriesBatch(index.Batch{ + Documents: []index.Document{series1, series2, series4, series3}, + })) tester.NoError(s.SeriesBatch(index.Batch{ - Documents: []index.Document{series1, series2, series3, series3}, + Documents: []index.Document{series3}, })) } diff --git a/pkg/index/inverted/query.go b/pkg/index/inverted/query.go index f44873528..2dc630930 100644 --- a/pkg/index/inverted/query.go +++ b/pkg/index/inverted/query.go @@ -31,6 +31,7 @@ import ( "github.com/apache/skywalking-banyandb/pkg/convert" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/query/logical" + "github.com/apache/skywalking-banyandb/pkg/timestamp" ) var ( @@ -466,3 +467,23 @@ func (m *wildcardNode) MarshalJSON() ([]byte, error) { func (m *wildcardNode) String() string { return convert.JSONToString(m) } + +type timeRangeNode struct { + timeRange *timestamp.TimeRange +} + +func newTimeRangeNode(timeRange *timestamp.TimeRange) *timeRangeNode { + return &timeRangeNode{ + timeRange: timeRange, + } +} + +func (t *timeRangeNode) MarshalJSON() ([]byte, error) { + data := make(map[string]interface{}, 1) + data["time_range"] = t.timeRange.String() + return json.Marshal(data) +} + +func (t *timeRangeNode) String() string { + return convert.JSONToString(t) +} diff --git a/pkg/index/inverted/sort.go b/pkg/index/inverted/sort.go index 68071cc90..23cc57df2 100644 --- a/pkg/index/inverted/sort.go +++ b/pkg/index/inverted/sort.go @@ -119,7 +119,7 @@ func (si *sortIterator) loadCurrent() bool { size := si.size + si.skipped if size < 0 { // overflow - size = math.MaxInt64 + size = math.MaxInt } topNSearch := bluge.NewTopNSearch(size, si.query.(*queryNode).query).SortBy([]string{si.sortedKey}) if si.skipped > 0 { diff --git a/pkg/partition/index.go b/pkg/partition/index.go index 9ccb08691..bbb4daf03 100644 --- a/pkg/partition/index.go +++ b/pkg/partition/index.go @@ -40,7 +40,7 @@ type FieldIndexLocation map[string]map[string]FieldWithType // ParseIndexRuleLocators returns a IndexRuleLocator based on the tag family spec and index rules. func ParseIndexRuleLocators(entity *databasev1.Entity, families []*databasev1.TagFamilySpec, - indexRules []*databasev1.IndexRule, + indexRules []*databasev1.IndexRule, indexMode bool, ) (locators IndexRuleLocator, fil FieldIndexLocation) { locators.EntitySet = make(map[string]struct{}, len(entity.TagNames)) fil = make(FieldIndexLocation) @@ -64,18 +64,28 @@ func ParseIndexRuleLocators(entity *databasev1.Entity, families []*databasev1.Ta ir := findIndexRuleByTagName(families[i].Tags[j].Name) if ir != nil { ttr[families[i].Tags[j].Name] = ir - tagFamily, ok := fil[families[i].Name] - if !ok { - tagFamily = make(map[string]FieldWithType) - fil[families[i].Name] = tagFamily + } + if ir == nil && !indexMode { + continue + } + tagFamily, ok := fil[families[i].Name] + if !ok { + tagFamily = make(map[string]FieldWithType) + fil[families[i].Name] = tagFamily + } + fwt := FieldWithType{ + Type: pbv1.MustTagValueSpecToValueType(families[i].Tags[j].Type), + } + if ir != nil { + fwt.Key = index.FieldKey{ + IndexRuleID: ir.Metadata.Id, } - tagFamily[families[i].Tags[j].Name] = FieldWithType{ - Key: index.FieldKey{ - IndexRuleID: ir.Metadata.Id, - }, - Type: pbv1.MustTagValueSpecToValueType(families[i].Tags[j].Type), + } else { + fwt.Key = index.FieldKey{ + TagName: families[i].Tags[j].Name, } } + tagFamily[families[i].Tags[j].Name] = fwt } } return locators, fil diff --git a/pkg/pb/v1/series.go b/pkg/pb/v1/series.go index a0e768276..542343d94 100644 --- a/pkg/pb/v1/series.go +++ b/pkg/pb/v1/series.go @@ -72,6 +72,7 @@ func (s *Series) MarshalWithWildcard() error { // Unmarshal decodes series from internal Buffer. func (s *Series) Unmarshal(src []byte) error { + s.ID = common.SeriesID(convert.Hash(src)) var err error s.Buffer = s.Buffer[:0] if s.Buffer, src, err = unmarshalEntityValue(s.Buffer, src); err != nil { diff --git a/pkg/query/logical/measure/measure_plan_indexscan_local.go b/pkg/query/logical/measure/measure_plan_indexscan_local.go index bf11e1115..4e2745f87 100644 --- a/pkg/query/logical/measure/measure_plan_indexscan_local.go +++ b/pkg/query/logical/measure/measure_plan_indexscan_local.go @@ -135,29 +135,33 @@ func (i *localIndexScan) Sort(order *logical.OrderBy) { } func (i *localIndexScan) Execute(ctx context.Context) (mit executor.MIterator, err error) { - var orderBy *model.OrderBy - orderByType := model.OrderByTypeTime + var orderBy *index.OrderBy + if i.order != nil { - if i.order.Index != nil { - orderByType = model.OrderByTypeIndex - } - orderBy = &model.OrderBy{ - Index: i.order.Index, + orderBy = &index.OrderBy{ Sort: i.order.Sort, + Index: i.order.Index, + } + if orderBy.Index == nil { + orderBy.Type = index.OrderByTypeTime + } else { + orderBy.Type = index.OrderByTypeIndex } } if i.groupByEntity { - orderByType = model.OrderByTypeSeries + if orderBy == nil { + orderBy = &index.OrderBy{} + } + orderBy.Type = index.OrderByTypeSeries } ec := executor.FromMeasureExecutionContext(ctx) - ctx, stop := i.startSpan(ctx, query.GetTracer(ctx), orderByType, orderBy) + ctx, stop := i.startSpan(ctx, query.GetTracer(ctx), orderBy) defer stop(err) result, err := ec.Query(ctx, model.MeasureQueryOptions{ Name: i.metadata.GetName(), TimeRange: &i.timeRange, Entities: i.entities, Query: i.query, - OrderByType: orderByType, Order: orderBy, TagProjection: i.projectionTags, FieldProjection: i.projectionFields, @@ -269,20 +273,24 @@ func (ei *resultMIterator) Close() error { return ei.err } -func (i *localIndexScan) startSpan(ctx context.Context, tracer *query.Tracer, orderType model.OrderByType, orderBy *model.OrderBy) (context.Context, func(error)) { +func (i *localIndexScan) startSpan(ctx context.Context, tracer *query.Tracer, orderBy *index.OrderBy) (context.Context, func(error)) { if tracer == nil { return ctx, func(error) {} } span, ctx := tracer.StartSpan(ctx, "indexScan-%s", i.metadata) - sortName := modelv1.Sort_name[int32(orderBy.Sort)] - switch orderType { - case model.OrderByTypeTime: - span.Tag("orderBy", "time "+sortName) - case model.OrderByTypeIndex: - span.Tag("orderBy", fmt.Sprintf("indexRule:%s", orderBy.Index.Metadata.Name)) - case model.OrderByTypeSeries: - span.Tag("orderBy", "series") + if orderBy != nil { + sortName := modelv1.Sort_name[int32(orderBy.Sort)] + switch orderBy.Type { + case index.OrderByTypeTime: + span.Tag("orderBy", "time-"+sortName) + case index.OrderByTypeIndex: + span.Tag("orderBy", fmt.Sprintf("indexRule:%s-%s", orderBy.Index.Metadata.Name, sortName)) + case index.OrderByTypeSeries: + span.Tag("orderBy", "series") + } + } else { + span.Tag("orderBy", "time-asc(default)") } span.Tag("details", i.String()) diff --git a/pkg/query/logical/measure/topn_plan_localscan.go b/pkg/query/logical/measure/topn_plan_localscan.go index f068c156c..9608af959 100644 --- a/pkg/query/logical/measure/topn_plan_localscan.go +++ b/pkg/query/logical/measure/topn_plan_localscan.go @@ -27,6 +27,7 @@ import ( commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" + "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/logger" pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1" "github.com/apache/skywalking-banyandb/pkg/query/executor" @@ -158,7 +159,7 @@ func (i *localScan) Execute(ctx context.Context) (mit executor.MIterator, err er Name: i.metadata.GetName(), TimeRange: &i.timeRange, Entities: [][]*modelv1.TagValue{i.entity}, - Order: &model.OrderBy{Sort: i.sort}, + Order: &index.OrderBy{Sort: i.sort}, TagProjection: i.projectionTags, FieldProjection: i.projectionFields, }) diff --git a/pkg/query/logical/optimizer.go b/pkg/query/logical/optimizer.go index a087cdbcb..471f78b8c 100644 --- a/pkg/query/logical/optimizer.go +++ b/pkg/query/logical/optimizer.go @@ -74,7 +74,8 @@ func NewPushDownOrder(order *modelv1.QueryOrder) PushDownOrder { // Optimize a Plan by pushing down the query order. func (pdo PushDownOrder) Optimize(plan Plan) (Plan, error) { if v, ok := plan.(Sorter); ok { - if order, err := ParseOrderBy(v.Schema(), pdo.order.GetIndexRuleName(), pdo.order.GetSort()); err == nil { + if order, err := ParseOrderBy(v.Schema(), + pdo.order.GetIndexRuleName(), pdo.order.GetSort()); err == nil && order != nil { v.Sort(order) } else { return nil, err diff --git a/pkg/query/logical/plan.go b/pkg/query/logical/plan.go index 5417b6d81..4d30a1f91 100644 --- a/pkg/query/logical/plan.go +++ b/pkg/query/logical/plan.go @@ -63,6 +63,9 @@ func (o *OrderBy) String() string { // ParseOrderBy parses an OrderBy from a Schema. func ParseOrderBy(s Schema, indexRuleName string, sort modelv1.Sort) (*OrderBy, error) { if indexRuleName == "" { + if sort == modelv1.Sort_SORT_UNSPECIFIED { + return nil, nil + } return &OrderBy{ Sort: sort, }, nil diff --git a/pkg/query/logical/stream/stream_plan_indexscan_local.go b/pkg/query/logical/stream/stream_plan_indexscan_local.go index f7db768a1..eda67111c 100644 --- a/pkg/query/logical/stream/stream_plan_indexscan_local.go +++ b/pkg/query/logical/stream/stream_plan_indexscan_local.go @@ -81,9 +81,9 @@ func (i *localIndexScan) Execute(ctx context.Context) ([]*streamv1.Element, erro if i.result != nil { return BuildElementsFromStreamResult(ctx, i.result), nil } - var orderBy *model.OrderBy + var orderBy *index.OrderBy if i.order != nil { - orderBy = &model.OrderBy{ + orderBy = &index.OrderBy{ Index: i.order.Index, Sort: i.order.Sort, } diff --git a/pkg/query/model/model.go b/pkg/query/model/model.go index 022bcdbc4..768e6fad0 100644 --- a/pkg/query/model/model.go +++ b/pkg/query/model/model.go @@ -22,7 +22,6 @@ import ( "context" "github.com/apache/skywalking-banyandb/api/common" - databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1" modelv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/model/v1" "github.com/apache/skywalking-banyandb/pkg/index" "github.com/apache/skywalking-banyandb/pkg/timestamp" @@ -52,34 +51,15 @@ type TagProjection struct { Names []string } -// OrderBy is the order by rule. -type OrderBy struct { - Index *databasev1.IndexRule - Sort modelv1.Sort -} - -// OrderByType is the type of order by. -type OrderByType int - -const ( - // OrderByTypeTime is the order by time. - OrderByTypeTime OrderByType = iota - // OrderByTypeIndex is the order by index. - OrderByTypeIndex - // OrderByTypeSeries is the order by series. - OrderByTypeSeries -) - // MeasureQueryOptions is the options of a measure query. type MeasureQueryOptions struct { Query index.Query TimeRange *timestamp.TimeRange - Order *OrderBy + Order *index.OrderBy Name string Entities [][]*modelv1.TagValue TagProjection []TagProjection FieldProjection []string - OrderByType OrderByType } // MeasureResult is the result of a query. @@ -104,7 +84,7 @@ type StreamQueryOptions struct { TimeRange *timestamp.TimeRange Entities [][]*modelv1.TagValue Filter index.Filter - Order *OrderBy + Order *index.OrderBy TagProjection []TagProjection MaxElementSize int } diff --git a/pkg/test/measure/testdata/measures/service_traffic.json b/pkg/test/measure/testdata/measures/service_traffic.json index 7edc682cd..887d1adb0 100644 --- a/pkg/test/measure/testdata/measures/service_traffic.json +++ b/pkg/test/measure/testdata/measures/service_traffic.json @@ -39,5 +39,6 @@ "id" ] }, + "index_mode": true, "updated_at": "2021-04-15T01:30:15.01Z" } \ No newline at end of file diff --git a/pkg/timestamp/range.go b/pkg/timestamp/range.go index b0b92afdf..f8db16523 100644 --- a/pkg/timestamp/range.go +++ b/pkg/timestamp/range.go @@ -60,6 +60,22 @@ func (t TimeRange) Overlapping(other TimeRange) bool { return !t.Start.After(other.End) && !other.Start.After(t.End) } +// Include returns whether the TimeRange includes the other TimeRange. +func (t TimeRange) Include(other TimeRange) bool { + var start, end bool + if t.Start.Equal(other.Start) { + start = t.IncludeStart || !other.IncludeStart + } else { + start = !t.Start.After(other.Start) + } + if t.End.Equal(other.End) { + end = t.IncludeEnd || !other.IncludeEnd + } else { + end = !t.End.Before(other.End) + } + return start && end +} + // Duration converts TimeRange to time.Duration. func (t TimeRange) Duration() time.Duration { return t.End.Sub(t.Start) diff --git a/test/cases/measure/data/input/index_mode_all.yaml b/test/cases/measure/data/input/index_mode_all.yaml new file mode 100644 index 000000000..d0c014506 --- /dev/null +++ b/test/cases/measure/data/input/index_mode_all.yaml @@ -0,0 +1,23 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "short_name", "service_group", "layer" ] diff --git a/test/cases/measure/data/input/index_mode_order_desc.yaml b/test/cases/measure/data/input/index_mode_order_desc.yaml new file mode 100644 index 000000000..0dd0063da --- /dev/null +++ b/test/cases/measure/data/input/index_mode_order_desc.yaml @@ -0,0 +1,26 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "layer" ] +orderBy: + sort: "SORT_DESC" + indexRuleName: "layer" diff --git a/test/cases/measure/data/input/index_mode_range.yaml b/test/cases/measure/data/input/index_mode_range.yaml new file mode 100644 index 000000000..c881b11cf --- /dev/null +++ b/test/cases/measure/data/input/index_mode_range.yaml @@ -0,0 +1,30 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +name: "service_traffic" +groups: [ "sw_metric" ] +tagProjection: + tagFamilies: + - name: "default" + tags: [ "id", "service_id", "name", "layer" ] +criteria: + condition: + name: "layer" + op: "BINARY_OP_GT" + value: + int: + value: "1" diff --git a/test/cases/measure/data/want/index_mode_all.yaml b/test/cases/measure/data/want/index_mode_all.yaml new file mode 100644 index 000000000..3d3edeb2d --- /dev/null +++ b/test/cases/measure/data/want/index_mode_all.yaml @@ -0,0 +1,107 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +dataPoints: +- sid: "15142466043926325685" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "1" + - key: service_id + value: + str: + value: service_1 + - key: name + value: + str: + value: service_name_1 + - key: short_name + value: + str: + value: service_short_name_1 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:02:00Z" + version: "1" +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: short_name + value: + str: + value: service_short_name_2 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:03:00Z" + version: "1" +- sid: "12370392692163567533" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "3" + - key: service_id + value: + str: + value: service_3 + - key: name + value: + str: + value: service_name_3 + - key: short_name + value: + str: + value: service_short_name_3 + - key: service_group + value: + str: + value: group1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:04:00Z" + version: "1" diff --git a/test/cases/measure/data/want/index_mode_order_desc.yaml b/test/cases/measure/data/want/index_mode_order_desc.yaml new file mode 100644 index 000000000..470fba1c3 --- /dev/null +++ b/test/cases/measure/data/want/index_mode_order_desc.yaml @@ -0,0 +1,83 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +dataPoints: +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:40:00Z" + version: "1" +- sid: "15142466043926325685" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "1" + - key: service_id + value: + str: + value: service_1 + - key: name + value: + str: + value: service_name_1 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:39:00Z" + version: "1" +- sid: "12370392692163567533" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "3" + - key: service_id + value: + str: + value: service_3 + - key: name + value: + str: + value: service_name_3 + - key: layer + value: + int: + value: "1" + timestamp: "2024-11-15T01:41:00Z" + version: "1" diff --git a/test/cases/measure/data/want/index_mode_range.yaml b/test/cases/measure/data/want/index_mode_range.yaml new file mode 100644 index 000000000..f9c672262 --- /dev/null +++ b/test/cases/measure/data/want/index_mode_range.yaml @@ -0,0 +1,39 @@ +# Licensed to Apache Software Foundation (ASF) under one or more contributor +# license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright +# ownership. Apache Software Foundation (ASF) licenses this file to you under +# the Apache License, Version 2.0 (the "License"); you may +# not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +dataPoints: +- sid: "3906119849472468294" + tagFamilies: + - name: default + tags: + - key: id + value: + str: + value: "2" + - key: service_id + value: + str: + value: service_2 + - key: name + value: + str: + value: service_name_2 + - key: layer + value: + int: + value: "2" + timestamp: "2024-11-15T01:43:00Z" + version: "1" diff --git a/test/cases/measure/measure.go b/test/cases/measure/measure.go index 8f843105b..bc31747d6 100644 --- a/test/cases/measure/measure.go +++ b/test/cases/measure/measure.go @@ -72,4 +72,7 @@ var _ = g.DescribeTable("Scanning Measures", verify, g.Entry("all_latency", helpers.Args{Input: "all_latency", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), g.Entry("duplicated in a part", helpers.Args{Input: "duplicated_part", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), g.Entry("match a tag belongs to the entity", helpers.Args{Input: "entity_match", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("all of index mode", helpers.Args{Input: "index_mode_all", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("order by desc of index mode", helpers.Args{Input: "index_mode_order_desc", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), + g.Entry("range of index mode", helpers.Args{Input: "index_mode_range", Duration: 25 * time.Minute, Offset: -20 * time.Minute}), ) diff --git a/test/stress/istio/report.md b/test/stress/istio/report.md index 8089e50f9..28ba80cce 100644 --- a/test/stress/istio/report.md +++ b/test/stress/istio/report.md @@ -16,122 +16,120 @@ VUS: 8192 Machine and OS information: -- CPU: CPU: AMD EPYC 7B12, VendorID: AuthenticAMD, Family: 23, Model: 49, Cores: 8 -- Memory: Total: 47176 MB -- OS: OS: linux, Platform: ubuntu, Platform Family: debian, Platform Version: 21.04 +- CPU: CPU: AMD EPYC 7B13, VendorID: AuthenticAMD, Family: 25, Model: 1, Cores: 16 +- Memory: Total: 15986 MB +- OS: OS: linux, Platform: ubuntu, Platform Family: debian, Platform Version: 24.04 ## Result -written 16186000 items in 38m38.657047557s -throughput: 6980.765015 items/s -throughput(kb/s) 1786.494948 -latency: 2ns +written 16186000 items in 38m25.757182449s +throughput: 7019.819833 items/s +throughput(kb/s) 1796.489734 +latency: 167.193µs -The Memory and Disk usage is improved due to moving indexed value from data files to index files. +CPU and Memory usage are improved due to reducing the updates of the index. ### Problem -Memory: Series Index(bluge) index building took most of the memory. - -CPU: As memory's part, Series Index(bluge) index building took most of the CPU. - -Next step: Optimize the Series Index(bluge) index building. +Series Index Searching took most of the time. Add a cache would be helpful. ## CPU -CPU 95th-percentile: 1.24 cores (8 cores in total) +CPU 95th-percentile: 0.75 cores (16 cores in total) ```bash -Showing top 10 nodes out of 326 +Showing top 10 nodes out of 301 flat flat% sum% cum cum% - 229.15s 7.80% 7.80% 229.15s 7.80% runtime/internal/syscall.Syscall6 - 152.36s 5.18% 12.98% 436.89s 14.86% runtime.mallocgc - 107.72s 3.66% 16.64% 321.02s 10.92% runtime.scanobject - 93.73s 3.19% 19.83% 179.11s 6.09% github.com/blevesearch/vellum.registryCache.entry - 87.88s 2.99% 22.82% 104.15s 3.54% runtime.findObject - 68.76s 2.34% 25.16% 68.96s 2.35% github.com/blevesearch/vellum.(*builderNode).equiv (inline) - 56.29s 1.91% 27.08% 56.29s 1.91% runtime.nextFreeFast (inline) - 49.87s 1.70% 28.77% 49.87s 1.70% runtime.memmove - 46.09s 1.57% 30.34% 69.86s 2.38% runtime.(*mspan).writeHeapBitsSmall - 45.67s 1.55% 31.89% 45.67s 1.55% runtime.memclrNoHeapPointers + 335.22s 18.10% 18.10% 335.22s 18.10% internal/runtime/syscall.Syscall6 + 75.99s 4.10% 22.20% 225.86s 12.19% runtime.mallocgc + 43.86s 2.37% 24.57% 48.40s 2.61% github.com/blevesearch/vellum.(*fstStateV1).atSingle + 39.80s 2.15% 26.72% 39.80s 2.15% runtime.memmove + 39.45s 2.13% 28.85% 39.45s 2.13% runtime.usleep + 36.23s 1.96% 30.80% 218.49s 11.80% github.com/blevesearch/vellum.(*FST).get + 32.43s 1.75% 32.56% 32.43s 1.75% runtime.nextFreeFast (inline) + 30.64s 1.65% 34.21% 30.64s 1.65% runtime.futex + 28.75s 1.55% 35.76% 148.49s 8.02% github.com/blevesearch/vellum.(*decoderV1).stateAt + 27.72s 1.50% 37.26% 27.72s 1.50% runtime.memclrNoHeapPointers ``` ## Heap Profile -`alloc_bytes` 95th-percentile: 947.97 MB. -`heap_inuse_bytes` 95th-percentile: 1.10 GB. -`sys_bytes` 95th-percentile: 1.75 GB. -`stack_inuse_bytes` 95th-percentile: 20.58 MB. +`alloc_bytes` 95th-percentile: 431.5 MB. +`heap_inuse_bytes` 95th-percentile: 452.97 GB. +`sys_bytes` 95th-percentile: 892.03 GB. +`stack_inuse_bytes` 95th-percentile: 2.35 MB. ```bash -Showing top 10 nodes out of 225 +Showing top 10 nodes out of 238 flat flat% sum% cum cum% - 32.36GB 9.63% 9.63% 32.36GB 9.63% github.com/blevesearch/vellum.(*unfinishedNodes).get - 26.01GB 7.74% 17.36% 26.01GB 7.74% reflect.New - 16.32GB 4.86% 22.22% 16.32GB 4.86% github.com/blevesearch/vellum.(*builderNodePool).Get - 13.81GB 4.11% 26.33% 13.81GB 4.11% github.com/apache/skywalking-banyandb/banyand/measure.(*blockPointer).append - 13.74GB 4.09% 30.42% 13.74GB 4.09% github.com/blugelabs/bluge/analysis.TokenFrequency - 12.63GB 3.76% 34.17% 12.63GB 3.76% github.com/blugelabs/ice.(*interim).processDocument.func1.1 - 10.44GB 3.11% 37.28% 10.44GB 3.11% github.com/blevesearch/vellum.(*builderNodeUnfinished).lastCompiled - 9.42GB 2.80% 40.08% 9.42GB 2.80% bytes.growSlice - 8.57GB 2.55% 42.63% 22.18GB 6.60% github.com/apache/skywalking-banyandb/banyand/measure.(*writeCallback).handle - 8.40GB 2.50% 45.13% 8.40GB 2.50% github.com/RoaringBitmap/roaring.(*Bitmap).Iterator + 27.77GB 12.09% 12.09% 27.77GB 12.09% reflect.New + 22.72GB 9.89% 21.98% 22.72GB 9.89% github.com/blugelabs/bluge.Identifier.Term + 8.50GB 3.70% 25.68% 8.50GB 3.70% github.com/blevesearch/vellum.(*decoderV1).stateAt + 7.54GB 3.28% 28.96% 54.19GB 23.59% github.com/apache/skywalking-banyandb/test/stress/istio.ReadAndWriteFromFile.func4 + 7.53GB 3.28% 32.23% 22.32GB 9.72% github.com/apache/skywalking-banyandb/banyand/measure.(*writeCallback).handle + 6.57GB 2.86% 35.09% 6.57GB 2.86% strings.(*Builder).grow + 5.39GB 2.34% 37.44% 5.69GB 2.48% google.golang.org/protobuf/proto.MarshalOptions.marshal + 5.30GB 2.30% 39.74% 5.30GB 2.30% google.golang.org/protobuf/internal/encoding/json.(*Decoder).parseString + 4.95GB 2.15% 41.90% 4.95GB 2.16% github.com/klauspost/compress/s2.encodeBlockBetter + 4.71GB 2.05% 43.95% 4.71GB 2.05% github.com/apache/skywalking-banyandb/banyand/measure.fastFieldAppend ``` ## Disk Usage ```bash -measure: 368 MB -measure/measure-default: 182 MB -measure/measure-default/seg-20240731: 182 MB -measure/measure-default/seg-20240731/shard-0: 147 MB -measure/measure-default/seg-20240731/shard-0/0000000000001b5c: 75 MB -measure/measure-default/seg-20240731/shard-0/0000000000001c0b: 76 kB -measure/measure-default/seg-20240731/shard-0/0000000000001c6c: 41 kB -measure/measure-default/seg-20240731/shard-0/0000000000001d37: 24 MB -measure/measure-default/seg-20240731/shard-0/0000000000001dae: 80 kB -measure/measure-default/seg-20240731/shard-0/0000000000001edc: 22 MB -measure/measure-default/seg-20240731/shard-0/0000000000001fad: 91 kB -measure/measure-default/seg-20240731/shard-0/0000000000001fb1: 218 kB -measure/measure-default/seg-20240731/shard-0/0000000000002018: 588 kB -measure/measure-default/seg-20240731/shard-0/0000000000002078: 23 MB -measure/measure-default/seg-20240731/shard-0/0000000000002081: 879 kB -measure/measure-default/seg-20240731/sidx: 36 MB -measure/measure-minute: 185 MB -measure/measure-minute/seg-20240731: 185 MB -measure/measure-minute/seg-20240731/shard-0: 80 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000e4d: 52 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000f21: 9.3 MB -measure/measure-minute/seg-20240731/shard-0/0000000000000ff8: 9.4 MB -measure/measure-minute/seg-20240731/shard-0/0000000000001068: 296 kB -measure/measure-minute/seg-20240731/shard-0/00000000000010cd: 9.1 MB -measure/measure-minute/seg-20240731/shard-1: 82 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000de3: 51 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000ead: 9.6 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000f82: 9.1 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fb0: 2.1 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe0: 2.3 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe6: 1.4 MB -measure/measure-minute/seg-20240731/shard-1/0000000000000fe9: 162 kB -measure/measure-minute/seg-20240731/shard-1/0000000000000ff1: 383 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001017: 1.8 MB -measure/measure-minute/seg-20240731/shard-1/000000000000101b: 229 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001053: 2.8 MB -measure/measure-minute/seg-20240731/shard-1/0000000000001057: 231 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001058: 74 kB -measure/measure-minute/seg-20240731/shard-1/0000000000001063: 477 kB -measure/measure-minute/seg-20240731/sidx: 24 MB +measure: 562 MB +measure/measure-default: 303 MB +measure/measure-default/seg-20241115: 303 MB +measure/measure-default/seg-20241115/shard-0: 271 MB +measure/measure-default/seg-20241115/shard-0/0000000000000b08: 83 MB +measure/measure-default/seg-20241115/shard-0/00000000000014bb: 80 MB +measure/measure-default/seg-20241115/shard-0/0000000000001e6b: 143 kB +measure/measure-default/seg-20241115/shard-0/0000000000001e6d: 329 kB +measure/measure-default/seg-20241115/shard-0/0000000000001e6e: 78 kB +measure/measure-default/seg-20241115/shard-0/0000000000001f35: 83 MB +measure/measure-default/seg-20241115/shard-0/0000000000001f75: 4.5 MB +measure/measure-default/seg-20241115/shard-0/0000000000001ff7: 9.2 MB +measure/measure-default/seg-20241115/shard-0/0000000000002009: 150 kB +measure/measure-default/seg-20241115/shard-0/0000000000002045: 11 MB +measure/measure-default/seg-20241115/sidx: 32 MB +measure/measure-minute: 259 MB +measure/measure-minute/seg-20241115: 259 MB +measure/measure-minute/seg-20241115/shard-0: 120 MB +measure/measure-minute/seg-20241115/shard-0/00000000000004d4: 33 MB +measure/measure-minute/seg-20241115/shard-0/00000000000009c5: 34 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000bc9: 59 kB +measure/measure-minute/seg-20241115/shard-0/0000000000000f2c: 35 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000f8f: 2.9 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000f96: 60 kB +measure/measure-minute/seg-20241115/shard-0/0000000000000fb8: 5.5 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000ff6: 2.8 MB +measure/measure-minute/seg-20241115/shard-0/0000000000000ff7: 60 kB +measure/measure-minute/seg-20241115/shard-0/000000000000101e: 1.8 MB +measure/measure-minute/seg-20241115/shard-0/0000000000001067: 3.3 MB +measure/measure-minute/seg-20241115/shard-0/000000000000106a: 114 kB +measure/measure-minute/seg-20241115/shard-0/000000000000107f: 1.2 MB +measure/measure-minute/seg-20241115/shard-1: 116 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000515: 35 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000a5c: 36 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000da4: 369 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000e67: 116 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000f34: 34 MB +measure/measure-minute/seg-20241115/shard-1/0000000000000fa1: 264 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000ffc: 116 kB +measure/measure-minute/seg-20241115/shard-1/0000000000000ffd: 59 kB +measure/measure-minute/seg-20241115/shard-1/0000000000001030: 11 MB +measure/measure-minute/seg-20241115/sidx: 23 MB ``` ## Disk IO -| Metric | 95th-percentile per second | -|---------------------|-----------------------------| -| read_count | 0.010000 | -| merged_read_count | 0.000000 | -| write_count | 20.978333 | -| merged_write_count | 25.373333 | -| read_bytes | 40.960000 | -| write_bytes | 2603731.626667 | -| io_time(ms) | 13.360000 | -| weighted_io(ms) | 60.828333 | +| Metric | 95th-percentile per second | +|---------------------|----------------------------| +| read_count | 0.06333 | +| merged_read_count | 0.000000 | +| write_count | 14.873333 | +| merged_write_count | 16.700000 | +| read_bytes | 996.693333 | +| write_bytes | 1689586.346667 | +| io_time(ms) | 7.486667 | +| weighted_io(ms) | 121.133333 | diff --git a/test/stress/istio/testdata/metrics/data.csv b/test/stress/istio/testdata/metrics/data.csv index 3b96410df..0e20b4f0b 100644 --- a/test/stress/istio/testdata/metrics/data.csv +++ b/test/stress/istio/testdata/metrics/data.csv @@ -1,153 +1,154 @@ -305855736.000000,328417280.000000,677992856.000000,5799936.000000,81.030000,314598531072.000000,532622512128.000000,19479256.000000,562851.000000,71599307.000000,123424074.000000,1369619356672.000000,2710871929856.000000,49022512.000000,364109771.000000 -253643696.000000,291848192.000000,678255000.000000,3997696.000000,136.900000,314643181568.000000,532622512128.000000,19479256.000000,562851.000000,71600117.000000,123424948.000000,1369619356672.000000,2710964245504.000000,49023032.000000,364111236.000000 -254710048.000000,293011456.000000,678255000.000000,3997696.000000,136.970000,314643189760.000000,532622512128.000000,19479256.000000,562851.000000,71600755.000000,123425610.000000,1369619356672.000000,2711033795584.000000,49023224.000000,364112532.000000 -255879192.000000,294207488.000000,678255000.000000,3997696.000000,137.040000,314643189760.000000,532622512128.000000,19479256.000000,562851.000000,71600792.000000,123425680.000000,1369619356672.000000,2711034364928.000000,49023252.000000,364112564.000000 -339392248.000000,357883904.000000,678255000.000000,6094848.000000,141.770000,314643197952.000000,532622512128.000000,19479256.000000,562851.000000,71600813.000000,123425691.000000,1369619356672.000000,2711034496000.000000,49023284.000000,364112583.000000 -219319848.000000,277151744.000000,759784872.000000,4325376.000000,211.010000,314665136128.000000,532622512128.000000,19479256.000000,562851.000000,71601860.000000,123426944.000000,1369619356672.000000,2711165113344.000000,49023964.000000,364115001.000000 -255773000.000000,299958272.000000,759784872.000000,5308416.000000,211.250000,314667843584.000000,532622512128.000000,19479256.000000,562851.000000,71601916.000000,123427058.000000,1369619356672.000000,2711168451584.000000,49024016.000000,364115053.000000 -256964024.000000,301228032.000000,759784872.000000,5308416.000000,211.310000,314667851776.000000,532622512128.000000,19479256.000000,562851.000000,71601968.000000,123427124.000000,1369619356672.000000,2711169041408.000000,49024052.000000,364115101.000000 -194574488.000000,270991360.000000,759784872.000000,3866624.000000,215.600000,314667851776.000000,532622512128.000000,19479256.000000,562851.000000,71601992.000000,123427158.000000,1369619356672.000000,2711169278976.000000,49024080.000000,364115124.000000 -374225056.000000,398573568.000000,768963000.000000,4718592.000000,281.670000,314718650368.000000,532622512128.000000,19479256.000000,562851.000000,71602926.000000,123428992.000000,1369619356672.000000,2711264330752.000000,49025020.000000,364116840.000000 -240959760.000000,284499968.000000,768963000.000000,5144576.000000,282.240000,314682363904.000000,532622512128.000000,19479256.000000,562851.000000,71603008.000000,123429121.000000,1369619356672.000000,2711271388160.000000,49025084.000000,364116950.000000 -242098304.000000,285908992.000000,768963000.000000,5144576.000000,282.300000,314682363904.000000,532622512128.000000,19479256.000000,562851.000000,71603267.000000,123429232.000000,1369619356672.000000,2711283655680.000000,49025136.000000,364117202.000000 -408027632.000000,429604864.000000,768963000.000000,5832704.000000,295.810000,314682372096.000000,532622512128.000000,19479256.000000,562851.000000,71603277.000000,123429260.000000,1369619356672.000000,2711283811328.000000,49025160.000000,364117211.000000 -302762976.000000,405700608.000000,841383368.000000,5341184.000000,355.340000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604261.000000,123430238.000000,1369619356672.000000,2711415743488.000000,49025720.000000,364119834.000000 -303820816.000000,406913024.000000,841383368.000000,5341184.000000,355.410000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604267.000000,123430252.000000,1369619356672.000000,2711415825408.000000,49025732.000000,364119838.000000 -304942192.000000,408166400.000000,841383368.000000,5341184.000000,355.480000,314688856064.000000,532622512128.000000,19479256.000000,562851.000000,71604333.000000,123430331.000000,1369619356672.000000,2711416722432.000000,49025756.000000,364119902.000000 -307280992.000000,353132544.000000,841383368.000000,6324224.000000,360.210000,314688860160.000000,532622512128.000000,19479256.000000,562851.000000,71604341.000000,123430336.000000,1369619356672.000000,2711416775680.000000,49025772.000000,364119909.000000 -276951840.000000,377872384.000000,841383368.000000,5439488.000000,425.580000,314734006272.000000,532622512128.000000,19479256.000000,562851.000000,71605006.000000,123431062.000000,1369619356672.000000,2711503111168.000000,49026172.000000,364121458.000000 -305161536.000000,404381696.000000,841383368.000000,5505024.000000,425.750000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605105.000000,123431218.000000,1369619356672.000000,2711506879488.000000,49026252.000000,364121533.000000 -306253904.000000,405716992.000000,841383368.000000,5537792.000000,425.830000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605161.000000,123431276.000000,1369619356672.000000,2711507444736.000000,49026284.000000,364121588.000000 -311480424.000000,369655808.000000,841383368.000000,6684672.000000,433.670000,314734567424.000000,532622512128.000000,19479256.000000,562851.000000,71605180.000000,123431305.000000,1369619356672.000000,2711507641344.000000,49026312.000000,364121607.000000 -355662328.000000,420216832.000000,871464392.000000,6946816.000000,499.790000,314761170944.000000,532622512128.000000,19479256.000000,562851.000000,71606250.000000,123432520.000000,1369619356672.000000,2711644345344.000000,49026968.000000,364124140.000000 -357481496.000000,421634048.000000,871464392.000000,7012352.000000,499.860000,314761285632.000000,532622512128.000000,19479256.000000,562851.000000,71606272.000000,123432572.000000,1369619356672.000000,2711644738560.000000,49026996.000000,364124154.000000 -358622664.000000,422887424.000000,871464392.000000,7012352.000000,499.930000,314761289728.000000,532622512128.000000,19479256.000000,562851.000000,71606322.000000,123432638.000000,1369619356672.000000,2711645307904.000000,49027024.000000,364124203.000000 -435911504.000000,463912960.000000,871464392.000000,7208960.000000,508.960000,314761293824.000000,532622512128.000000,19479256.000000,562851.000000,71606344.000000,123432648.000000,1369619356672.000000,2711645438976.000000,49027056.000000,364124221.000000 -462738048.000000,518053888.000000,889997768.000000,7503872.000000,571.800000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607147.000000,123433655.000000,1369619356672.000000,2711740048384.000000,49027604.000000,364125827.000000 -463871216.000000,519503872.000000,889997768.000000,7503872.000000,571.870000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607151.000000,123433668.000000,1369619356672.000000,2711740118016.000000,49027616.000000,364125830.000000 -465032480.000000,520642560.000000,889997768.000000,7503872.000000,571.940000,314756165632.000000,532622512128.000000,19479256.000000,562851.000000,71607214.000000,123433752.000000,1369619356672.000000,2711740892160.000000,49027640.000000,364125897.000000 -519785024.000000,544251904.000000,889997768.000000,7110656.000000,583.620000,314756169728.000000,532622512128.000000,19479256.000000,562851.000000,71607229.000000,123433763.000000,1369619356672.000000,2711740998656.000000,49027664.000000,364125909.000000 -350615368.000000,428613632.000000,944723416.000000,7962624.000000,646.920000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608441.000000,123435261.000000,1369619356672.000000,2711882605568.000000,49028440.000000,364128509.000000 -351709736.000000,429875200.000000,944723416.000000,7962624.000000,646.990000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608445.000000,123435274.000000,1369619356672.000000,2711882675200.000000,49028452.000000,364128512.000000 -352884328.000000,431095808.000000,944723416.000000,7962624.000000,647.060000,314782023680.000000,532622512128.000000,19479256.000000,562851.000000,71608501.000000,123435357.000000,1369619356672.000000,2711883600896.000000,49028484.000000,364128578.000000 -448501768.000000,489857024.000000,944723416.000000,7012352.000000,659.190000,314782031872.000000,532622512128.000000,19479256.000000,562851.000000,71608517.000000,123435380.000000,1369619356672.000000,2711883760640.000000,49028504.000000,364128593.000000 -458298752.000000,541818880.000000,987783656.000000,8323072.000000,722.270000,314762813440.000000,532622512128.000000,19479256.000000,562851.000000,71609452.000000,123436209.000000,1369619356672.000000,2712031224832.000000,49029068.000000,364131885.000000 -459417648.000000,543014912.000000,987783656.000000,8323072.000000,722.330000,314762817536.000000,532622512128.000000,19479256.000000,562851.000000,71609458.000000,123436224.000000,1369619356672.000000,2712031310848.000000,49029084.000000,364131889.000000 -460575584.000000,544374784.000000,987783656.000000,8323072.000000,722.400000,314762817536.000000,532622512128.000000,19479256.000000,562851.000000,71609512.000000,123436306.000000,1369619356672.000000,2712032007168.000000,49029108.000000,364131941.000000 -578158120.000000,605192192.000000,987783656.000000,8486912.000000,736.330000,314762825728.000000,532622512128.000000,19479256.000000,562851.000000,71609535.000000,123436322.000000,1369619356672.000000,2712032166912.000000,49029140.000000,364131964.000000 -552250168.000000,624828416.000000,1021862376.000000,8224768.000000,799.190000,314747748352.000000,532622512128.000000,19479256.000000,562851.000000,71610854.000000,123437665.000000,1369619356672.000000,2712208659456.000000,49029900.000000,364135964.000000 -553382072.000000,626049024.000000,1021862376.000000,8224768.000000,799.260000,314747748352.000000,532622512128.000000,19479256.000000,562851.000000,71610861.000000,123437687.000000,1369619356672.000000,2712208798720.000000,49029924.000000,364135972.000000 -554508440.000000,627564544.000000,1021862376.000000,8224768.000000,799.330000,314747752448.000000,532622512128.000000,19479256.000000,562851.000000,71610916.000000,123437760.000000,1369619356672.000000,2712209437696.000000,49029960.000000,364136030.000000 -514697272.000000,558940160.000000,1021862376.000000,8912896.000000,812.500000,314747760640.000000,532622512128.000000,19479256.000000,562851.000000,71610933.000000,123437778.000000,1369619356672.000000,2712209581056.000000,49029984.000000,364136045.000000 -389808504.000000,480747520.000000,1022517736.000000,8650752.000000,871.590000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611848.000000,123438964.000000,1369619356672.000000,2712312013824.000000,49030612.000000,364137889.000000 -390936616.000000,481910784.000000,1022517736.000000,8650752.000000,871.660000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611854.000000,123438980.000000,1369619356672.000000,2712312103936.000000,49030628.000000,364137894.000000 -392095584.000000,483172352.000000,1022517736.000000,8650752.000000,871.730000,314797207552.000000,532622512128.000000,19479256.000000,562851.000000,71611908.000000,123439052.000000,1369619356672.000000,2712312816640.000000,49030660.000000,364137955.000000 -546401608.000000,587390976.000000,1022517736.000000,8617984.000000,876.640000,314797211648.000000,532622512128.000000,19479256.000000,562851.000000,71611941.000000,123439082.000000,1369619356672.000000,2712313074688.000000,49030692.000000,364137989.000000 -436601888.000000,549380096.000000,1052732920.000000,9601024.000000,945.160000,314793103360.000000,532622512128.000000,19479256.000000,562851.000000,71612937.000000,123440208.000000,1369619356672.000000,2712437945344.000000,49031300.000000,364140485.000000 -447374704.000000,557989888.000000,1052732920.000000,9601024.000000,945.280000,314793078784.000000,532622512128.000000,19479256.000000,562851.000000,71613011.000000,123440336.000000,1369619356672.000000,2712439690240.000000,49031348.000000,364140534.000000 -447780976.000000,558415872.000000,1052732920.000000,9601024.000000,945.340000,314793082880.000000,532622512128.000000,19479256.000000,562851.000000,71613207.000000,123440431.000000,1369619356672.000000,2712441156608.000000,49031396.000000,364140682.000000 -439056616.000000,532365312.000000,1052732920.000000,9994240.000000,952.170000,314793082880.000000,532622512128.000000,19479256.000000,562851.000000,71613213.000000,123440456.000000,1369619356672.000000,2712441283584.000000,49031408.000000,364140687.000000 -593982144.000000,640008192.000000,1065709048.000000,10125312.000000,1017.430000,314813480960.000000,532622512128.000000,19479256.000000,562851.000000,71614302.000000,123442591.000000,1369619356672.000000,2712549970944.000000,49032372.000000,364142543.000000 -601819248.000000,646660096.000000,1065709048.000000,10125312.000000,1017.540000,314814439424.000000,532622512128.000000,19479256.000000,562851.000000,71614477.000000,123442750.000000,1369619356672.000000,2712552166400.000000,49032432.000000,364142665.000000 -603089008.000000,647929856.000000,1065709048.000000,10125312.000000,1017.610000,314814439424.000000,532622512128.000000,19479256.000000,562851.000000,71614538.000000,123442785.000000,1369619356672.000000,2712563323904.000000,49032476.000000,364142791.000000 -528318200.000000,586022912.000000,1065709048.000000,9338880.000000,1023.740000,314814443520.000000,532622512128.000000,19479256.000000,562851.000000,71614649.000000,123442820.000000,1369619356672.000000,2712564065280.000000,49032508.000000,364142876.000000 -630398168.000000,702529536.000000,1151433224.000000,10452992.000000,1091.190000,314837487616.000000,532622512128.000000,19479256.000000,562851.000000,71615594.000000,123443822.000000,1369619356672.000000,2712689730560.000000,49033100.000000,364145217.000000 -631545448.000000,703791104.000000,1151433224.000000,10452992.000000,1091.260000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615605.000000,123443856.000000,1369619356672.000000,2712689914880.000000,49033116.000000,364145224.000000 -632730024.000000,705069056.000000,1151433224.000000,10452992.000000,1091.330000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615661.000000,123443922.000000,1369619356672.000000,2712690697216.000000,49033140.000000,364145279.000000 -649311792.000000,683393024.000000,1151433224.000000,10878976.000000,1099.050000,314837565440.000000,532622512128.000000,19479256.000000,562851.000000,71615678.000000,123443946.000000,1369619356672.000000,2712690865152.000000,49033164.000000,364145295.000000 -683288800.000000,720412672.000000,1151564296.000000,10911744.000000,1163.530000,314868514816.000000,532622512128.000000,19479256.000000,562851.000000,71616515.000000,123445064.000000,1369619356672.000000,2712786048000.000000,49033760.000000,364146992.000000 -684492936.000000,721551360.000000,1151564296.000000,10977280.000000,1163.600000,314868514816.000000,532622512128.000000,19479256.000000,562851.000000,71616547.000000,123445118.000000,1369619356672.000000,2712786420736.000000,49033780.000000,364147026.000000 -685791256.000000,722837504.000000,1151564296.000000,10977280.000000,1163.670000,314868518912.000000,532622512128.000000,19479256.000000,562851.000000,71616579.000000,123445152.000000,1369619356672.000000,2712786781184.000000,49033808.000000,364147057.000000 -627911368.000000,679247872.000000,1151564296.000000,11206656.000000,1174.060000,314868506624.000000,532622512128.000000,19479256.000000,562851.000000,71616600.000000,123445181.000000,1369619356672.000000,2712786985984.000000,49033832.000000,364147075.000000 -561551184.000000,654450688.000000,1185711640.000000,10715136.000000,1241.060000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617895.000000,123446557.000000,1369619356672.000000,2712960173056.000000,49034612.000000,364150408.000000 -562652016.000000,655663104.000000,1185711640.000000,10715136.000000,1241.150000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617900.000000,123446571.000000,1369619356672.000000,2712960275456.000000,49034628.000000,364150412.000000 -563841024.000000,656867328.000000,1185711640.000000,10715136.000000,1241.220000,314841956352.000000,532622512128.000000,19479256.000000,562851.000000,71617961.000000,123446648.000000,1369619356672.000000,2712960840704.000000,49034652.000000,364150475.000000 -470494544.000000,616341504.000000,1185711640.000000,11468800.000000,1251.550000,314841968640.000000,532622512128.000000,19479256.000000,562851.000000,71617967.000000,123446657.000000,1369619356672.000000,2712960902144.000000,49034664.000000,364150479.000000 -622930280.000000,721223680.000000,1194821144.000000,11796480.000000,1312.850000,314880311296.000000,532622512128.000000,19479256.000000,562851.000000,71618732.000000,123447584.000000,1369619356672.000000,2713053627392.000000,49035220.000000,364152039.000000 -624079088.000000,722518016.000000,1194821144.000000,11796480.000000,1312.930000,314880319488.000000,532622512128.000000,19479256.000000,562851.000000,71618765.000000,123447635.000000,1369619356672.000000,2713054012416.000000,49035240.000000,364152082.000000 -625268816.000000,723640320.000000,1194821144.000000,11796480.000000,1312.990000,314880319488.000000,532622512128.000000,19479256.000000,562851.000000,71618816.000000,123447692.000000,1369619356672.000000,2713054589952.000000,49035276.000000,364152124.000000 -655298304.000000,714186752.000000,1194821144.000000,12025856.000000,1324.660000,314880323584.000000,532622512128.000000,19479256.000000,562851.000000,71618841.000000,123447708.000000,1369619356672.000000,2713054757888.000000,49035312.000000,364152146.000000 -829546272.000000,892215296.000000,1228637720.000000,12124160.000000,1387.030000,314889940992.000000,532622512128.000000,19479256.000000,562851.000000,71619902.000000,123448839.000000,1369619356672.000000,2713191506944.000000,49035916.000000,364154676.000000 -830827504.000000,893411328.000000,1228637720.000000,12124160.000000,1387.100000,314889945088.000000,532622512128.000000,19479256.000000,562851.000000,71619908.000000,123448859.000000,1369619356672.000000,2713191613440.000000,49035932.000000,364154683.000000 -831988896.000000,894803968.000000,1228637720.000000,12124160.000000,1387.170000,314889945088.000000,532622512128.000000,19479256.000000,562851.000000,71619963.000000,123448931.000000,1369619356672.000000,2713192252416.000000,49035956.000000,364154740.000000 -495933176.000000,672210944.000000,1228637720.000000,13303808.000000,1401.240000,314889936896.000000,532622512128.000000,19479256.000000,562851.000000,71619980.000000,123448951.000000,1369619356672.000000,2713192403968.000000,49035984.000000,364154757.000000 -416059432.000000,625598464.000000,1237026328.000000,11468800.000000,1462.930000,314884075520.000000,532622512128.000000,19479256.000000,562851.000000,71621087.000000,123450098.000000,1369619356672.000000,2713346245632.000000,49036588.000000,364158382.000000 -417113920.000000,626696192.000000,1237026328.000000,11468800.000000,1463.000000,314884075520.000000,532622512128.000000,19479256.000000,562851.000000,71621093.000000,123450113.000000,1369619356672.000000,2713346331648.000000,49036604.000000,364158387.000000 -418235696.000000,627818496.000000,1237026328.000000,11468800.000000,1463.070000,314884079616.000000,532622512128.000000,19479256.000000,562851.000000,71621154.000000,123450202.000000,1369619356672.000000,2713347179520.000000,49036640.000000,364158465.000000 -753567512.000000,786923520.000000,1237026328.000000,15040512.000000,1467.980000,314884079616.000000,532622512128.000000,19479256.000000,562851.000000,71621204.000000,123450214.000000,1369619356672.000000,2713347437568.000000,49036660.000000,364158499.000000 -808786128.000000,895328256.000000,1292997160.000000,13107200.000000,1535.790000,314906832896.000000,532622512128.000000,19479256.000000,562851.000000,71622238.000000,123451547.000000,1369619356672.000000,2713467433984.000000,49037288.000000,364160883.000000 -835308816.000000,919363584.000000,1292997160.000000,13107200.000000,1535.970000,314898255872.000000,532622512128.000000,19479256.000000,562851.000000,71622307.000000,123451661.000000,1369619356672.000000,2713470149632.000000,49037332.000000,364160969.000000 -836598072.000000,920616960.000000,1292997160.000000,13107200.000000,1536.040000,314898255872.000000,532622512128.000000,19479256.000000,562851.000000,71622340.000000,123451696.000000,1369619356672.000000,2713470501888.000000,49037364.000000,364161005.000000 -664427496.000000,736362496.000000,1292997160.000000,12255232.000000,1544.230000,314898264064.000000,532622512128.000000,19479256.000000,562851.000000,71622371.000000,123451717.000000,1369619356672.000000,2713470714880.000000,49037388.000000,364161034.000000 -827173408.000000,920330240.000000,1311130664.000000,13402112.000000,1608.980000,314921054208.000000,532622512128.000000,19479256.000000,562851.000000,71623284.000000,123452711.000000,1369619356672.000000,2713581626368.000000,49037864.000000,364162974.000000 -829019408.000000,922042368.000000,1311130664.000000,13402112.000000,1609.050000,314921164800.000000,532622512128.000000,19479256.000000,562851.000000,71623306.000000,123452777.000000,1369619356672.000000,2713582072832.000000,49037892.000000,364163004.000000 -830274224.000000,923328512.000000,1311130664.000000,13402112.000000,1609.130000,314921164800.000000,532622512128.000000,19479256.000000,562851.000000,71623407.000000,123452844.000000,1369619356672.000000,2713582932992.000000,49037924.000000,364163092.000000 -824181752.000000,864722944.000000,1311130664.000000,13762560.000000,1619.200000,314921168896.000000,532622512128.000000,19479256.000000,562851.000000,71623427.000000,123452867.000000,1369619356672.000000,2713583109120.000000,49037948.000000,364163112.000000 -525004136.000000,681893888.000000,1370771512.000000,13271040.000000,1680.790000,314955726848.000000,532622512128.000000,19479256.000000,562851.000000,71624233.000000,123453898.000000,1369619356672.000000,2713675129856.000000,49038528.000000,364164725.000000 -526188576.000000,683008000.000000,1370771512.000000,13271040.000000,1680.860000,314955730944.000000,532622512128.000000,19479256.000000,562851.000000,71624240.000000,123453916.000000,1369619356672.000000,2713675256832.000000,49038548.000000,364164731.000000 -526569144.000000,683294720.000000,1370771512.000000,13271040.000000,1680.930000,314955730944.000000,532622512128.000000,19479256.000000,562851.000000,71624299.000000,123454007.000000,1369619356672.000000,2713676112896.000000,49038576.000000,364164794.000000 -904246112.000000,939720704.000000,1370771512.000000,15302656.000000,1693.600000,314955739136.000000,532622512128.000000,19479256.000000,562851.000000,71624317.000000,123454024.000000,1369619356672.000000,2713676256256.000000,49038604.000000,364164811.000000 -849273232.000000,945029120.000000,1400200264.000000,14221312.000000,1756.160000,314954559488.000000,532622512128.000000,19479256.000000,562851.000000,71625582.000000,123456225.000000,1369619356672.000000,2713818780672.000000,49039656.000000,364167543.000000 -850474912.000000,946388992.000000,1400200264.000000,14221312.000000,1756.210000,314954559488.000000,532622512128.000000,19479256.000000,562851.000000,71625589.000000,123456241.000000,1369619356672.000000,2713818899456.000000,49039676.000000,364167548.000000 -851693352.000000,947666944.000000,1400200264.000000,14221312.000000,1756.280000,314954563584.000000,532622512128.000000,19479256.000000,562851.000000,71625853.000000,123456369.000000,1369619356672.000000,2713831318528.000000,49039736.000000,364167805.000000 -865553416.000000,916873216.000000,1400200264.000000,14581760.000000,1767.460000,314954563584.000000,532622512128.000000,19479256.000000,562851.000000,71625880.000000,123456400.000000,1369619356672.000000,2713831560192.000000,49039764.000000,364167834.000000 -981327984.000000,1092034560.000000,1422744648.000000,14680064.000000,1828.730000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626580.000000,123457162.000000,1369619356672.000000,2713920824320.000000,49040244.000000,364169456.000000 -982411776.000000,1093533696.000000,1422744648.000000,14680064.000000,1828.800000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626616.000000,123457209.000000,1369619356672.000000,2713921184768.000000,49040268.000000,364169492.000000 -982834784.000000,1094000640.000000,1422744648.000000,14680064.000000,1828.870000,314985230336.000000,532622512128.000000,19479256.000000,562851.000000,71626650.000000,123457249.000000,1369619356672.000000,2713921635328.000000,49040296.000000,364169528.000000 -790178184.000000,867696640.000000,1422744648.000000,14843904.000000,1842.390000,314985242624.000000,532622512128.000000,19479256.000000,562851.000000,71626668.000000,123457266.000000,1369619356672.000000,2713921778688.000000,49040316.000000,364169544.000000 -592460408.000000,767918080.000000,1422744648.000000,15073280.000000,1904.320000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627746.000000,123458555.000000,1369619356672.000000,2714050171904.000000,49040988.000000,364171844.000000 -593657432.000000,768999424.000000,1422744648.000000,15073280.000000,1904.380000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627781.000000,123458609.000000,1369619356672.000000,2714050565120.000000,49041008.000000,364171879.000000 -594812696.000000,770293760.000000,1422744648.000000,15073280.000000,1904.460000,315023949824.000000,532622512128.000000,19479256.000000,562851.000000,71627817.000000,123458643.000000,1369619356672.000000,2714050933760.000000,49041044.000000,364171913.000000 -622071528.000000,822345728.000000,1422744648.000000,15269888.000000,1919.250000,315023958016.000000,532622512128.000000,19479256.000000,562851.000000,71627827.000000,123458652.000000,1369619356672.000000,2714051011584.000000,49041072.000000,364171921.000000 -703657512.000000,856440832.000000,1494640728.000000,15532032.000000,1982.500000,314982649856.000000,532622512128.000000,19479256.000000,562851.000000,71628927.000000,123459744.000000,1369619356672.000000,2714217292800.000000,49041712.000000,364177557.000000 -704819704.000000,857808896.000000,1494640728.000000,15532032.000000,1982.560000,314982653952.000000,532622512128.000000,19479256.000000,562851.000000,71628933.000000,123459760.000000,1369619356672.000000,2714217382912.000000,49041732.000000,364177562.000000 -705195952.000000,858095616.000000,1494640728.000000,15532032.000000,1982.630000,314982653952.000000,532622512128.000000,19479256.000000,562851.000000,71628996.000000,123459839.000000,1369619356672.000000,2714218099712.000000,49041764.000000,364177623.000000 -608812656.000000,843268096.000000,1494640728.000000,15663104.000000,1997.590000,314982662144.000000,532622512128.000000,19479256.000000,562851.000000,71629018.000000,123459858.000000,1369619356672.000000,2714218267648.000000,49041792.000000,364177644.000000 -713688152.000000,900595712.000000,1494706264.000000,15302656.000000,2056.340000,315004256256.000000,532622512128.000000,19479256.000000,562851.000000,71630124.000000,123461129.000000,1369619356672.000000,2714344248320.000000,49042408.000000,364180114.000000 -714856248.000000,901750784.000000,1494706264.000000,15302656.000000,2056.400000,315004256256.000000,532622512128.000000,19479256.000000,562851.000000,71630131.000000,123461147.000000,1369619356672.000000,2714344371200.000000,49042432.000000,364180120.000000 -716007032.000000,902938624.000000,1494706264.000000,15302656.000000,2056.490000,315004260352.000000,532622512128.000000,19479256.000000,562851.000000,71630186.000000,123461214.000000,1369619356672.000000,2714344944640.000000,49042464.000000,364180174.000000 -985264080.000000,1043243008.000000,1494706264.000000,17694720.000000,2062.930000,315004264448.000000,532622512128.000000,19479256.000000,562851.000000,71630204.000000,123461236.000000,1369619356672.000000,2714345108480.000000,49042488.000000,364180190.000000 -849464448.000000,984743936.000000,1545892968.000000,15761408.000000,2132.510000,315016851456.000000,532622512128.000000,19479256.000000,562851.000000,71631226.000000,123462101.000000,1369619356672.000000,2714498618368.000000,49043080.000000,364183920.000000 -855159480.000000,987545600.000000,1545892968.000000,15761408.000000,2132.600000,315017502720.000000,532622512128.000000,19479256.000000,562851.000000,71631253.000000,123462161.000000,1369619356672.000000,2714499589120.000000,49043112.000000,364183940.000000 -856285272.000000,988782592.000000,1545892968.000000,15761408.000000,2132.670000,315017502720.000000,532622512128.000000,19479256.000000,562851.000000,71631310.000000,123462222.000000,1369619356672.000000,2714500146176.000000,49043148.000000,364183987.000000 -717144064.000000,882286592.000000,1545892968.000000,16777216.000000,2140.010000,315017515008.000000,532622512128.000000,19479256.000000,562851.000000,71631334.000000,123462249.000000,1369619356672.000000,2714500355072.000000,49043176.000000,364184010.000000 -1127243488.000000,1224171520.000000,1568175208.000000,16875520.000000,2205.560000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632209.000000,123463155.000000,1369619356672.000000,2714614715392.000000,49043716.000000,364186276.000000 -1128451776.000000,1225441280.000000,1568175208.000000,16875520.000000,2205.640000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632214.000000,123463168.000000,1369619356672.000000,2714614813696.000000,49043732.000000,364186280.000000 -1129683248.000000,1226711040.000000,1568175208.000000,16875520.000000,2205.700000,315014066176.000000,532622512128.000000,19479256.000000,562851.000000,71632269.000000,123463231.000000,1369619356672.000000,2714615526400.000000,49043764.000000,364186329.000000 -666545256.000000,882573312.000000,1568175208.000000,17072128.000000,2215.330000,315014070272.000000,532622512128.000000,19479256.000000,562851.000000,71632287.000000,123463251.000000,1369619356672.000000,2714615682048.000000,49043796.000000,364186345.000000 -672478616.000000,864477184.000000,1572369512.000000,16646144.000000,2280.760000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633263.000000,123464443.000000,1369619356672.000000,2714728408064.000000,49044420.000000,364188350.000000 -673620928.000000,865558528.000000,1572369512.000000,16646144.000000,2280.830000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633267.000000,123464456.000000,1369619356672.000000,2714728477696.000000,49044428.000000,364188353.000000 -674780304.000000,866672640.000000,1572369512.000000,16678912.000000,2280.910000,315053543424.000000,532622512128.000000,19479256.000000,562851.000000,71633329.000000,123464521.000000,1369619356672.000000,2714729325568.000000,49044452.000000,364188412.000000 -994481000.000000,1052557312.000000,1572369512.000000,17629184.000000,2291.030000,315053551616.000000,532622512128.000000,19479256.000000,562851.000000,71633347.000000,123464553.000000,1369619356672.000000,2714729530368.000000,49044480.000000,364188432.000000 -812500464.000000,985030656.000000,1623425144.000000,17432576.000000,2354.210000,315063529472.000000,532622512128.000000,19479256.000000,562851.000000,71634377.000000,123465823.000000,1369619356672.000000,2714846696448.000000,49045108.000000,364190796.000000 -813687280.000000,986193920.000000,1623425144.000000,17432576.000000,2354.280000,315063533568.000000,532622512128.000000,19479256.000000,562851.000000,71634381.000000,123465837.000000,1369619356672.000000,2714846770176.000000,49045120.000000,364190799.000000 -813987424.000000,986439680.000000,1623425144.000000,17432576.000000,2354.350000,315063533568.000000,532622512128.000000,19479256.000000,562851.000000,71634486.000000,123465906.000000,1369619356672.000000,2714847650816.000000,49045160.000000,364190893.000000 -711556800.000000,938696704.000000,1623425144.000000,17858560.000000,2367.070000,315063541760.000000,532622512128.000000,19479256.000000,562851.000000,71634507.000000,123465928.000000,1369619356672.000000,2714847826944.000000,49045188.000000,364190911.000000 -638610360.000000,910008320.000000,1691847816.000000,16842752.000000,2427.850000,315078250496.000000,532622512128.000000,19479256.000000,562851.000000,71635556.000000,123467201.000000,1369619356672.000000,2714963448832.000000,49045816.000000,364193043.000000 -639694808.000000,911040512.000000,1691847816.000000,16842752.000000,2427.930000,315078250496.000000,532622512128.000000,19479256.000000,562851.000000,71635561.000000,123467214.000000,1369619356672.000000,2714963547136.000000,49045832.000000,364193047.000000 -640850744.000000,912146432.000000,1691847816.000000,16842752.000000,2428.000000,315078254592.000000,532622512128.000000,19479256.000000,562851.000000,71635624.000000,123467286.000000,1369619356672.000000,2714964141056.000000,49045856.000000,364193101.000000 -806852704.000000,982786048.000000,1691847816.000000,18153472.000000,2442.590000,315078262784.000000,532622512128.000000,19479256.000000,562851.000000,71635643.000000,123467300.000000,1369619356672.000000,2714964276224.000000,49045876.000000,364193116.000000 -739564168.000000,915890176.000000,1693027464.000000,18284544.000000,2503.540000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71636821.000000,123469266.000000,1369619356672.000000,2715116307456.000000,49046880.000000,364197309.000000 -740773384.000000,917241856.000000,1693027464.000000,18284544.000000,2503.600000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71636827.000000,123469283.000000,1369619356672.000000,2715116401664.000000,49046892.000000,364197315.000000 -741962552.000000,918339584.000000,1693027464.000000,18284544.000000,2503.670000,315081498624.000000,532622512128.000000,19479256.000000,562851.000000,71637027.000000,123469399.000000,1369619356672.000000,2715128566784.000000,49046940.000000,364197549.000000 -699075344.000000,952786944.000000,1693027464.000000,18710528.000000,2516.390000,315081502720.000000,532622512128.000000,19479256.000000,562851.000000,71637115.000000,123469433.000000,1369619356672.000000,2715129111552.000000,49046972.000000,364197616.000000 -860078672.000000,1024188416.000000,1701481608.000000,18808832.000000,2577.450000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638125.000000,123470594.000000,1369619356672.000000,2715243373568.000000,49047604.000000,364199714.000000 -861296912.000000,1025490944.000000,1701481608.000000,18808832.000000,2577.520000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638129.000000,123470607.000000,1369619356672.000000,2715243443200.000000,49047612.000000,364199718.000000 -862507872.000000,1026842624.000000,1701481608.000000,18808832.000000,2577.600000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638183.000000,123470656.000000,1369619356672.000000,2715243918336.000000,49047640.000000,364199764.000000 -943308120.000000,1065197568.000000,1701481608.000000,19136512.000000,2592.130000,315101392896.000000,532622512128.000000,19479256.000000,562851.000000,71638195.000000,123470664.000000,1369619356672.000000,2715244000256.000000,49047664.000000,364199774.000000 -763948960.000000,1050279936.000000,1743755416.000000,17235968.000000,2657.570000,314927493120.000000,532622512128.000000,19479256.000000,562851.000000,71639484.000000,123471703.000000,1369619356672.000000,2715438994432.000000,49048344.000000,364208889.000000 -765004256.000000,1051582464.000000,1743755416.000000,17301504.000000,2657.630000,314927497216.000000,532622512128.000000,19479256.000000,562851.000000,71639490.000000,123471751.000000,1369619356672.000000,2715439215616.000000,49048372.000000,364208912.000000 -766091024.000000,1052852224.000000,1743755416.000000,17301504.000000,2657.700000,314927497216.000000,532622512128.000000,19479256.000000,562851.000000,71639534.000000,123471825.000000,1369619356672.000000,2715439768576.000000,49048400.000000,364208954.000000 -1117176560.000000,1169891328.000000,1743755416.000000,19431424.000000,2663.720000,314927501312.000000,532622512128.000000,19479256.000000,562851.000000,71639559.000000,123471847.000000,1369619356672.000000,2715439961088.000000,49048428.000000,364208977.000000 -978271928.000000,1221427200.000000,1816437928.000000,18350080.000000,2744.140000,314823303168.000000,532622512128.000000,19479256.000000,562851.000000,71640951.000000,123473013.000000,1369619356672.000000,2715655197696.000000,49049172.000000,364214936.000000 -1025150440.000000,1239318528.000000,1816437928.000000,19464192.000000,2744.430000,314826928128.000000,532622512128.000000,19479256.000000,562851.000000,71641000.000000,123473094.000000,1369619356672.000000,2715659293696.000000,49049220.000000,364214996.000000 -1026299384.000000,1240563712.000000,1816437928.000000,19464192.000000,2744.500000,314826932224.000000,532622512128.000000,19479256.000000,562851.000000,71641061.000000,123473169.000000,1369619356672.000000,2715659932672.000000,49049252.000000,364215050.000000 -1219366472.000000,1269211136.000000,1816437928.000000,20021248.000000,2759.620000,314826940416.000000,532622512128.000000,19479256.000000,562851.000000,71641086.000000,123473200.000000,1369619356672.000000,2715660162048.000000,49049272.000000,364215071.000000 -751104224.000000,987734016.000000,1817289896.000000,19988480.000000,2818.570000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642086.000000,123474389.000000,1369619356672.000000,2715779834880.000000,49049904.000000,364217289.000000 -752259784.000000,988815360.000000,1817289896.000000,19988480.000000,2818.620000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642092.000000,123474415.000000,1369619356672.000000,2715779965952.000000,49049924.000000,364217300.000000 -753416904.000000,989954048.000000,1817289896.000000,19988480.000000,2818.710000,314832457728.000000,532622512128.000000,19479256.000000,562851.000000,71642150.000000,123474479.000000,1369619356672.000000,2715780551680.000000,49049956.000000,364217349.000000 -1120425984.000000,1199390720.000000,1817289896.000000,21168128.000000,2825.740000,314832461824.000000,532622512128.000000,19479256.000000,562851.000000,71642171.000000,123474490.000000,1369619356672.000000,2715780682752.000000,49049988.000000,364217368.000000 -899177200.000000,1112252416.000000,1834329256.000000,20447232.000000,2892.840000,314850746368.000000,532622512128.000000,19479256.000000,562851.000000,71643039.000000,123475499.000000,1369619356672.000000,2715888276480.000000,49050516.000000,364219231.000000 -901678232.000000,1114161152.000000,1834329256.000000,20447232.000000,2892.930000,314850906112.000000,532622512128.000000,19479256.000000,562851.000000,71643061.000000,123475545.000000,1369619356672.000000,2715888690176.000000,49050544.000000,364219244.000000 -901985880.000000,1114718208.000000,1834329256.000000,20447232.000000,2892.990000,314850906112.000000,532622512128.000000,19479256.000000,562851.000000,71643117.000000,123475607.000000,1369619356672.000000,2715889304576.000000,49050580.000000,364219298.000000 -848858120.000000,1071980544.000000,1834329256.000000,21004288.000000,2902.110000,314850910208.000000,532622512128.000000,19479256.000000,562851.000000,71643139.000000,123475633.000000,1369619356672.000000,2715889501184.000000,49050604.000000,364219319.000000 +304956928.000000,352313344.000000,891505096.000000,2195456.000000,130.380000,63312080896.000000,132011507712.000000,114108.000000,29935.000000,896750.000000,922431.000000,3107416576.000000,88043596800.000000,448943.000000,6558766.000000 +231226472.000000,262070272.000000,891505096.000000,2195456.000000,182.710000,63356002304.000000,132011507712.000000,114276.000000,29935.000000,898547.000000,923917.000000,3129371136.000000,88160779264.000000,449522.000000,6562686.000000 +408990328.000000,434716672.000000,891505096.000000,2228224.000000,190.070000,63368527872.000000,132011507712.000000,114276.000000,29935.000000,898671.000000,924101.000000,3129371136.000000,88174476288.000000,449576.000000,6562871.000000 +410505752.000000,436174848.000000,891505096.000000,2228224.000000,190.120000,63368531968.000000,132011507712.000000,114283.000000,29935.000000,898784.000000,924154.000000,3129412096.000000,88175356928.000000,449586.000000,6562951.000000 +412024072.000000,437755904.000000,891505096.000000,2228224.000000,190.170000,63368540160.000000,132011507712.000000,114283.000000,29935.000000,898822.000000,924276.000000,3129412096.000000,88176045056.000000,449600.000000,6562997.000000 +208037424.000000,243539968.000000,891505096.000000,2195456.000000,223.080000,63383695360.000000,132011507712.000000,114284.000000,29935.000000,900046.000000,925651.000000,3129424384.000000,88319441920.000000,450208.000000,6570442.000000 +409157816.000000,437305344.000000,891505096.000000,2162688.000000,234.630000,63381020672.000000,132011507712.000000,114284.000000,29935.000000,900424.000000,926004.000000,3129424384.000000,88371100672.000000,450321.000000,6571231.000000 +410423256.000000,438910976.000000,891505096.000000,2162688.000000,234.690000,63381037056.000000,132011507712.000000,114284.000000,29935.000000,900431.000000,926061.000000,3129424384.000000,88371371008.000000,450334.000000,6571249.000000 +410973128.000000,439296000.000000,891505096.000000,2162688.000000,234.740000,63381041152.000000,132011507712.000000,114284.000000,29935.000000,900493.000000,926108.000000,3129424384.000000,88371874816.000000,450338.000000,6571305.000000 +411175896.000000,430006272.000000,891505096.000000,2326528.000000,274.890000,63392956416.000000,132011507712.000000,114284.000000,29935.000000,900875.000000,926464.000000,3129424384.000000,88394046464.000000,450450.000000,6571716.000000 +418392872.000000,440909824.000000,891505096.000000,2326528.000000,277.490000,63400587264.000000,132011507712.000000,114284.000000,29935.000000,900988.000000,926648.000000,3129424384.000000,88406891520.000000,450505.000000,6571892.000000 +419773416.000000,442236928.000000,891505096.000000,2326528.000000,277.550000,63400587264.000000,132011507712.000000,114284.000000,29935.000000,901129.000000,926681.000000,3129424384.000000,88407702528.000000,450515.000000,6571993.000000 +421152928.000000,443867136.000000,891505096.000000,2326528.000000,277.600000,63400591360.000000,132011507712.000000,114284.000000,29935.000000,901166.000000,926827.000000,3129424384.000000,88408452096.000000,450521.000000,6572033.000000 +223244104.000000,268509184.000000,891505096.000000,2260992.000000,314.080000,63415857152.000000,132011507712.000000,114291.000000,29935.000000,901476.000000,927285.000000,3129485824.000000,88434215936.000000,450662.000000,6572421.000000 +386948072.000000,416612352.000000,891505096.000000,2260992.000000,320.310000,63427706880.000000,132011507712.000000,114291.000000,29935.000000,901590.000000,927448.000000,3129485824.000000,88446999552.000000,450712.000000,6572589.000000 +388280160.000000,418308096.000000,891505096.000000,2260992.000000,320.360000,63427710976.000000,132011507712.000000,114291.000000,29935.000000,901690.000000,927478.000000,3129485824.000000,88447593472.000000,450718.000000,6572661.000000 +388893768.000000,418848768.000000,891505096.000000,2260992.000000,320.420000,63427715072.000000,132011507712.000000,114291.000000,29935.000000,901717.000000,927508.000000,3129485824.000000,88447826944.000000,450721.000000,6572689.000000 +276385360.000000,298213376.000000,891505096.000000,2293760.000000,356.210000,63455547392.000000,132011507712.000000,114291.000000,29935.000000,902104.000000,928007.000000,3129485824.000000,88491613184.000000,450887.000000,6573339.000000 +397554384.000000,416915456.000000,891505096.000000,2293760.000000,367.020000,63435976704.000000,132011507712.000000,114291.000000,29935.000000,902418.000000,928306.000000,3129485824.000000,88533154816.000000,451006.000000,6573995.000000 +398950392.000000,418791424.000000,891505096.000000,2293760.000000,367.070000,63435980800.000000,132011507712.000000,114291.000000,29935.000000,902495.000000,928344.000000,3129485824.000000,88533646336.000000,451011.000000,6574052.000000 +400392888.000000,420315136.000000,891505096.000000,2326528.000000,367.130000,63435984896.000000,132011507712.000000,114291.000000,29935.000000,902516.000000,928377.000000,3129485824.000000,88533867520.000000,451014.000000,6574071.000000 +392205808.000000,409747456.000000,891767240.000000,2359296.000000,406.220000,63447695360.000000,132011507712.000000,114291.000000,29935.000000,902862.000000,928928.000000,3129485824.000000,88557689856.000000,451155.000000,6574428.000000 +372634992.000000,398794752.000000,891767240.000000,2326528.000000,410.020000,63455338496.000000,132011507712.000000,114291.000000,29935.000000,902974.000000,929099.000000,3129485824.000000,88570514432.000000,451205.000000,6574606.000000 +374031912.000000,400465920.000000,891767240.000000,2326528.000000,410.070000,63455342592.000000,132011507712.000000,114291.000000,29935.000000,903016.000000,929128.000000,3129485824.000000,88570813440.000000,451206.000000,6574639.000000 +374705960.000000,401121280.000000,891767240.000000,2326528.000000,410.120000,63455346688.000000,132011507712.000000,114291.000000,29935.000000,903045.000000,929165.000000,3129485824.000000,88571083776.000000,451212.000000,6574671.000000 +271383544.000000,292659200.000000,891767240.000000,2293760.000000,445.490000,63471034368.000000,132011507712.000000,114291.000000,29935.000000,903330.000000,929633.000000,3129485824.000000,88594013184.000000,451340.000000,6575020.000000 +389958152.000000,414695424.000000,891767240.000000,2228224.000000,454.510000,63478394880.000000,132011507712.000000,114291.000000,29935.000000,903623.000000,929962.000000,3129485824.000000,88628444160.000000,451460.000000,6575522.000000 +391336840.000000,416325632.000000,891767240.000000,2228224.000000,454.570000,63478398976.000000,132011507712.000000,114291.000000,29935.000000,903720.000000,930021.000000,3129485824.000000,88629222400.000000,451469.000000,6575593.000000 +391931928.000000,417103872.000000,891767240.000000,2228224.000000,454.620000,63478403072.000000,132011507712.000000,114291.000000,29935.000000,903724.000000,930022.000000,3129485824.000000,88629242880.000000,451469.000000,6575596.000000 +250797888.000000,273629184.000000,891767240.000000,2359296.000000,487.270000,63509241856.000000,132011507712.000000,114291.000000,29935.000000,903913.000000,930320.000000,3129485824.000000,88649321472.000000,451549.000000,6575903.000000 +379917584.000000,404873216.000000,891767240.000000,2359296.000000,498.870000,63491731456.000000,132011507712.000000,114291.000000,29935.000000,904228.000000,930623.000000,3129485824.000000,88690715648.000000,451678.000000,6576522.000000 +381337416.000000,406380544.000000,891767240.000000,2359296.000000,498.920000,63491739648.000000,132011507712.000000,114291.000000,29935.000000,904264.000000,930651.000000,3129485824.000000,88691043328.000000,451680.000000,6576552.000000 +382674072.000000,407912448.000000,891767240.000000,2359296.000000,498.980000,63491743744.000000,132011507712.000000,114291.000000,29935.000000,904286.000000,930685.000000,3129485824.000000,88691272704.000000,451684.000000,6576572.000000 +387419728.000000,403865600.000000,891767240.000000,2359296.000000,537.410000,63398203392.000000,132011507712.000000,114300.000000,29935.000000,904592.000000,931135.000000,3129637376.000000,88719166464.000000,451847.000000,6577006.000000 +310743880.000000,339705856.000000,891767240.000000,2260992.000000,542.400000,63510138880.000000,132011507712.000000,114309.000000,29935.000000,905106.000000,931358.000000,3129780736.000000,88836533248.000000,452363.000000,6597231.000000 +312096792.000000,341188608.000000,891767240.000000,2260992.000000,542.450000,63510147072.000000,132011507712.000000,114309.000000,29935.000000,905298.000000,931398.000000,3129780736.000000,88837696512.000000,452375.000000,6597375.000000 +312572008.000000,341786624.000000,891767240.000000,2260992.000000,542.510000,63510151168.000000,132011507712.000000,114309.000000,29935.000000,905343.000000,931453.000000,3129780736.000000,88838106112.000000,452379.000000,6597425.000000 +370008352.000000,385966080.000000,891767240.000000,2392064.000000,576.670000,63526109184.000000,132011507712.000000,114309.000000,29935.000000,905613.000000,931892.000000,3129780736.000000,88860044288.000000,452477.000000,6597764.000000 +297477640.000000,322281472.000000,891767240.000000,2260992.000000,585.800000,63531696128.000000,132011507712.000000,114310.000000,29935.000000,905909.000000,932236.000000,3129784832.000000,88896052224.000000,452592.000000,6598299.000000 +298858936.000000,323870720.000000,891767240.000000,2293760.000000,585.850000,63531700224.000000,132011507712.000000,114310.000000,29935.000000,905947.000000,932275.000000,3129784832.000000,88896547840.000000,452595.000000,6598339.000000 +300201784.000000,325386240.000000,891767240.000000,2326528.000000,585.910000,63531704320.000000,132011507712.000000,114310.000000,29935.000000,905971.000000,932307.000000,3129784832.000000,88896777216.000000,452598.000000,6598362.000000 +228412872.000000,285589504.000000,892029384.000000,2293760.000000,632.720000,63522222080.000000,132011507712.000000,114310.000000,29935.000000,906664.000000,933009.000000,3129784832.000000,88988400640.000000,452877.000000,6600300.000000 +326778144.000000,356925440.000000,892029384.000000,2293760.000000,633.280000,63529590784.000000,132011507712.000000,114310.000000,29935.000000,906760.000000,933177.000000,3129784832.000000,88996719616.000000,452926.000000,6600423.000000 +328214448.000000,358277120.000000,892029384.000000,2293760.000000,633.330000,63529594880.000000,132011507712.000000,114310.000000,29935.000000,906805.000000,933240.000000,3129784832.000000,88997342208.000000,452929.000000,6600463.000000 +328814928.000000,358785024.000000,892029384.000000,2293760.000000,633.390000,63529594880.000000,132011507712.000000,114310.000000,29935.000000,906824.000000,933278.000000,3129784832.000000,88997575680.000000,452932.000000,6600479.000000 +306967784.000000,325386240.000000,892029384.000000,2359296.000000,674.080000,63526989824.000000,132011507712.000000,114310.000000,29935.000000,908096.000000,934851.000000,3129784832.000000,89170488320.000000,453610.000000,6612154.000000 +256065960.000000,293167104.000000,892029384.000000,2293760.000000,678.980000,63532638208.000000,132011507712.000000,114310.000000,29935.000000,908278.000000,935027.000000,3129784832.000000,89181756416.000000,453667.000000,6612351.000000 +257365096.000000,294584320.000000,892029384.000000,2293760.000000,679.030000,63532642304.000000,132011507712.000000,114310.000000,29935.000000,908343.000000,935087.000000,3129784832.000000,89182452736.000000,453671.000000,6612397.000000 +257913448.000000,295165952.000000,892029384.000000,2293760.000000,679.090000,63532650496.000000,132011507712.000000,114310.000000,29935.000000,908364.000000,935130.000000,3129784832.000000,89182714880.000000,453674.000000,6612416.000000 +244023184.000000,289947648.000000,892029384.000000,2326528.000000,715.110000,63584702464.000000,132011507712.000000,114310.000000,29935.000000,908541.000000,935402.000000,3129784832.000000,89202355200.000000,453744.000000,6612689.000000 +343033568.000000,367419392.000000,892029384.000000,2260992.000000,726.600000,63512010752.000000,132011507712.000000,114310.000000,29935.000000,909039.000000,935662.000000,3129784832.000000,89299131392.000000,454128.000000,6618516.000000 +344389104.000000,368910336.000000,892029384.000000,2260992.000000,726.660000,63512018944.000000,132011507712.000000,114310.000000,29935.000000,909082.000000,935694.000000,3129784832.000000,89299504128.000000,454131.000000,6618553.000000 +344983088.000000,369524736.000000,892029384.000000,2260992.000000,726.710000,63512018944.000000,132011507712.000000,114310.000000,29935.000000,909104.000000,935727.000000,3129784832.000000,89299729408.000000,454137.000000,6618578.000000 +425435408.000000,441245696.000000,892029384.000000,2359296.000000,768.170000,63521304576.000000,132011507712.000000,114319.000000,29935.000000,909431.000000,936189.000000,3129858560.000000,89333423104.000000,454285.000000,6619057.000000 +211751128.000000,242704384.000000,892029384.000000,2260992.000000,770.830000,63526207488.000000,132011507712.000000,114319.000000,29935.000000,909653.000000,936436.000000,3129858560.000000,89359457280.000000,454369.000000,6619413.000000 +213035304.000000,244072448.000000,892029384.000000,2260992.000000,770.890000,63526207488.000000,132011507712.000000,114319.000000,29935.000000,909739.000000,936473.000000,3129858560.000000,89360108544.000000,454373.000000,6619482.000000 +214295912.000000,245637120.000000,892029384.000000,2293760.000000,770.940000,63526211584.000000,132011507712.000000,114319.000000,29935.000000,909760.000000,936507.000000,3129858560.000000,89360333824.000000,454377.000000,6619503.000000 +315074768.000000,334110720.000000,892029384.000000,2359296.000000,808.180000,63534690304.000000,132011507712.000000,114319.000000,29935.000000,910200.000000,937127.000000,3129858560.000000,89396222976.000000,454571.000000,6620082.000000 +284700328.000000,319160320.000000,892029384.000000,2260992.000000,814.740000,63542231040.000000,132011507712.000000,114319.000000,29935.000000,910312.000000,937295.000000,3129858560.000000,89408973824.000000,454627.000000,6620240.000000 +285964976.000000,320569344.000000,892029384.000000,2260992.000000,814.790000,63542235136.000000,132011507712.000000,114319.000000,29935.000000,910344.000000,937333.000000,3129858560.000000,89409272832.000000,454629.000000,6620263.000000 +286545616.000000,321077248.000000,892029384.000000,2260992.000000,814.850000,63542235136.000000,132011507712.000000,114319.000000,29935.000000,910363.000000,937364.000000,3129858560.000000,89409477632.000000,454633.000000,6620284.000000 +376810880.000000,393887744.000000,892029384.000000,2359296.000000,848.190000,63558877184.000000,132011507712.000000,114319.000000,29935.000000,910643.000000,937784.000000,3129858560.000000,89432144896.000000,454761.000000,6620618.000000 +307697808.000000,334110720.000000,892029384.000000,2260992.000000,856.830000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910850.000000,938032.000000,3129858560.000000,89455156224.000000,454829.000000,6620940.000000 +309038000.000000,335486976.000000,892029384.000000,2260992.000000,856.880000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910850.000000,938032.000000,3129858560.000000,89455156224.000000,454829.000000,6620940.000000 +310551488.000000,337117184.000000,892029384.000000,2293760.000000,856.930000,63567491072.000000,132011507712.000000,114319.000000,29935.000000,910927.000000,938097.000000,3129858560.000000,89455823872.000000,454839.000000,6621008.000000 +434090184.000000,450969600.000000,892029384.000000,2293760.000000,899.740000,63568433152.000000,132011507712.000000,114319.000000,29935.000000,911402.000000,938642.000000,3129858560.000000,89511775232.000000,455043.000000,6621780.000000 +319961176.000000,349306880.000000,892029384.000000,2195456.000000,900.400000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911505.000000,938803.000000,3129858560.000000,89521241088.000000,455092.000000,6621914.000000 +321353192.000000,350920704.000000,892029384.000000,2195456.000000,900.460000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911589.000000,938841.000000,3129858560.000000,89521802240.000000,455098.000000,6621981.000000 +322672152.000000,352567296.000000,892029384.000000,2228224.000000,900.510000,63576969216.000000,132011507712.000000,114319.000000,29935.000000,911601.000000,938865.000000,3129858560.000000,89521949696.000000,455101.000000,6621992.000000 +239280152.000000,294674432.000000,892029384.000000,2260992.000000,942.170000,63586906112.000000,132011507712.000000,114319.000000,29935.000000,912015.000000,939414.000000,3129858560.000000,89562078208.000000,455262.000000,6622541.000000 +347575656.000000,376143872.000000,892029384.000000,2260992.000000,942.810000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912121.000000,939576.000000,3129858560.000000,89572891648.000000,455310.000000,6622685.000000 +348935136.000000,377700352.000000,892029384.000000,2260992.000000,942.870000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912159.000000,939619.000000,3129858560.000000,89573338112.000000,455313.000000,6622723.000000 +350310176.000000,379166720.000000,892029384.000000,2293760.000000,942.930000,63596789760.000000,132011507712.000000,114319.000000,29935.000000,912172.000000,939644.000000,3129858560.000000,89573493760.000000,455315.000000,6622736.000000 +401366528.000000,418070528.000000,892029384.000000,2293760.000000,981.380000,63609491456.000000,132011507712.000000,114319.000000,29935.000000,912456.000000,940011.000000,3129858560.000000,89596005376.000000,455422.000000,6623091.000000 +334867376.000000,361021440.000000,892029384.000000,2195456.000000,984.920000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912657.000000,940260.000000,3129862656.000000,89619520512.000000,455514.000000,6623432.000000 +336264608.000000,362430464.000000,892029384.000000,2195456.000000,984.970000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912736.000000,940324.000000,3129862656.000000,89620184064.000000,455523.000000,6623491.000000 +336786960.000000,363126784.000000,892029384.000000,2195456.000000,985.030000,63613825024.000000,132011507712.000000,114320.000000,29935.000000,912736.000000,940324.000000,3129862656.000000,89620184064.000000,455523.000000,6623491.000000 +351857352.000000,368885760.000000,892029384.000000,2326528.000000,1023.070000,63621074944.000000,132011507712.000000,114321.000000,29935.000000,913218.000000,940893.000000,3129866752.000000,89675226112.000000,455728.000000,6624431.000000 +391189520.000000,419880960.000000,892029384.000000,2293760.000000,1028.490000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913322.000000,941056.000000,3129866752.000000,89684982784.000000,455788.000000,6624587.000000 +392652096.000000,421306368.000000,892029384.000000,2293760.000000,1028.550000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913363.000000,941101.000000,3129866752.000000,89685429248.000000,455791.000000,6624622.000000 +394060712.000000,422887424.000000,892029384.000000,2293760.000000,1028.600000,63629893632.000000,132011507712.000000,114321.000000,29935.000000,913376.000000,941131.000000,3129866752.000000,89685605376.000000,455794.000000,6624635.000000 +351935208.000000,369205248.000000,892029384.000000,2260992.000000,1064.490000,63638695936.000000,132011507712.000000,114321.000000,29935.000000,913724.000000,941649.000000,3129866752.000000,89720781824.000000,455939.000000,6625138.000000 +293986560.000000,328458240.000000,892029384.000000,2195456.000000,1070.580000,63649800192.000000,132011507712.000000,114321.000000,29935.000000,913834.000000,941818.000000,3129866752.000000,89732848640.000000,455987.000000,6625288.000000 +295344848.000000,329850880.000000,892029384.000000,2195456.000000,1070.650000,63649800192.000000,132011507712.000000,114321.000000,29935.000000,913918.000000,941856.000000,3129866752.000000,89733426176.000000,455993.000000,6625356.000000 +295913760.000000,330416128.000000,892029384.000000,2195456.000000,1070.710000,63649792000.000000,132011507712.000000,114321.000000,29935.000000,913930.000000,941880.000000,3129866752.000000,89733573632.000000,455997.000000,6625367.000000 +240596848.000000,274628608.000000,892029384.000000,2293760.000000,1104.400000,63662354432.000000,132011507712.000000,114321.000000,29935.000000,914530.000000,943233.000000,3129866752.000000,89761709056.000000,456246.000000,6625944.000000 +248172880.000000,286908416.000000,892029384.000000,2260992.000000,1112.540000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915257.000000,943470.000000,3129866752.000000,89891703808.000000,456653.000000,6628579.000000 +249468200.000000,288161792.000000,892029384.000000,2260992.000000,1112.600000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915283.000000,943493.000000,3129866752.000000,89891970048.000000,456656.000000,6628604.000000 +249973640.000000,288710656.000000,892029384.000000,2260992.000000,1112.660000,63666212864.000000,132011507712.000000,114321.000000,29935.000000,915323.000000,943576.000000,3129866752.000000,89892473856.000000,456673.000000,6628650.000000 +302335280.000000,340049920.000000,892029384.000000,2326528.000000,1158.020000,63661408256.000000,132011507712.000000,114321.000000,29935.000000,915911.000000,944145.000000,3129866752.000000,89980664832.000000,456928.000000,6630695.000000 +429684360.000000,451485696.000000,892029384.000000,2326528.000000,1158.750000,63668527104.000000,132011507712.000000,114321.000000,29935.000000,916078.000000,944363.000000,3129866752.000000,89994157056.000000,457002.000000,6630879.000000 +209887464.000000,250134528.000000,892029384.000000,2228224.000000,1158.880000,63668523008.000000,132011507712.000000,114321.000000,29935.000000,916110.000000,944393.000000,3129866752.000000,89994513408.000000,457005.000000,6630913.000000 +211134312.000000,251265024.000000,892029384.000000,2228224.000000,1158.930000,63668523008.000000,132011507712.000000,114321.000000,29935.000000,916135.000000,944450.000000,3129866752.000000,89994849280.000000,457020.000000,6630949.000000 +257293280.000000,305676288.000000,892029384.000000,2392064.000000,1199.950000,63678472192.000000,132011507712.000000,114321.000000,29935.000000,916483.000000,944927.000000,3129866752.000000,90029890560.000000,457166.000000,6631455.000000 +256067288.000000,294313984.000000,892029384.000000,2293760.000000,1201.340000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916591.000000,945093.000000,3129866752.000000,90041318400.000000,457220.000000,6631607.000000 +257325048.000000,295747584.000000,892029384.000000,2293760.000000,1201.390000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916681.000000,945160.000000,3129866752.000000,90042117120.000000,457227.000000,6631686.000000 +258622120.000000,297304064.000000,892029384.000000,2326528.000000,1201.440000,63688945664.000000,132011507712.000000,114321.000000,29935.000000,916683.000000,945160.000000,3129866752.000000,90042125312.000000,457227.000000,6631688.000000 +299840152.000000,327041024.000000,892029384.000000,2326528.000000,1243.670000,63729782784.000000,132011507712.000000,114322.000000,29935.000000,917088.000000,945622.000000,3129870848.000000,90100435968.000000,457406.000000,6633057.000000 +326690152.000000,352878592.000000,892029384.000000,2260992.000000,1250.900000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917586.000000,945897.000000,3129870848.000000,90193492992.000000,457787.000000,6648071.000000 +327977568.000000,354525184.000000,892029384.000000,2260992.000000,1250.960000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917632.000000,945940.000000,3129870848.000000,90193857536.000000,457790.000000,6648115.000000 +328485952.000000,355270656.000000,892029384.000000,2293760.000000,1251.010000,63651233792.000000,132011507712.000000,114322.000000,29935.000000,917643.000000,945966.000000,3129870848.000000,90194009088.000000,457793.000000,6648125.000000 +255574816.000000,286892032.000000,892029384.000000,2326528.000000,1285.150000,63663865856.000000,132011507712.000000,114322.000000,29935.000000,917929.000000,946333.000000,3129870848.000000,90225728512.000000,457926.000000,6648603.000000 +278908352.000000,309059584.000000,892029384.000000,2260992.000000,1293.120000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918042.000000,946499.000000,3129870848.000000,90238942208.000000,457982.000000,6648775.000000 +280189448.000000,310575104.000000,892029384.000000,2260992.000000,1293.170000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918075.000000,946532.000000,3129870848.000000,90239261696.000000,457985.000000,6648811.000000 +281526944.000000,312008704.000000,892029384.000000,2260992.000000,1293.220000,63676129280.000000,132011507712.000000,114322.000000,29935.000000,918091.000000,946556.000000,3129870848.000000,90239425536.000000,457988.000000,6648829.000000 +321758296.000000,357138432.000000,892029384.000000,2326528.000000,1334.920000,63685742592.000000,132011507712.000000,114323.000000,29935.000000,918479.000000,947161.000000,3129874944.000000,90273950720.000000,458159.000000,6649357.000000 +404728504.000000,427229184.000000,892029384.000000,2260992.000000,1336.360000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918698.000000,947380.000000,3129874944.000000,90300517376.000000,458236.000000,6649703.000000 +406066824.000000,428736512.000000,892029384.000000,2260992.000000,1336.420000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918771.000000,947404.000000,3129874944.000000,90300996608.000000,458242.000000,6649754.000000 +407381896.000000,430374912.000000,892029384.000000,2293760.000000,1336.480000,63689998336.000000,132011507712.000000,114323.000000,29935.000000,918799.000000,947474.000000,3129874944.000000,90301398016.000000,458256.000000,6649792.000000 +248985056.000000,311910400.000000,892029384.000000,2359296.000000,1375.840000,63702720512.000000,132011507712.000000,114323.000000,29935.000000,919081.000000,947923.000000,3129874944.000000,90324880384.000000,458387.000000,6650139.000000 +367452424.000000,392192000.000000,892029384.000000,2260992.000000,1377.970000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919276.000000,948157.000000,3129879040.000000,90347056128.000000,458457.000000,6650434.000000 +368794632.000000,393863168.000000,892029384.000000,2260992.000000,1378.030000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919303.000000,948184.000000,3129879040.000000,90347453440.000000,458460.000000,6650460.000000 +369418240.000000,394444800.000000,892029384.000000,2260992.000000,1378.080000,63707570176.000000,132011507712.000000,114324.000000,29935.000000,919326.000000,948244.000000,3129879040.000000,90347793408.000000,458473.000000,6650495.000000 +273971304.000000,306610176.000000,892029384.000000,2326528.000000,1417.370000,63713411072.000000,132011507712.000000,114325.000000,29935.000000,919786.000000,948793.000000,3129883136.000000,90400738304.000000,458644.000000,6651183.000000 +360067328.000000,386875392.000000,892029384.000000,2293760.000000,1421.810000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919888.000000,948958.000000,3129883136.000000,90410511360.000000,458699.000000,6651319.000000 +361578960.000000,388513792.000000,892029384.000000,2293760.000000,1421.870000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919975.000000,948993.000000,3129883136.000000,90411031552.000000,458705.000000,6651386.000000 +363034648.000000,390062080.000000,892029384.000000,2293760.000000,1421.910000,63722237952.000000,132011507712.000000,114325.000000,29935.000000,919988.000000,949013.000000,3129883136.000000,90411166720.000000,458708.000000,6651399.000000 +395768520.000000,412950528.000000,892029384.000000,2326528.000000,1457.170000,63731904512.000000,132011507712.000000,114325.000000,29935.000000,920305.000000,949431.000000,3129883136.000000,90445265920.000000,458850.000000,6651877.000000 +321255112.000000,349708288.000000,892029384.000000,2260992.000000,1464.420000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920414.000000,949593.000000,3129887232.000000,90457062400.000000,458899.000000,6652017.000000 +322547288.000000,351518720.000000,892029384.000000,2260992.000000,1464.470000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920452.000000,949624.000000,3129887232.000000,90457345024.000000,458902.000000,6652053.000000 +323064648.000000,352239616.000000,892029384.000000,2260992.000000,1464.530000,63742767104.000000,132011507712.000000,114326.000000,29935.000000,920468.000000,949650.000000,3129887232.000000,90457517056.000000,458905.000000,6652069.000000 +324013144.000000,366125056.000000,892029384.000000,2326528.000000,1506.320000,63753170944.000000,132011507712.000000,114326.000000,29935.000000,920846.000000,950158.000000,3129887232.000000,90495532032.000000,459056.000000,6652606.000000 +436467688.000000,463945728.000000,892029384.000000,2326528.000000,1506.980000,63759286272.000000,132011507712.000000,114326.000000,29935.000000,920953.000000,950328.000000,3129887232.000000,90506857472.000000,459108.000000,6652752.000000 +437135216.000000,464617472.000000,892029384.000000,2326528.000000,1507.030000,63759286272.000000,132011507712.000000,114326.000000,29935.000000,920995.000000,950365.000000,3129887232.000000,90507242496.000000,459111.000000,6652791.000000 +438670808.000000,466272256.000000,892029384.000000,2326528.000000,1507.090000,63759282176.000000,132011507712.000000,114326.000000,29935.000000,921042.000000,950389.000000,3129887232.000000,90507545600.000000,459117.000000,6652827.000000 +234362352.000000,295239680.000000,892029384.000000,2260992.000000,1549.160000,63766687744.000000,132011507712.000000,114327.000000,29935.000000,922339.000000,951805.000000,3129891328.000000,90673024000.000000,459745.000000,6663341.000000 +338122280.000000,367788032.000000,892029384.000000,2260992.000000,1549.740000,63775334400.000000,132011507712.000000,114327.000000,29935.000000,922442.000000,951968.000000,3129891328.000000,90682612736.000000,459797.000000,6663477.000000 +339439464.000000,369344512.000000,892029384.000000,2260992.000000,1549.800000,63775334400.000000,132011507712.000000,114327.000000,29935.000000,922499.000000,952002.000000,3129891328.000000,90683042816.000000,459801.000000,6663520.000000 +340713832.000000,370966528.000000,892029384.000000,2260992.000000,1549.840000,63775338496.000000,132011507712.000000,114327.000000,29935.000000,922517.000000,952029.000000,3129891328.000000,90683227136.000000,459811.000000,6663544.000000 +396025560.000000,416022528.000000,892029384.000000,2326528.000000,1590.140000,63784239104.000000,132011507712.000000,114327.000000,29935.000000,922800.000000,952407.000000,3129891328.000000,90715843584.000000,459934.000000,6663998.000000 +308094120.000000,341278720.000000,892029384.000000,2228224.000000,1590.940000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922910.000000,952570.000000,3129891328.000000,90728451072.000000,459981.000000,6664163.000000 +309457096.000000,342745088.000000,892029384.000000,2228224.000000,1590.990000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922955.000000,952601.000000,3129891328.000000,90728762368.000000,459984.000000,6664195.000000 +310855800.000000,344244224.000000,892029384.000000,2260992.000000,1591.060000,63795916800.000000,132011507712.000000,114327.000000,29935.000000,922968.000000,952623.000000,3129891328.000000,90728905728.000000,459986.000000,6664206.000000 +372472568.000000,391905280.000000,892029384.000000,2293760.000000,1631.310000,63805771776.000000,132011507712.000000,114327.000000,29935.000000,923287.000000,953064.000000,3129891328.000000,90764184576.000000,460122.000000,6664716.000000 +434987728.000000,461324288.000000,892029384.000000,2260992.000000,1633.020000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923398.000000,953238.000000,3129891328.000000,90775735296.000000,460182.000000,6664876.000000 +436475552.000000,462733312.000000,892029384.000000,2260992.000000,1633.080000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923487.000000,953280.000000,3129891328.000000,90776366080.000000,460188.000000,6664943.000000 +437839616.000000,464396288.000000,892029384.000000,2293760.000000,1633.130000,63812075520.000000,132011507712.000000,114327.000000,29935.000000,923508.000000,953310.000000,3129891328.000000,90776574976.000000,460191.000000,6664964.000000 +302824216.000000,331333632.000000,892029384.000000,2293760.000000,1672.960000,63820083200.000000,132011507712.000000,114327.000000,29935.000000,924024.000000,953968.000000,3129891328.000000,90832346112.000000,460371.000000,6665869.000000 +428324424.000000,452198400.000000,892029384.000000,2326528.000000,1676.460000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924124.000000,954130.000000,3129891328.000000,90841803776.000000,460420.000000,6665997.000000 +429922648.000000,453746688.000000,892029384.000000,2326528.000000,1676.520000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924178.000000,954174.000000,3129891328.000000,90842242048.000000,460424.000000,6666041.000000 +431455768.000000,455213056.000000,892029384.000000,2326528.000000,1676.580000,63828606976.000000,132011507712.000000,114327.000000,29935.000000,924189.000000,954200.000000,3129891328.000000,90842393600.000000,460425.000000,6666051.000000 +410268600.000000,429359104.000000,892029384.000000,2326528.000000,1715.340000,63823212544.000000,132011507712.000000,114327.000000,29935.000000,924804.000000,954882.000000,3129891328.000000,90918894592.000000,460714.000000,6667936.000000 +378320944.000000,409804800.000000,892029384.000000,2326528.000000,1721.660000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924906.000000,955040.000000,3129891328.000000,90928917504.000000,460765.000000,6668073.000000 +379705464.000000,411254784.000000,892029384.000000,2326528.000000,1721.720000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924952.000000,955090.000000,3129891328.000000,90929359872.000000,460768.000000,6668113.000000 +380354824.000000,411803648.000000,892029384.000000,2326528.000000,1721.780000,63832317952.000000,132011507712.000000,114327.000000,29935.000000,924964.000000,955113.000000,3129891328.000000,90929503232.000000,460772.000000,6668126.000000 +431552576.000000,451264512.000000,892029384.000000,2326528.000000,1764.450000,63831511040.000000,132011507712.000000,114327.000000,29935.000000,925470.000000,955738.000000,3129891328.000000,90988424192.000000,460991.000000,6669715.000000 +366189440.000000,391520256.000000,892029384.000000,2228224.000000,1765.360000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925636.000000,955972.000000,3129891328.000000,91004967936.000000,461069.000000,6669952.000000 +367680600.000000,393011200.000000,892029384.000000,2228224.000000,1765.420000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925733.000000,956020.000000,3129891328.000000,91005725696.000000,461076.000000,6670027.000000 +369027928.000000,394485760.000000,892029384.000000,2260992.000000,1765.470000,63839870976.000000,132011507712.000000,114327.000000,29935.000000,925747.000000,956051.000000,3129891328.000000,91005910016.000000,461079.000000,6670040.000000 +333798608.000000,380764160.000000,892029384.000000,2293760.000000,1810.640000,63919005696.000000,132011507712.000000,114327.000000,29935.000000,925924.000000,956332.000000,3129891328.000000,91026258944.000000,461163.000000,6670305.000000 +242130464.000000,278732800.000000,892029384.000000,2228224.000000,1812.150000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926433.000000,956611.000000,3129891328.000000,91122723840.000000,461544.000000,6676148.000000 +243407136.000000,280125440.000000,892029384.000000,2228224.000000,1812.210000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926484.000000,956637.000000,3129891328.000000,91123096576.000000,461547.000000,6676190.000000 +243978768.000000,280608768.000000,892029384.000000,2228224.000000,1812.260000,63812001792.000000,132011507712.000000,114327.000000,29935.000000,926505.000000,956670.000000,3129891328.000000,91123317760.000000,461550.000000,6676209.000000 +371007152.000000,387579904.000000,892029384.000000,2293760.000000,1850.930000,63828652032.000000,132011507712.000000,114327.000000,29935.000000,926789.000000,957052.000000,3129891328.000000,91145636864.000000,461655.000000,6676573.000000 diff --git a/test/stress/istio/testdata/metrics/intermediate.csv b/test/stress/istio/testdata/metrics/intermediate.csv index 46304636d..973552fa0 100644 --- a/test/stress/istio/testdata/metrics/intermediate.csv +++ b/test/stress/istio/testdata/metrics/intermediate.csv @@ -1,9 +1,9 @@ -weighted_io,rate[5m],39.39,34.57,38.693333333333335,38.63333333333333,38.733333333333334,30.733333333333334,35.913333333333334,35.763333333333335,35.91,30.23,38.53,37.7,37.89,29.196666666666665,40.156666666666666,39.95666666666666,40.10666666666667,35.02,48.10333333333333,47.946666666666665,48.07666666666667,39.68333333333333,45.78333333333333,45.63666666666666,45.78,40.54,48.85,48.79,49.24333333333333,40.593333333333334,46.77,46.95666666666666,47.32666666666667,36.63666666666666,44.42666666666667,44.276666666666664,44.38333333333333,31.10333333333333,36.733333333333334,36.653333333333336,36.70666666666666,30.62,41.71333333333333,41.52333333333333,41.62,33.31333333333333,38.35,38,38.123333333333335,32.01,40.03666666666667,39.64,39.54666666666667,31.8,43.86,43.693333333333335,43.9,38.35666666666667,46.19,46.373333333333335,46.43333333333333,35.42,41.873333333333335,41.763333333333335,42.04333333333334,36.91,42.14333333333333,42.02333333333333,42.16,33.78333333333333,42.86666666666667,42.693333333333335,43.49333333333333,31.506666666666668,36.89666666666667,36.75666666666667,36.763333333333335,28.87,36.25,36.24666666666667,36.263333333333335,29.823333333333334,48.51,48.233333333333334,48.37,43.06333333333333,51.276666666666664,51.086666666666666,51.21,42.156666666666666,54.57333333333333,53.78333333333333,53.843333333333334,48.513333333333335,55.946666666666665,55.84,55.95,48.336666666666666,54.903333333333336,54.8,54.97,36.25,44.11333333333334,43.92,44.163333333333334,35.99,43.07666666666667,42.91,43.03666666666667,30.653333333333332,44.56333333333333,44.42666666666667,45.13,37.8,44.78,44.63,44.73,38.08,68.45333333333333,68.33333333333333,68.40666666666667,60.60333333333333,80.45666666666666,80.34333333333333,80.46333333333334,73.42666666666666,80.80666666666667,80.66333333333333,80.77666666666667,66.86333333333333,73.05333333333333,72.31666666666666,72.27333333333333,65.35 -io_time,rate[5m],12.573333333333334,10.933333333333334,12.48,12.48,12.466666666666667,10.306666666666667,11.96,11.88,11.866666666666667,8.813333333333333,11.186666666666667,11.053333333333333,11.08,9.28,11.12,11.093333333333334,11.12,9.893333333333333,12.16,12.133333333333333,12.16,10.053333333333333,12.053333333333333,12.013333333333334,12.013333333333334,10.293333333333333,12.28,12.36,12.44,9.893333333333333,13.066666666666666,13.16,13.24,11.466666666666667,13.386666666666667,13.36,13.333333333333334,10.88,12.786666666666667,12.733333333333333,12.746666666666666,10.733333333333333,13.28,13.226666666666667,13.2,11.213333333333333,12.906666666666666,12.813333333333333,12.893333333333333,9.8,11.613333333333333,11.52,11.493333333333334,9.613333333333333,11.573333333333334,11.546666666666667,11.586666666666666,9.666666666666666,11.693333333333333,11.746666666666666,11.773333333333333,9.253333333333334,10.786666666666667,10.8,10.866666666666667,9.093333333333334,10.96,10.906666666666666,10.88,8.96,12.413333333333334,12.4,12.506666666666666,10.586666666666666,12.133333333333333,12.093333333333334,12.12,10.093333333333334,12.186666666666667,12.146666666666667,12.186666666666667,10.693333333333333,12.733333333333333,12.693333333333333,12.72,10.88,12.866666666666667,12.853333333333333,12.866666666666667,9.44,11.346666666666666,11.253333333333334,11.28,9.773333333333333,11.493333333333334,11.453333333333333,11.493333333333334,9.36,11.373333333333333,11.28,11.266666666666667,9.226666666666667,11.253333333333334,11.186666666666667,11.226666666666667,9.266666666666667,11.28,11.226666666666667,11.226666666666667,9.32,12.56,12.48,12.546666666666667,10.853333333333333,12.906666666666666,12.826666666666666,12.813333333333333,10.813333333333333,13.053333333333333,13.066666666666666,13.066666666666666,11.066666666666666,13.506666666666666,13.533333333333333,13.546666666666667,11.52,13.573333333333334,13.56,13.6,10.36,12.08,12.013333333333334,12.026666666666667,10 -write_bytes,rate[5m],2118382.933333333,1811319.4666666666,2035165.8666666667,2034578.7733333334,2036039.68,1601085.44,1905322.6666666667,1903588.6933333334,1905377.28,1588893.0133333334,2037391.36,1996731.7333333334,1999298.56,1560057.1733333333,2051331.4133333333,2048628.0533333332,2050771.6266666667,1763519.1466666667,2339266.56,2337846.6133333333,2339321.1733333333,1884119.04,2224250.88,2222653.44,2224592.2133333334,1910087.68,2326091.0933333333,2329326.933333333,2333859.84,1862260.0533333332,2224319.1466666665,2228551.68,2265210.88,1776134.8266666667,2194732.3733333335,2193025.7066666665,2195101.013333333,1607352.32,1924164.2666666666,1923276.8,1924000.4266666668,1583240.5333333334,2160230.4,2158196.053333333,2159220.053333333,1743189.3333333333,2046457.1733333333,2042852.6933333334,2044354.56,1682623.1466666667,2131135.1466666665,2094298.4533333334,2093957.12,1675578.0266666666,2187769.1733333333,2185448.1066666665,2187714.56,1871298.56,2270044.16,2277894.8266666667,2278386.3466666667,1701806.08,2071169.7066666668,2070773.76,2073436.16,1764939.0933333333,2070391.4666666666,2068889.6,2071183.36,1615831.04,2090557.44,2088823.4666666666,2129715.2,1617715.2,1914975.5733333332,1913350.8266666667,1913992.5333333334,1514482.3466666667,1933407.5733333332,1933544.1066666667,1934062.9333333333,1564617.3866666667,2117399.8933333335,2114833.066666667,2116635.3066666666,1810459.3066666666,2229971.6266666665,2227527.68,2228961.28,1754426.0266666666,2265729.7066666665,2227568.64,2228619.946666667,1931769.1733333333,2311768.7466666666,2310594.56,2312492.3733333335,1885033.8133333332,2259476.48,2258479.7866666666,2261046.6133333333,1707458.56,2097711.7866666666,2095568.2133333334,2097943.8933333335,1678595.4133333333,2063592.1066666667,2062008.32,2063441.92,1552192.8533333333,2055727.7866666666,2054184.96,2094039.04,1714653.8666666667,2095199.5733333332,2093056,2094120.96,1718640.64,2368389.12,2366300.16,2367460.6933333334,1977548.8,2694758.4,2705476.2666666666,2707019.0933333333,2322377.3866666667,2720959.1466666665,2719416.32,2720918.1866666665,2214584.32,2572916.053333333,2533744.64,2533976.7466666666,2153758.72 -read_bytes,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -merged_write_count,rate[5m],24.006666666666668,21.19,23.033333333333335,22.973333333333333,23.156666666666666,19.013333333333332,21.99,21.813333333333333,21.98,15.903333333333334,20.466666666666665,20.14,20.323333333333334,17.14,19.856666666666666,19.643333333333334,19.9,17.533333333333335,21.49,21.37,21.516666666666666,17.526666666666667,21.30666666666667,21.14,21.346666666666668,18.09,21.8,21.946666666666665,22.226666666666667,17.316666666666666,24.39,24.643333333333334,24.683333333333334,22.036666666666665,25.326666666666668,25.166666666666668,25.333333333333332,20.936666666666667,24.59,24.526666666666667,24.58,20.723333333333333,25.256666666666668,25.063333333333333,25.22,21.496666666666666,24.16,24.013333333333332,24.12,17.05666666666667,20.296666666666667,20.246666666666666,20.37,17.096666666666668,20.80666666666667,20.636666666666667,20.85333333333333,17.166666666666668,21.43,21.696666666666665,21.716666666666665,17.2,20.466666666666665,20.43,20.623333333333335,17.61,20.876666666666665,20.746666666666666,20.996666666666666,17.283333333333335,24.553333333333335,24.366666666666667,24.726666666666667,21.006666666666668,23.496666666666666,23.356666666666666,23.45,19.063333333333333,22.98,23.043333333333333,23.086666666666666,19.803333333333335,23.223333333333333,23.053333333333335,23.24,19.866666666666667,24.043333333333333,23.8,23.966666666666665,16.703333333333333,19.533333333333335,19.30666666666667,19.406666666666666,16.956666666666667,19.82,19.73,19.883333333333333,15.653333333333334,19.446666666666665,19.376666666666665,19.563333333333333,16.03,20.21,19.993333333333332,20.16,15.996666666666666,20.18,20,20.166666666666668,17.33,23.683333333333334,23.536666666666665,23.833333333333332,20.926666666666666,24.753333333333334,24.586666666666666,24.683333333333334,20.736666666666668,24.156666666666666,24.1,24.24,20.08,23.92,23.96,24.136666666666667,19.996666666666666,23.916666666666668,23.763333333333332,23.93,17.413333333333334,20.72,20.486666666666668,20.58,16.796666666666667 -write_count,rate[5m],19.513333333333332,16.876666666666665,18.316666666666666,18.266666666666666,18.363333333333333,14.946666666666667,17.436666666666667,17.276666666666667,17.406666666666666,14.343333333333334,18.11,17.26,17.413333333333334,14.186666666666667,17.283333333333335,17.083333333333332,17.236666666666668,15.096666666666666,19.163333333333334,19,19.12,15.61,18.586666666666666,18.44,18.546666666666667,15.98,19.286666666666665,19.323333333333334,19.926666666666666,15.906666666666666,19.523333333333333,19.92,20.07,17.323333333333334,20.453333333333333,20.31,20.42,16.08,18.846666666666668,18.77,18.82,15.84,20.136666666666667,19.973333333333333,20.066666666666666,16.766666666666666,19.07,18.526666666666667,18.676666666666666,15.13,18.083333333333332,17.9,17.713333333333335,14.62,18.273333333333333,18.106666666666666,18.253333333333334,15.63,18.97,19.093333333333334,19.133333333333333,14.92,17.946666666666665,17.816666666666666,18.133333333333333,15.65,18.226666666666667,18.08,18.19333333333333,14.716666666666667,18.913333333333334,18.753333333333334,19.576666666666668,15.976666666666667,18.29,18.206666666666667,18.153333333333332,14.766666666666667,18.13,18.136666666666667,18.153333333333332,15.143333333333333,18.736666666666668,18.42,18.563333333333333,15.95,19.613333333333333,19.44,19.563333333333333,15.406666666666666,18.79,18,18.1,15.846666666666666,18.643333333333334,18.546666666666667,18.67,15.136666666666667,18.273333333333333,18.166666666666668,18.34,14.733333333333333,18.14666666666667,17.95,18.226666666666667,14.61,18.083333333333332,17.916666666666668,18.066666666666666,14.723333333333333,18.56,18.39,18.976666666666667,16.35333333333333,19.703333333333333,19.533333333333335,19.653333333333332,16.44,20.723333333333333,20.536666666666665,20.623333333333335,17.273333333333333,21.9,21.713333333333335,21.846666666666668,18.433333333333334,21.75,21.56,21.69,17.833333333333332,20.706666666666667,20.113333333333333,20.006666666666668,16.713333333333335 -merged_read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 -process_cpu_seconds_total,rate[5m],1.149333333333333,0.9892333333333333,1.2094000000000003,1.2094000000000003,1.1938666666666666,0.9931666666666666,1.201833333333333,1.2018666666666666,1.1878000000000002,1.0065,1.2155999999999998,1.2156333333333333,1.1708333333333332,1.0128333333333337,1.2228666666666665,1.2228333333333334,1.2073,1.0358333333333336,1.2448000000000001,1.2447666666666668,1.2188666666666668,1.0423666666666667,1.2391,1.2390999999999999,1.2092333333333334,1.0161333333333333,1.2443,1.2444666666666664,1.2057333333333333,1.0175,1.2348,1.2349333333333334,1.1947333333333332,1.0049000000000001,1.2295333333333334,1.2295333333333334,1.183333333333333,0.999533333333333,1.2142333333333333,1.2142333333333328,1.170566666666667,1.008233333333333,1.2313333333333332,1.2314000000000003,1.2152666666666667,1.0212999999999999,1.2252333333333332,1.2253,1.2027333333333334,1.0241000000000005,1.2316333333333334,1.231633333333333,1.2114333333333336,1.0334999999999999,1.2389000000000003,1.2389000000000003,1.2134,1.0148333333333335,1.2406333333333335,1.2409999999999999,1.2066000000000001,1.0105666666666668,1.2260999999999997,1.2260999999999997,1.191933333333334,1.0211666666666672,1.2261999999999997,1.226233333333333,1.1875666666666667,1.0218999999999998,1.2302000000000006,1.2301333333333333,1.1834666666666664,1.0151,1.2191,1.2191,1.2029666666666663,1.0220000000000005,1.227833333333333,1.2278000000000004,1.2007666666666668,1.0342333333333333,1.2448333333333335,1.2447666666666661,1.2114333333333336,1.0559999999999998,1.2516000000000007,1.2515666666666667,1.209633333333333,1.022566666666666,1.254333333333334,1.2543999999999997,1.2173666666666667,1.0376000000000007,1.2558666666666667,1.2559,1.2110333333333323,1.0367,1.2546000000000004,1.2545666666666664,1.205533333333333,1.028433333333334,1.2388333333333337,1.2388333333333337,1.1892,1.0357666666666667,1.238166666666666,1.2381333333333335,1.2169000000000005,1.0335999999999999,1.2364666666666668,1.2364333333333328,1.2121999999999995,1.0360999999999998,1.2393666666666665,1.2394000000000005,1.2075666666666667,1.0378999999999996,1.2558000000000007,1.2557333333333343,1.222233333333332,1.0316999999999992,1.2995333333333323,1.3002666666666665,1.2580999999999996,1.1058999999999999,1.3021333333333345,1.3020666666666663,1.253733333333333,1.0739999999999994,1.2974666666666674,1.2975333333333325,1.255333333333333,1.082200000000001 +weighted_io,rate[5m],50.95333333333333,37.95,38.52333333333333,38.85,38.806666666666665,14.096666666666666,12.63,14.243333333333334,14.293333333333333,12.933333333333334,13.37,15.096666666666666,15.063333333333333,13.836666666666666,14.723333333333333,81.9,82.28666666666666,80.28666666666666,79.23,80.82333333333334,80.89333333333333,79.78,85.64666666666666,85.94666666666667,85.97333333333333,84.86333333333333,122.10666666666667,122.52666666666667,122.67,121.71,120.55666666666667,139.88,139.93666666666667,138.57333333333332,72.75333333333333,73.46,73.52333333333333,72.46333333333334,72.61,73.00333333333333,73.00333333333333,66.61333333333333,67.31666666666666,68.25666666666666,68.20333333333333,29.513333333333332,31.43,31.723333333333333,31.883333333333333,31.01,13.416666666666666,13.773333333333333,13.816666666666666,12.263333333333334,12.26,13.166666666666666,13.293333333333333,11.363333333333333,13.97,14.413333333333334,14.46,13.39,13.993333333333334,14.493333333333334,14.493333333333334,11.956666666666667,13.433333333333334,21.993333333333332,22.04,20.363333333333333,26.7,27.186666666666667,27.256666666666668,26.19333333333333,26.743333333333332,27.053333333333335,27.316666666666666,24.19,28.233333333333334,78.16333333333333,78.26666666666667,76.62333333333333,77.71666666666667,78.06333333333333,78.14666666666666,76.28333333333333,69.26,70.33,70.34666666666666,63.656666666666666,64.2,65.07,65.03666666666666,63.46666666666667,65.25333333333333,65.44333333333333,65.66,61.14,12.686666666666667,13.006666666666666,13.093333333333334,11.553333333333333,12.77,13.136666666666667,13.206666666666667,11.566666666666666,45.46,45.74333333333333,45.76,44.68333333333333,45.21333333333333,45.67666666666667,45.666666666666664,43.41,44.656666666666666,44.96666666666667,45.14666666666667,43.623333333333335,46.17333333333333,46.48,46.57333333333333,44.81666666666667,50.61333333333334,50.94,50.95333333333333,15.95,20.793333333333333,21.44,21.61,20.14,20.473333333333333,39.843333333333334,39.946666666666665,38.31,38.99 +io_time,rate[5m],6.8933333333333335,4.973333333333334,5.263333333333334,5.3966666666666665,5.3533333333333335,3.3466666666666667,3.3966666666666665,3.7533333333333334,3.77,3.3966666666666665,3.48,3.8766666666666665,3.8633333333333333,3.4066666666666667,3.783333333333333,5.483333333333333,5.513333333333334,4.973333333333334,4.903333333333333,5.27,5.27,4.81,5.573333333333333,5.733333333333333,5.723333333333334,5.306666666666667,7.166666666666667,7.326666666666667,7.34,7.083333333333333,6.886666666666667,8.16,8.156666666666666,7.633333333333334,6.406666666666666,6.6466666666666665,6.6466666666666665,6.333333333333333,6.596666666666667,6.773333333333333,6.77,5.8533333333333335,6.116666666666666,6.333333333333333,6.323333333333333,4.096666666666667,4.586666666666667,4.736666666666666,4.746666666666667,4.523333333333333,3.78,3.93,3.92,3.433333333333333,3.51,3.8033333333333332,3.82,3.1733333333333333,3.67,3.8633333333333333,3.86,3.4433333333333334,3.7,3.86,3.8466666666666667,3.18,3.8466666666666667,5.183333333333334,5.183333333333334,4.703333333333333,5.3933333333333335,5.63,5.633333333333334,5.326666666666667,5.506666666666667,5.656666666666666,5.68,4.996666666666667,5.3933333333333335,6.653333333333333,6.653333333333333,6.18,6.463333333333333,6.63,6.626666666666667,5.806666666666667,5.02,5.266666666666667,5.23,4.426666666666667,4.616666666666666,4.84,4.8,4.3566666666666665,4.746666666666667,4.906666666666666,4.926666666666667,4.34,3.5433333333333334,3.6966666666666668,3.6966666666666668,3.263333333333333,3.58,3.743333333333333,3.743333333333333,3.1933333333333334,5.03,5.183333333333334,5.15,4.746666666666667,4.923333333333333,5.07,5.036666666666667,4.473333333333334,4.743333333333333,4.923333333333333,4.933333333333334,4.47,4.906666666666666,5.06,5.0633333333333335,4.5633333333333335,5.3533333333333335,5.513333333333334,5.503333333333333,3.4233333333333333,3.98,4.226666666666667,4.216666666666667,3.816666666666667,3.94,5.2,5.203333333333333,4.76,4.91 +write_bytes,rate[5m],1633498.4533333334,1243627.52,1277378.56,1317191.68,1315894.6133333333,838806.1866666666,743041.7066666667,856910.5066666667,857825.28,783988.0533333333,808099.84,943377.0666666667,941970.7733333333,856855.8933333333,907223.04,1296465.92,1299565.2266666666,1154976.4266666668,1089631.5733333332,1208019.6266666667,1208934.4,1130291.2,1392954.0266666666,1419687.2533333334,1420861.44,1345208.32,1806813.8666666667,1841780.0533333332,1844032.8533333333,1777978.0266666666,1705465.1733333333,2026960.2133333334,2027438.08,1935209.8133333332,1656299.52,1739202.56,1740008.1066666667,1667631.7866666666,1667235.84,1708086.6133333333,1708318.72,1403589.9733333334,1451417.6,1526046.72,1525268.48,951118.5066666667,1100062.72,1129294.5066666666,1130291.2,1065314.9866666666,876489.3866666667,911291.7333333333,912029.0133333333,800235.52,788493.6533333333,864706.56,866167.4666666667,746536.96,887507.6266666667,919033.1733333333,919838.72,844868.2666666667,885418.6666666666,925641.3866666667,925341.0133333333,739328,801559.8933333333,1233005.2266666666,1233401.1733333333,1101318.8266666667,1359243.9466666668,1402729.8133333332,1403398.8266666667,1329479.68,1367900.16,1403781.12,1406443.52,1222997.3333333333,1384843.9466666668,1693545.8133333332,1694173.8666666667,1577424.2133333334,1642932.9066666667,1685053.44,1685626.88,1592388.2666666666,1274156.3733333333,1361824.4266666668,1361742.5066666666,1069110.6133333333,1102411.0933333333,1175142.4,1175347.2,1059676.16,1198066.3466666667,1227980.8,1229687.4666666666,1035769.1733333333,839243.0933333334,877349.5466666666,877786.4533333334,772628.48,855299.4133333333,891985.92,892723.2,778649.6,1241688.7466666666,1272053.76,1272149.3333333333,1194489.1733333333,1229291.52,1269992.1066666667,1269896.5333333334,1093891.4133333333,1178910.72,1215679.1466666667,1217331.2,1104363.52,1250945.7066666668,1281529.1733333333,1282416.64,1156205.2266666666,1373457.0666666667,1405583.36,1406047.5733333332,854930.7733333333,1019371.52,1073083.7333333334,1074995.2,966888.1066666667,992692.9066666666,1313204.9066666667,1313969.4933333334,1197110.6133333333,1233005.2266666666 +read_bytes,rate[5m],73564.16,382.29333333333335,382.29333333333335,245.76,245.76,204.8,204.8,204.8,204.8,204.8,204.8,204.8,204.8,0,505.17333333333335,983.04,983.04,983.04,983.04,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,996.6933333333334,491.52,259.41333333333336,259.41333333333336,259.41333333333336,259.41333333333336,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,245.76,0,0,13.653333333333334,13.653333333333334,13.653333333333334,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,13.653333333333334,13.653333333333334,13.653333333333334,0,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,40.96,40.96,40.96,54.61333333333334,54.61333333333334,54.61333333333334,40.96,40.96,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,40.96,54.61333333333334,54.61333333333334,54.61333333333334,54.61333333333334,40.96,40.96,40.96,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,27.30666666666667,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,0,0,0,0,0,0,0,0,0,0 +merged_write_count,rate[5m],19.71,14.866666666666667,16.09,16.483333333333334,16.173333333333332,11.713333333333333,12.096666666666666,13.003333333333334,13.043333333333333,11.86,12.24,13.14,12.746666666666666,11.333333333333334,12.29,12.933333333333334,12.966666666666667,11.486666666666666,11.953333333333333,12.973333333333333,12.993333333333334,11.263333333333334,13.033333333333333,13.496666666666666,13.583333333333334,12.15,16.296666666666667,16.686666666666667,16.883333333333333,16.033333333333335,15.93,16.703333333333333,16.696666666666665,15.306666666666667,16.10333333333333,16.793333333333333,16.733333333333334,15.383333333333333,16.303333333333335,16.733333333333334,16.753333333333334,14.516666666666667,15.356666666666667,15.973333333333333,15.846666666666666,10.82,12.05,12.386666666666667,12.37,11.543333333333333,12.506666666666666,12.94,12.973333333333333,11.516666666666667,11.916666666666666,12.623333333333333,12.723333333333333,10.656666666666666,11.993333333333334,12.41,12.456666666666667,11.156666666666666,12.056666666666667,12.62,12.53,10.793333333333333,14.766666666666667,15.43,15.426666666666666,13.873333333333333,15.23,15.813333333333333,15.83,14.796666666666667,15.556666666666667,15.896666666666667,16.12,14.223333333333333,15.22,15.986666666666666,16.03,14.39,15.05,15.476666666666667,15.506666666666666,11.076666666666666,12.303333333333333,12.956666666666667,12.76,11.096666666666666,11.866666666666667,12.546666666666667,12.446666666666667,11.056666666666667,12.333333333333334,12.66,12.776666666666667,11.303333333333333,11.78,12.176666666666666,12.193333333333333,11.056666666666667,12.196666666666667,12.653333333333334,12.696666666666667,10.76,14.75,15.213333333333333,15.093333333333334,13.686666666666667,14.166666666666666,14.62,14.523333333333333,12.766666666666667,13.686666666666667,14.15,14.223333333333333,12.93,14.583333333333334,15.02,15.08,13.473333333333333,15.18,15.583333333333334,15.67,11.026666666666667,12.566666666666666,13.233333333333333,13.303333333333333,12.146666666666667,12.54,13.366666666666667,13.38,12.02,12.713333333333333 +write_count,rate[5m],19.15,13.23,13.97,13.966666666666667,13.98,9.996666666666666,9.686666666666667,10.64,10.756666666666666,9.496666666666666,9.75,10.33,10.326666666666666,9.366666666666667,10.006666666666666,11.386666666666667,11.936666666666667,10.796666666666667,10.65,11.38,11.436666666666667,10.363333333333333,12.3,12.48,12.533333333333333,11.646666666666667,14.91,15.193333333333333,15.396666666666667,14.836666666666666,14.376666666666667,15.916666666666666,15.986666666666666,15.04,14.416666666666666,14.516666666666667,14.653333333333334,13.823333333333334,14.303333333333333,14.55,14.576666666666666,12.33,12.943333333333333,13.483333333333333,13.42,9.436666666666667,10.413333333333334,10.54,10.75,10.2,9.92,10.13,10.183333333333334,9.136666666666667,9.343333333333334,9.726666666666667,9.92,8.453333333333333,9.686666666666667,9.926666666666666,10,9.11,9.58,9.946666666666667,9.97,8.426666666666666,10.083333333333334,12.226666666666667,12.273333333333333,11.026666666666667,12.633333333333333,13.063333333333333,13.126666666666667,12.263333333333334,12.753333333333334,12.85,13.15,11.55,12.553333333333333,14.076666666666666,14.186666666666667,13.063333333333333,13.65,13.746666666666666,13.816666666666666,11.87,10.74,11.383333333333333,11.493333333333334,9.626666666666667,10.01,10.553333333333333,10.56,9.476666666666667,10.65,10.69,10.973333333333333,9.666666666666666,9.063333333333333,9.273333333333333,9.363333333333333,8.463333333333333,9.346666666666666,9.593333333333334,9.68,8.543333333333333,12.136666666666667,12.236666666666666,12.333333333333334,11.453333333333333,11.746666666666666,12.023333333333333,12.096666666666666,10.606666666666667,11.33,11.41,11.663333333333334,10.676666666666666,12.033333333333333,12.24,12.366666666666667,11.143333333333333,12.836666666666666,13.036666666666667,13.033333333333333,8.75,10.093333333333334,10.456666666666667,10.72,9.823333333333334,10.046666666666667,11.593333333333334,11.72,10.726666666666667,11.303333333333333 +merged_read_count,rate[5m],0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0 +read_count,rate[5m],0.61,0.05,0.05,0.02666666666666667,0.02666666666666667,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0.023333333333333334,0,0.03,0.06,0.06,0.06,0.06,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.06333333333333334,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03333333333333333,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0.03,0,0,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.01,0.01,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.013333333333333334,0.01,0.01,0.01,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.006666666666666667,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0,0,0,0,0,0,0,0,0,0 +process_cpu_seconds_total,rate[5m],0.7889666666666667,0.6147333333333332,0.7205000000000001,0.7329999999999999,0.733,0.6234666666666666,0.7028666666666668,0.7327333333333333,0.7327666666666666,0.5991000000000001,0.6992666666666666,0.7377333333333334,0.7377333333333334,0.6163333333333334,0.7236666666666666,0.7401333333333332,0.7401000000000001,0.621,0.6988333333333333,0.7290999999999999,0.7290666666666668,0.5989666666666664,0.7423333333333335,0.7440333333333332,0.7440333333333334,0.6263333333333333,0.7319000000000002,0.7480333333333334,0.7480333333333332,0.6394000000000002,0.7208,0.7589333333333333,0.7589333333333331,0.6310000000000002,0.7525666666666666,0.7612666666666666,0.7612666666666666,0.647566666666667,0.7412666666666666,0.7629666666666666,0.7629333333333334,0.6071,0.7163666666666669,0.745,0.7449666666666667,0.6094999999999997,0.7358666666666667,0.7379,0.7379,0.6179999999999999,0.7185666666666665,0.7204999999999999,0.7205333333333332,0.5825333333333333,0.7018333333333332,0.7134333333333333,0.7134333333333333,0.5895,0.6944333333333335,0.7123333333333335,0.7123333333333332,0.6013666666666662,0.6921999999999999,0.7123333333333332,0.7124000000000005,0.5699000000000001,0.6800000000000004,0.7069333333333331,0.7069666666666664,0.5683000000000004,0.7173666666666668,0.7196,0.7198333333333339,0.5918333333333335,0.716766666666667,0.721233333333333,0.7212000000000004,0.5945666666666667,0.7172666666666668,0.7411666666666671,0.7412000000000004,0.6217333333333332,0.7152333333333338,0.741566666666666,0.7415333333333335,0.6293999999999997,0.7412666666666671,0.7458666666666667,0.7458666666666667,0.5948666666666668,0.723633333333333,0.7302999999999997,0.7303333333333331,0.5937666666666662,0.7201,0.7347333333333328,0.7347666666666661,0.5941333333333334,0.6875666666666667,0.7115333333333335,0.7115333333333335,0.597933333333333,0.7106666666666668,0.7126999999999998,0.7126999999999998,0.5738999999999995,0.7093333333333339,0.7110666666666664,0.7110666666666664,0.58,0.7072333333333336,0.7097000000000002,0.7097000000000002,0.5789666666666669,0.6983333333333334,0.7038333333333336,0.7038999999999995,0.5865333333333335,0.6951333333333332,0.7066333333333333,0.7066333333333333,0.5675333333333333,0.694533333333333,0.7154333333333337,0.7154333333333337,0.5753999999999997,0.7157000000000001,0.7185333333333331,0.7186000000000005,0.5844333333333331,0.7323333333333335,0.7372000000000003,0.737166666666667,0.6031666666666669,0.7263666666666669 diff --git a/test/stress/istio/testdata/metrics/result.csv b/test/stress/istio/testdata/metrics/result.csv index 0c459e727..0a64f2904 100644 --- a/test/stress/istio/testdata/metrics/result.csv +++ b/test/stress/istio/testdata/metrics/result.csv @@ -1,16 +1,16 @@ Metric Name, Min, Max, Mean, Median, P90, P95, P98, P99 -go_memstats_alloc_bytes, 194574488.000000, 1219366472.000000, 636633944.000000, 638610360.000000, 903115996.000000, 1009815720.000000, 1123834736.000000, 1129067512.000000 -go_memstats_heap_inuse_bytes, 270991360.000000, 1269211136.000000, 752114105.725490, 736362496.000000, 1068589056.000000, 1184641024.000000, 1226076160.000000, 1239941120.000000 -go_memstats_sys_bytes, 677992856.000000, 1834329256.000000, 1277992721.307189, 1237026328.000000, 1722618512.000000, 1816863912.000000, 1825809576.000000, 1834329256.000000 -go_memstats_stack_inuse_bytes, 3866624.000000, 21168128.000000, 12528512.836601, 13107200.000000, 18759680.000000, 19988480.000000, 20447232.000000, 20725760.000000 -process_cpu_seconds_total, 0.989233, 1.302133, 1.177522, 1.215433, 1.254483, 1.257000, 1.299900, 1.301167 -disk_used, 314598531072.000000, 315101392896.000000, 314877542801.568604, 314868518912.000000, 315058540544.000000, 315079880704.000000, 315091447808.000000, 315101392896.000000 -disk_total, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000, 532622512128.000000 -read_count, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 +go_memstats_alloc_bytes, 208037424.000000, 438670808.000000, 336485138.909091, 337454620.000000, 415208472.000000, 431504172.000000, 436471620.000000, 437487416.000000 +go_memstats_heap_inuse_bytes, 242704384.000000, 466272256.000000, 365532412.675325, 367603712.000000, 440102912.000000, 452972544.000000, 463339520.000000, 464506880.000000 +go_memstats_sys_bytes, 891505096.000000, 892029384.000000, 891923845.506493, 892029384.000000, 892029384.000000, 892029384.000000, 892029384.000000, 892029384.000000 +go_memstats_stack_inuse_bytes, 2162688.000000, 2392064.000000, 2280354.909091, 2293760.000000, 2326528.000000, 2359296.000000, 2359296.000000, 2375680.000000 +process_cpu_seconds_total, 0.567533, 0.788967, 0.693620, 0.713433, 0.744500, 0.755750, 0.762100, 0.762950 +disk_used, 63312080896.000000, 63919005696.000000, 63615498718.753250, 63617449984.000000, 63812075520.000000, 63830081536.000000, 63836094464.000000, 63839870976.000000 +disk_total, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000, 132011507712.000000 +read_count, 0.000000, 0.610000, 0.023926, 0.010000, 0.063333, 0.063333, 0.063333, 0.063333 merged_read_count, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 -write_count, 14.186667, 21.900000, 18.114826, 18.260000, 20.125000, 20.715000, 21.731667, 21.798333 -merged_write_count, 15.653333, 25.333333, 21.324801, 21.073333, 24.566667, 24.740000, 25.238333, 25.291667 -read_bytes, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000 -write_bytes, 1514482.346667, 2720959.146667, 2074918.769353, 2089690.453333, 2338556.586667, 2553446.400000, 2713217.706667, 2720167.253333 -io_time, 8.813333, 13.600000, 11.646070, 11.653333, 13.113333, 13.373333, 13.553333, 13.566667 -weighted_io, 28.870000, 80.806667, 45.567090, 42.973333, 66.106667, 73.240000, 80.563333, 80.720000 +write_count, 8.426667, 19.150000, 11.598420, 11.383333, 14.340000, 14.873333, 15.656667, 15.951667 +merged_write_count, 10.656667, 19.710000, 13.701605, 13.043333, 16.111667, 16.700000, 16.773333, 16.838333 +read_bytes, 0.000000, 73564.160000, 756.596938, 40.960000, 996.693333, 996.693333, 996.693333, 996.693333 +write_bytes, 739328.000000, 2027438.080000, 1224245.045728, 1222997.333333, 1689586.346667, 1758993.066667, 1889621.333333, 1981085.013333 +io_time, 3.173333, 8.160000, 4.977457, 4.926667, 6.646667, 6.988333, 7.486667, 7.895000 +weighted_io, 11.363333, 139.936667, 46.949284, 43.623333, 81.396667, 121.133333, 130.621667, 139.226667