This is an automated email from the ASF dual-hosted git repository.

wusheng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/skywalking-banyandb.git


The following commit(s) were added to refs/heads/main by this push:
     new 30ef8c7d Add sequential writer for merging (#372)
30ef8c7d is described below

commit 30ef8c7d4a6d9b2e981140cf43fc15609fe0d71e
Author: Gao Hongtao <hanahm...@gmail.com>
AuthorDate: Wed Jan 17 17:36:54 2024 +0800

    Add sequential writer for merging (#372)
---
 banyand/internal/storage/index.go                  |  18 +-
 banyand/internal/storage/index_test.go             |   4 +-
 banyand/internal/storage/tsdb.go                   |  15 +-
 banyand/measure/block.go                           |  15 +
 banyand/measure/block_reader.go                    |   7 +-
 banyand/measure/block_test.go                      |  11 +-
 banyand/measure/block_writer.go                    |  16 +-
 banyand/measure/column_test.go                     |   3 +-
 banyand/measure/merger.go                          |   9 +-
 banyand/measure/metadata.go                        |  13 +-
 banyand/measure/primary_metadata.go                |   4 +-
 pkg/bytes/buffer.go                                |  14 +-
 pkg/fs/file_system.go                              |  24 +-
 pkg/fs/local_file_system.go                        |  82 +++++-
 pkg/fs/local_file_system_test.go                   |   3 +-
 pkg/index/index.go                                 |   8 +-
 pkg/index/inverted/inverted.go                     |  32 ++-
 pkg/index/inverted/inverted_series_test.go         |   8 +-
 pkg/index/lsm/lsm.go                               |   2 +-
 .../query_ondisk/query_ondisk_suite_test.go        |   1 +
 test/stress/cases/istio/report.md                  |  76 ++---
 test/stress/cases/istio/testdata/metrics/data.csv  | 312 ++++++++++-----------
 .../cases/istio/testdata/metrics/intermediate.csv  |  18 +-
 .../stress/cases/istio/testdata/metrics/result.csv |  28 +-
 24 files changed, 442 insertions(+), 281 deletions(-)

diff --git a/banyand/internal/storage/index.go 
b/banyand/internal/storage/index.go
index 11d35f9e..bf50a0c8 100644
--- a/banyand/internal/storage/index.go
+++ b/banyand/internal/storage/index.go
@@ -46,14 +46,15 @@ type seriesIndex struct {
        l     *logger.Logger
 }
 
-func newSeriesIndex(ctx context.Context, root string) (*seriesIndex, error) {
+func newSeriesIndex(ctx context.Context, root string, flushTimeoutSeconds 
int64) (*seriesIndex, error) {
        si := &seriesIndex{
                l: logger.Fetch(ctx, "series_index"),
        }
        var err error
        if si.store, err = inverted.NewStore(inverted.StoreOpts{
-               Path:   path.Join(root, "idx"),
-               Logger: si.l,
+               Path:         path.Join(root, "idx"),
+               Logger:       si.l,
+               BatchWaitSec: flushTimeoutSeconds,
        }); err != nil {
                return nil, err
        }
@@ -61,7 +62,16 @@ func newSeriesIndex(ctx context.Context, root string) 
(*seriesIndex, error) {
 }
 
 func (s *seriesIndex) Write(docs index.Documents) error {
-       return s.store.Batch(docs)
+       applied := make(chan struct{})
+       err := s.store.Batch(index.Batch{
+               Documents: docs,
+               Applied:   applied,
+       })
+       if err != nil {
+               return err
+       }
+       <-applied
+       return nil
 }
 
 var rangeOpts = index.RangeOpts{}
diff --git a/banyand/internal/storage/index_test.go 
b/banyand/internal/storage/index_test.go
index f1fd0a42..4f26c67f 100644
--- a/banyand/internal/storage/index_test.go
+++ b/banyand/internal/storage/index_test.go
@@ -38,7 +38,7 @@ var testSeriesPool pbv1.SeriesPool
 func TestSeriesIndex_Primary(t *testing.T) {
        ctx := context.Background()
        path, fn := setUp(require.New(t))
-       si, err := newSeriesIndex(ctx, path)
+       si, err := newSeriesIndex(ctx, path, 0)
        require.NoError(t, err)
        defer func() {
                require.NoError(t, si.Close())
@@ -69,7 +69,7 @@ func TestSeriesIndex_Primary(t *testing.T) {
        require.NoError(t, si.Write(docs))
        // Restart the index
        require.NoError(t, si.Close())
-       si, err = newSeriesIndex(ctx, path)
+       si, err = newSeriesIndex(ctx, path, 0)
        require.NoError(t, err)
        tests := []struct {
                name         string
diff --git a/banyand/internal/storage/tsdb.go b/banyand/internal/storage/tsdb.go
index 16b5fb09..2166d355 100644
--- a/banyand/internal/storage/tsdb.go
+++ b/banyand/internal/storage/tsdb.go
@@ -44,12 +44,13 @@ const (
 
 // TSDBOpts wraps options to create a tsdb.
 type TSDBOpts[T TSTable, O any] struct {
-       Option          O
-       TSTableCreator  TSTableCreator[T, O]
-       Location        string
-       SegmentInterval IntervalRule
-       TTL             IntervalRule
-       ShardNum        uint32
+       Option                         O
+       TSTableCreator                 TSTableCreator[T, O]
+       Location                       string
+       SegmentInterval                IntervalRule
+       TTL                            IntervalRule
+       ShardNum                       uint32
+       SeriesIndexFlushTimeoutSeconds int64
 }
 
 type (
@@ -92,7 +93,7 @@ func OpenTSDB[T TSTable, O any](ctx context.Context, opts 
TSDBOpts[T, O]) (TSDB[
        p := common.GetPosition(ctx)
        location := filepath.Clean(opts.Location)
        lfs.MkdirIfNotExist(location, dirPerm)
-       si, err := newSeriesIndex(ctx, location)
+       si, err := newSeriesIndex(ctx, location, 
opts.SeriesIndexFlushTimeoutSeconds)
        if err != nil {
                return nil, errors.Wrap(errOpenDatabase, 
errors.WithMessage(err, "create series index failed").Error())
        }
diff --git a/banyand/measure/block.go b/banyand/measure/block.go
index 7da65663..7d29a76d 100644
--- a/banyand/measure/block.go
+++ b/banyand/measure/block.go
@@ -748,3 +748,18 @@ func (bi *blockPointer) reset() {
        bi.block.reset()
        bi.bm = blockMetadata{}
 }
+
+func generateBlockPointer() *blockPointer {
+       v := blockPointerPool.Get()
+       if v == nil {
+               return &blockPointer{}
+       }
+       return v.(*blockPointer)
+}
+
+func releaseBlockPointer(bi *blockPointer) {
+       bi.reset()
+       blockPointerPool.Put(bi)
+}
+
+var blockPointerPool sync.Pool
diff --git a/banyand/measure/block_reader.go b/banyand/measure/block_reader.go
index 219d0a94..eb221eef 100644
--- a/banyand/measure/block_reader.go
+++ b/banyand/measure/block_reader.go
@@ -30,13 +30,16 @@ import (
 )
 
 type seqReader struct {
-       sr        io.Reader
+       sr        fs.SeqReader
        r         fs.Reader
        bytesRead uint64
 }
 
 func (sr *seqReader) reset() {
        sr.r = nil
+       if sr.sr != nil {
+               fs.MustClose(sr.sr)
+       }
        sr.sr = nil
        sr.bytesRead = 0
 }
@@ -47,7 +50,7 @@ func (sr *seqReader) Path() string {
 
 func (sr *seqReader) init(r fs.Reader) {
        sr.reset()
-       sr.sr = r.StreamRead()
+       sr.sr = r.SequentialRead()
        sr.r = r
 }
 
diff --git a/banyand/measure/block_test.go b/banyand/measure/block_test.go
index b51e33c2..2431b735 100644
--- a/banyand/measure/block_test.go
+++ b/banyand/measure/block_test.go
@@ -264,7 +264,9 @@ func Test_mustWriteAndReadTimestamps(t *testing.T) {
                        }()
                        tm := &timestampsMetadata{}
                        b := &bytes.Buffer{}
-                       mustWriteTimestampsTo(tm, tt.args, &writer{w: b})
+                       w := new(writer)
+                       w.init(b)
+                       mustWriteTimestampsTo(tm, tt.args, w)
                        timestamps := mustReadTimestampsFrom(nil, tm, 
len(tt.args), b)
                        if !reflect.DeepEqual(timestamps, tt.args) {
                                t.Errorf("mustReadTimestampsFrom() = %v, want 
%v", timestamps, tt.args)
@@ -349,14 +351,17 @@ func Test_marshalAndUnmarshalTagFamily(t *testing.T) {
 
 func Test_marshalAndUnmarshalBlock(t *testing.T) {
        timestampBuffer, fieldBuffer := &bytes.Buffer{}, &bytes.Buffer{}
+       timestampWriter, fieldWriter := &writer{}, &writer{}
+       timestampWriter.init(timestampBuffer)
+       fieldWriter.init(fieldBuffer)
        ww := &writers{
                mustCreateTagFamilyWriters: func(name string) (fs.Writer, 
fs.Writer) {
                        return &bytes.Buffer{}, &bytes.Buffer{}
                },
                tagFamilyMetadataWriters: make(map[string]*writer),
                tagFamilyWriters:         make(map[string]*writer),
-               timestampsWriter:         writer{w: timestampBuffer},
-               fieldValuesWriter:        writer{w: fieldBuffer},
+               timestampsWriter:         *timestampWriter,
+               fieldValuesWriter:        *fieldWriter,
        }
        p := &part{
                primary:     &bytes.Buffer{},
diff --git a/banyand/measure/block_writer.go b/banyand/measure/block_writer.go
index 35d8d3ca..8f97c066 100644
--- a/banyand/measure/block_writer.go
+++ b/banyand/measure/block_writer.go
@@ -28,12 +28,14 @@ import (
 )
 
 type writer struct {
+       sw           fs.SeqWriter
        w            fs.Writer
        bytesWritten uint64
 }
 
 func (w *writer) reset() {
        w.w = nil
+       w.sw = nil
        w.bytesWritten = 0
 }
 
@@ -41,14 +43,16 @@ func (w *writer) init(wc fs.Writer) {
        w.reset()
 
        w.w = wc
+       w.sw = wc.SequentialWrite()
 }
 
 func (w *writer) MustWrite(data []byte) {
-       fs.MustWriteData(w.w, data)
+       fs.MustWriteData(w.sw, data)
        w.bytesWritten += uint64(len(data))
 }
 
 func (w *writer) MustClose() {
+       fs.MustClose(w.sw)
        fs.MustClose(w.w)
        w.reset()
 }
@@ -115,12 +119,10 @@ func (sw *writers) 
getColumnMetadataWriterAndColumnWriter(columnName string) (*w
                return chw, cw
        }
        hw, w := sw.mustCreateTagFamilyWriters(columnName)
-       chw = &writer{
-               w: hw,
-       }
-       cw = &writer{
-               w: w,
-       }
+       chw = new(writer)
+       chw.init(hw)
+       cw = new(writer)
+       cw.init(w)
        sw.tagFamilyMetadataWriters[columnName] = chw
        sw.tagFamilyWriters[columnName] = cw
        return chw, cw
diff --git a/banyand/measure/column_test.go b/banyand/measure/column_test.go
index ecf7752f..502d9d4e 100644
--- a/banyand/measure/column_test.go
+++ b/banyand/measure/column_test.go
@@ -64,7 +64,8 @@ func TestColumn_mustWriteTo_mustReadValues(t *testing.T) {
        cm := &columnMetadata{}
 
        buf := &bytes.Buffer{}
-       w := &writer{w: buf}
+       w := &writer{}
+       w.init(buf)
        original.mustWriteTo(cm, w)
        assert.Equal(t, w.bytesWritten, cm.size)
        assert.Equal(t, uint64(len(buf.Buf)), cm.size)
diff --git a/banyand/measure/merger.go b/banyand/measure/merger.go
index 1f03ca1e..efaa99a9 100644
--- a/banyand/measure/merger.go
+++ b/banyand/measure/merger.go
@@ -239,7 +239,8 @@ var errClosed = fmt.Errorf("the merger is closed")
 
 func mergeBlocks(closeCh <-chan struct{}, bw *blockWriter, br *blockReader) 
(*partMetadata, error) {
        pendingBlockIsEmpty := true
-       pendingBlock := &blockPointer{}
+       pendingBlock := generateBlockPointer()
+       defer releaseBlockPointer(pendingBlock)
        var tmpBlock, tmpBlock2 *blockPointer
        var decoder *encoding.BytesBlockDecoder
        getDecoder := func() *encoding.BytesBlockDecoder {
@@ -280,7 +281,8 @@ func mergeBlocks(closeCh <-chan struct{}, bw *blockWriter, 
br *blockReader) (*pa
                }
 
                if tmpBlock == nil {
-                       tmpBlock = &blockPointer{}
+                       tmpBlock = generateBlockPointer()
+                       defer releaseBlockPointer(tmpBlock)
                }
                tmpBlock.reset()
                tmpBlock.bm.seriesID = b.bm.seriesID
@@ -304,7 +306,8 @@ func mergeBlocks(closeCh <-chan struct{}, bw *blockWriter, 
br *blockReader) (*pa
                l := tmpBlock.idx
                tmpBlock.idx = 0
                if tmpBlock2 == nil {
-                       tmpBlock2 = &blockPointer{}
+                       tmpBlock2 = generateBlockPointer()
+                       defer releaseBlockPointer(tmpBlock2)
                }
                tmpBlock2.reset()
                tmpBlock2.append(tmpBlock, l)
diff --git a/banyand/measure/metadata.go b/banyand/measure/metadata.go
index 14c940d6..26e7aa40 100644
--- a/banyand/measure/metadata.go
+++ b/banyand/measure/metadata.go
@@ -346,12 +346,13 @@ func (s *supplier) ResourceSchema(md *commonv1.Metadata) 
(resourceSchema.Resourc
 
 func (s *supplier) OpenDB(groupSchema *commonv1.Group) (io.Closer, error) {
        opts := storage.TSDBOpts[*tsTable, option]{
-               ShardNum:        groupSchema.ResourceOpts.ShardNum,
-               Location:        path.Join(s.path, groupSchema.Metadata.Name),
-               TSTableCreator:  newTSTable,
-               SegmentInterval: 
storage.MustToIntervalRule(groupSchema.ResourceOpts.SegmentInterval),
-               TTL:             
storage.MustToIntervalRule(groupSchema.ResourceOpts.Ttl),
-               Option:          s.option,
+               ShardNum:                       
groupSchema.ResourceOpts.ShardNum,
+               Location:                       path.Join(s.path, 
groupSchema.Metadata.Name),
+               TSTableCreator:                 newTSTable,
+               SegmentInterval:                
storage.MustToIntervalRule(groupSchema.ResourceOpts.SegmentInterval),
+               TTL:                            
storage.MustToIntervalRule(groupSchema.ResourceOpts.Ttl),
+               Option:                         s.option,
+               SeriesIndexFlushTimeoutSeconds: 
s.option.flushTimeout.Nanoseconds() / int64(time.Second),
        }
        name := groupSchema.Metadata.Name
        return storage.OpenTSDB(
diff --git a/banyand/measure/primary_metadata.go 
b/banyand/measure/primary_metadata.go
index ed3e4a82..612ec252 100644
--- a/banyand/measure/primary_metadata.go
+++ b/banyand/measure/primary_metadata.go
@@ -83,10 +83,12 @@ func (ph *primaryBlockMetadata) unmarshal(src []byte) 
([]byte, error) {
 }
 
 func mustReadPrimaryBlockMetadata(dst []primaryBlockMetadata, r fs.Reader) 
[]primaryBlockMetadata {
-       data, err := io.ReadAll(r.StreamRead())
+       sr := r.SequentialRead()
+       data, err := io.ReadAll(sr)
        if err != nil {
                logger.Panicf("cannot read primaryBlockMetadata entries from 
%s: %s", r.Path(), err)
        }
+       fs.MustClose(sr)
 
        bb := bigValuePool.Generate()
        bb.Buf, err = zstd.Decompress(bb.Buf[:0], data)
diff --git a/pkg/bytes/buffer.go b/pkg/bytes/buffer.go
index be4db36b..c55c2a33 100644
--- a/pkg/bytes/buffer.go
+++ b/pkg/bytes/buffer.go
@@ -62,11 +62,16 @@ func (b *Buffer) Read(offset int64, buffer []byte) (int, 
error) {
        return n, err
 }
 
-// StreamRead implements fs.Reader.
-func (b *Buffer) StreamRead() io.Reader {
+// SequentialRead implements fs.Reader.
+func (b *Buffer) SequentialRead() fs.SeqReader {
        return &reader{bb: b}
 }
 
+// SequentialWrite implements fs.Writer.
+func (b *Buffer) SequentialWrite() fs.SeqWriter {
+       return b
+}
+
 // Reset resets the buffer.
 func (b *Buffer) Reset() {
        b.Buf = b.Buf[:0]
@@ -97,6 +102,11 @@ func (r *reader) MustClose() {
        r.readOffset = 0
 }
 
+func (r *reader) Close() error {
+       r.MustClose()
+       return nil
+}
+
 // BufferPool is a pool of Buffer.
 type BufferPool struct {
        p sync.Pool
diff --git a/pkg/fs/file_system.go b/pkg/fs/file_system.go
index 4efca46d..a2ed0950 100644
--- a/pkg/fs/file_system.go
+++ b/pkg/fs/file_system.go
@@ -29,22 +29,38 @@ const moduleName string = "filesystem"
 // Mode contains permission of file and directory.
 type Mode uint64
 
+// SeqWriter allows writing data to a file in a sequential way.
+type SeqWriter interface {
+       io.Writer
+       Path() string
+       Close() error
+}
+
 // Writer allows writing data to a file.
 type Writer interface {
        // Append mode, which adds new data to the end of a file.
        Write(buffer []byte) (int, error)
+       // SequentialWrite mode, which supports appending consecutive buffers 
to the end of the file.
+       SequentialWrite() SeqWriter
        // Returns the absolute path of the file.
        Path() string
        // Close File.
        Close() error
 }
 
+// SeqReader allows reading data from a file in a sequential way.
+type SeqReader interface {
+       io.Reader
+       Path() string
+       Close() error
+}
+
 // Reader allows reading data from a file.
 type Reader interface {
-       // Read the entire file using streaming read.
+       // Read the entire file at a specified offset.
        Read(offset int64, buffer []byte) (int, error)
-       // Read the entire file using streaming read.
-       StreamRead() io.Reader
+       // Read the entire file using sequential read.
+       SequentialRead() SeqReader
        // Returns the absolute path of the file.
        Path() string
        // Close File.
@@ -135,7 +151,7 @@ func MustFlush(fs FileSystem, buffer []byte, name string, 
permission Mode) {
 }
 
 // MustWriteData writes data to w and panics if it cannot write all data.
-func MustWriteData(w Writer, data []byte) {
+func MustWriteData(w SeqWriter, data []byte) {
        if len(data) == 0 {
                return
        }
diff --git a/pkg/fs/local_file_system.go b/pkg/fs/local_file_system.go
index b3438265..7a8b1904 100644
--- a/pkg/fs/local_file_system.go
+++ b/pkg/fs/local_file_system.go
@@ -25,6 +25,7 @@ import (
        "io"
        "os"
        "path/filepath"
+       "sync"
        "time"
 
        "github.com/shirou/gopsutil/v3/disk"
@@ -32,6 +33,8 @@ import (
        "github.com/apache/skywalking-banyandb/pkg/logger"
 )
 
+const defaultIOSize = 256 * 1024
+
 // localFileSystem implements the File System interface.
 type localFileSystem struct {
        logger *logger.Logger
@@ -347,6 +350,12 @@ func (file *LocalFile) Writev(iov *[][]byte) (int, error) {
        return size, nil
 }
 
+// SequentialWrite supports appending consecutive buffers to the end of the 
file.
+func (file *LocalFile) SequentialWrite() SeqWriter {
+       writer := generateWriter(file.file)
+       return &seqWriter{writer: writer, fileName: file.file.Name()}
+}
+
 // Read is used to read a specified location of file.
 func (file *LocalFile) Read(offset int64, buffer []byte) (int, error) {
        rsize, err := file.file.ReadAt(buffer, offset)
@@ -373,9 +382,9 @@ func (file *LocalFile) Readv(offset int64, iov *[][]byte) 
(int, error) {
        return size, nil
 }
 
-// StreamRead is used to read the entire file using streaming read.
-func (file *LocalFile) StreamRead() io.Reader {
-       reader := bufio.NewReader(file.file)
+// SequentialRead is used to read the entire file using streaming read.
+func (file *LocalFile) SequentialRead() SeqReader {
+       reader := generateReader(file.file)
        return &seqReader{reader: reader, fileName: file.file.Name()}
 }
 
@@ -437,3 +446,70 @@ func (i *seqReader) Read(p []byte) (int, error) {
        }
        return rsize, nil
 }
+
+func (i *seqReader) Path() string {
+       return i.fileName
+}
+
+func (i *seqReader) Close() error {
+       releaseReader(i.reader)
+       return nil
+}
+
+type seqWriter struct {
+       writer   *bufio.Writer
+       fileName string
+}
+
+func (w *seqWriter) Write(p []byte) (n int, err error) {
+       return w.writer.Write(p)
+}
+
+func (w *seqWriter) Path() string {
+       return w.fileName
+}
+
+func (w *seqWriter) Close() error {
+       if err := w.writer.Flush(); err != nil {
+               return &FileSystemError{
+                       Code:    closeError,
+                       Message: fmt.Sprintf("Flush File error, directory name: 
%s, error message: %s", w.fileName, err),
+               }
+       }
+       releaseWriter(w.writer)
+       return nil
+}
+
+func generateReader(f *os.File) *bufio.Reader {
+       v := bufReaderPool.Get()
+       if v == nil {
+               return bufio.NewReaderSize(f, defaultIOSize)
+       }
+       br := v.(*bufio.Reader)
+       br.Reset(f)
+       return br
+}
+
+func releaseReader(br *bufio.Reader) {
+       br.Reset(nil)
+       bufReaderPool.Put(br)
+}
+
+var bufReaderPool sync.Pool
+
+func generateWriter(f *os.File) *bufio.Writer {
+       v := bufWriterPool.Get()
+       if v == nil {
+               return bufio.NewWriterSize(f, defaultIOSize)
+       }
+       bw := v.(*bufio.Writer)
+       bw.Reset(f)
+       return bw
+}
+
+func releaseWriter(bw *bufio.Writer) {
+       bw.Reset(nil)
+       bufWriterPool.Put(bw)
+}
+
+var bufWriterPool sync.Pool
diff --git a/pkg/fs/local_file_system_test.go b/pkg/fs/local_file_system_test.go
index 1c96a127..eaf278a5 100644
--- a/pkg/fs/local_file_system_test.go
+++ b/pkg/fs/local_file_system_test.go
@@ -109,7 +109,8 @@ var _ = ginkgo.Describe("Loacl File System", func() {
                        gomega.Expect(size == len(data)).To(gomega.BeTrue())
 
                        buffer := make([]byte, len(data))
-                       iter := file.StreamRead()
+                       iter := file.SequentialRead()
+                       defer iter.Close()
                        for {
                                size, err := iter.Read(buffer)
                                if err == nil {
diff --git a/pkg/index/index.go b/pkg/index/index.go
index 9561b2f8..1796d66b 100644
--- a/pkg/index/index.go
+++ b/pkg/index/index.go
@@ -201,10 +201,16 @@ type Document struct {
 // Documents is a collection of documents.
 type Documents []Document
 
+// Batch is a collection of documents.
+type Batch struct {
+       Applied   chan struct{}
+       Documents Documents
+}
+
 // Writer allows writing fields and docID in a document to a index.
 type Writer interface {
        Write(fields []Field, docID uint64) error
-       Batch(docs Documents) error
+       Batch(batch Batch) error
 }
 
 // FieldIterable allows building a FieldIterator.
diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go
index 1a578aad..de11591f 100644
--- a/pkg/index/inverted/inverted.go
+++ b/pkg/index/inverted/inverted.go
@@ -91,14 +91,14 @@ type store struct {
        batchInterval time.Duration
 }
 
-func (s *store) Batch(docs index.Documents) error {
+func (s *store) Batch(batch index.Batch) error {
        if !s.closer.AddRunning() {
                return nil
        }
        defer s.closer.Done()
        select {
        case <-s.closer.CloseNotify():
-       case s.ch <- docs:
+       case s.ch <- batch:
        }
        return nil
 }
@@ -106,9 +106,10 @@ func (s *store) Batch(docs index.Documents) error {
 // NewStore create a new inverted index repository.
 func NewStore(opts StoreOpts) (index.SeriesStore, error) {
        indexConfig := blugeIndex.DefaultConfig(opts.Path)
-       // TODO:// parameterize the following options
-       // WithUnsafeBatches().
-       // WithPersisterNapTimeMSec(60 * 1000)
+       if opts.BatchWaitSec > 0 {
+               indexConfig = indexConfig.WithUnsafeBatches().
+                       WithPersisterNapTimeMSec(int(opts.BatchWaitSec * 1000))
+       }
        config := bluge.DefaultConfigWithIndexConfig(indexConfig)
        config.DefaultSearchAnalyzer = 
analyzers[databasev1.IndexRule_ANALYZER_KEYWORD]
        config.Logger = log.New(opts.Logger, opts.Logger.Module(), 0)
@@ -312,17 +313,20 @@ func (s *store) run() {
                }()
                size := 0
                batch := bluge.NewBatch()
-               flush := func() {
+               flush := func(applied chan struct{}) {
                        if size < 1 {
                                return
                        }
                        if err := s.writer.Batch(batch); err != nil {
                                s.l.Error().Err(err).Msg("write to the inverted 
index")
                        }
+                       if applied != nil {
+                               close(applied)
+                       }
                        batch.Reset()
                        size = 0
                }
-               defer flush()
+               defer flush(nil)
                var docIDBuffer bytes.Buffer
                for {
                        timer := time.NewTimer(s.batchInterval)
@@ -337,16 +341,18 @@ func (s *store) run() {
                                }
                                switch d := event.(type) {
                                case flushEvent:
-                                       flush()
+                                       flush(nil)
                                        close(d.onComplete)
-                               case index.Document, index.Documents:
+                               case index.Document, index.Batch:
                                        var docs []index.Document
                                        var isBatch bool
+                                       var applied chan struct{}
                                        switch v := d.(type) {
                                        case index.Document:
                                                docs = []index.Document{v}
-                                       case index.Documents:
-                                               docs = v
+                                       case index.Batch:
+                                               docs = v.Documents
+                                               applied = v.Applied
                                                isBatch = true
                                        }
 
@@ -384,11 +390,11 @@ func (s *store) run() {
                                                batch.Update(doc.ID(), doc)
                                        }
                                        if isBatch || size >= batchSize {
-                                               flush()
+                                               flush(applied)
                                        }
                                }
                        case <-timer.C:
-                               flush()
+                               flush(nil)
                        }
                        timer.Stop()
                }
diff --git a/pkg/index/inverted/inverted_series_test.go 
b/pkg/index/inverted/inverted_series_test.go
index 03860f83..702d95b8 100644
--- a/pkg/index/inverted/inverted_series_test.go
+++ b/pkg/index/inverted/inverted_series_test.go
@@ -207,6 +207,10 @@ func setupData(tester *assert.Assertions, s 
index.SeriesStore) {
                DocID:        3,
                EntityValues: []byte("test3"),
        }
-       tester.NoError(s.Batch([]index.Document{series1, series2, series3, 
series3}))
-       s.(*store).flush()
+       applied := make(chan struct{})
+       tester.NoError(s.Batch(index.Batch{
+               Applied:   applied,
+               Documents: []index.Document{series1, series2, series3, series3},
+       }))
+       <-applied
 }
diff --git a/pkg/index/lsm/lsm.go b/pkg/index/lsm/lsm.go
index ff2d7b40..d7cb0a21 100644
--- a/pkg/index/lsm/lsm.go
+++ b/pkg/index/lsm/lsm.go
@@ -40,7 +40,7 @@ type store struct {
 }
 
 // Batch implements index.Store.
-func (*store) Batch(_ index.Documents) error {
+func (*store) Batch(_ index.Batch) error {
        panic("unimplemented")
 }
 
diff --git 
a/test/integration/standalone/query_ondisk/query_ondisk_suite_test.go 
b/test/integration/standalone/query_ondisk/query_ondisk_suite_test.go
index ac89d108..f6fcb14c 100644
--- a/test/integration/standalone/query_ondisk/query_ondisk_suite_test.go
+++ b/test/integration/standalone/query_ondisk/query_ondisk_suite_test.go
@@ -68,6 +68,7 @@ var _ = SynchronizedBeforeSuite(func() []byte {
        ns := timestamp.NowMilli().UnixNano()
        now = time.Unix(0, ns-ns%int64(time.Minute))
        test_cases.Initialize(addr, now)
+       time.Sleep(10 * time.Second)
        closeFunc()
        time.Sleep(time.Second)
        addr, _, closeFunc = setup.EmptyClosableStandalone(path, ports)
diff --git a/test/stress/cases/istio/report.md 
b/test/stress/cases/istio/report.md
index c42e1018..67ccd719 100644
--- a/test/stress/cases/istio/report.md
+++ b/test/stress/cases/istio/report.md
@@ -29,37 +29,37 @@ latency: 2ns
 
 ### Problem
 
-Memory: blockPointer.append used 22.55% of total memory, that is significantly 
larger than expected.
+Memory: blockPointer.append used 22.55% of total memory. That's due to the 
fact that parts without the time overlap are merged. The unnecessary unpacking 
block operations take up a lot of memory.
 
-Disk IO: weighted_io(ms) is 504.606667, which is larger than 100ms. This means 
a lot of IO operations are blocked.
+CPU: As memory's part, the unnecessary unpacking block operations take up a 
lot of CPU.
 
 ## CPU
 
-CPU 95th-percentile: 2.74 cores (8 cores in total)
+CPU 95th-percentile: 2.91 cores (8 cores in total)
 
 ```bash
-Showing top 10 nodes out of 291
+Showing top 10 nodes out of 327
       flat  flat%   sum%        cum   cum%
-   657.01s 10.54% 10.54%    657.01s 10.54%  runtime/internal/syscall.Syscall6
-   242.03s  3.88% 14.43%    457.06s  7.33%  
github.com/blevesearch/vellum.registryCache.entry
-   211.69s  3.40% 17.82%    218.18s  3.50%  
github.com/klauspost/compress/zstd.(*fseEncoder).buildCTable
-   173.92s  2.79% 20.61%    697.34s 11.19%  runtime.mallocgc
-   172.26s  2.76% 23.38%    172.75s  2.77%  
github.com/blevesearch/vellum.(*builderNode).equiv (inline)
-   164.02s  2.63% 26.01%    188.35s  3.02%  runtime.findObject
-   131.81s  2.12% 28.13%    169.79s  2.72%  
github.com/klauspost/compress/zstd.(*fastEncoder).EncodeNoHist
-   127.66s  2.05% 30.17%    127.66s  2.05%  runtime.memmove
-   123.01s  1.97% 32.15%    141.11s  2.26%  
github.com/klauspost/compress/huff0.(*Scratch).huffSort
-    99.42s  1.60% 33.74%    134.90s  2.16%  
github.com/blevesearch/vellum.(*builderNodeUnfinished).lastCompiled
+   216.45s  4.14%  4.14%    226.17s  4.32%  
github.com/klauspost/compress/zstd.(*fseEncoder).buildCTable
+   171.32s  3.27%  7.41%    171.32s  3.27%  runtime/internal/syscall.Syscall6
+   155.50s  2.97% 10.38%    606.59s 11.59%  runtime.mallocgc
+   135.71s  2.59% 12.98%    169.48s  3.24%  
github.com/klauspost/compress/zstd.(*fastEncoder).EncodeNoHist
+   134.63s  2.57% 15.55%    248.72s  4.75%  
github.com/blevesearch/vellum.registryCache.entry
+   133.90s  2.56% 18.11%    133.90s  2.56%  runtime.memmove
+   130.33s  2.49% 20.60%    149.37s  2.85%  
github.com/klauspost/compress/huff0.(*Scratch).huffSort
+   118.60s  2.27% 22.87%    136.91s  2.62%  runtime.findObject
+    90.35s  1.73% 24.59%     92.01s  1.76%  
github.com/blevesearch/vellum.(*builderNode).equiv (inline)
+    83.19s  1.59% 26.18%     83.19s  1.59%  runtime.nextFreeFast (inline)
 ```
 
-From the top 10 list, we can see that the CPU is mainly used by `Syscall6`.
+From the top 10 list, we can see that the CPU is mainly used by `zstd.Encoder`.
 
 ## Heap Profile
 
-`alloc_bytes` 95th-percentile: 968.94 MB.
-`heap_inuse_bytes` 95th-percentile: 1054.00 MB.
-`sys_bytes` 95th-percentile: 1445.43 MB.
-`stack_inuse_bytes` 95th-percentile: 18.03 MB.
+`alloc_bytes` 95th-percentile: 1.33 GB.
+`heap_inuse_bytes` 95th-percentile: 1.46 GB.
+`sys_bytes` 95th-percentile: 1.95 GB.
+`stack_inuse_bytes` 95th-percentile: 17.89 MB.
 
 ```bash
 Showing top 10 nodes out of 212
@@ -81,31 +81,31 @@ Showing top 10 nodes out of 212
 ## Disk Usage
 
 ```bash
-measure: 300 MB
-measure/measure-default: 137 MB
-measure/measure-default/idx: 53 MB
+measure: 281 MB
+measure/measure-default: 122 MB
+measure/measure-default/idx: 38 MB
 measure/measure-default/shard-0: 84 MB
-measure/measure-default/shard-0/seg-20240111: 84 MB
-measure/measure-default/shard-0/seg-20240111/000000000000208b: 84 MB
-measure/measure-minute: 162 MB
-measure/measure-minute/idx: 32 MB
+measure/measure-default/shard-0/seg-20240117: 84 MB
+measure/measure-default/shard-0/seg-20240117/0000000000002157: 84 MB
+measure/measure-minute: 159 MB
+measure/measure-minute/idx: 29 MB
 measure/measure-minute/shard-0: 65 MB
-measure/measure-minute/shard-0/seg-20240111: 65 MB
-measure/measure-minute/shard-0/seg-20240111/00000000000010f4: 65 MB
+measure/measure-minute/shard-0/seg-20240117: 65 MB
+measure/measure-minute/shard-0/seg-20240117/0000000000001240: 65 MB
 measure/measure-minute/shard-1: 65 MB
-measure/measure-minute/shard-1/seg-20240111: 65 MB
-measure/measure-minute/shard-1/seg-20240111/000000000000109a: 65 MB
+measure/measure-minute/shard-1/seg-20240117: 65 MB
+measure/measure-minute/shard-1/seg-20240117/00000000000011ce: 65 MB
 ```
 
 ## Disk IO
 
 | Metric              | 95th-percentile per second  |
 |---------------------|-----------------------------|
-| read_count          | 0.856667                    |
-| merged_read_count   | 0.603333                    |
-| write_count         | 136.976667                  |
-| merged_write_count  | 269.993333                  |
-| read_bytes          | 13817.173333                |
-| write_bytes         | 17439505.066667             |
-| io_time(ms)         | 125.560000                  |
-| weighted_io(ms)     | 504.606667                  |
+| read_count          | 0.003333                    |
+| merged_read_count   | 0                           |
+| write_count         | 114.936667                  |
+| merged_write_count  | 70.315                      |
+| read_bytes          | 13.653333                   |
+| write_bytes         | 22672254.29                 |
+| io_time(ms)         | 57.906667                   |
+| weighted_io(ms)     | 922.218333                  |
diff --git a/test/stress/cases/istio/testdata/metrics/data.csv 
b/test/stress/cases/istio/testdata/metrics/data.csv
index fd586907..3de7189e 100644
--- a/test/stress/cases/istio/testdata/metrics/data.csv
+++ b/test/stress/cases/istio/testdata/metrics/data.csv
@@ -1,157 +1,155 @@
-1047477344.000000,1076912128.000000,1149135880.000000,9109504.000000,71.290000,215987310592.000000,532622512128.000000,2579166.000000,934717.000000,24098711.000000,33261592.000000,78010393600.000000,822417236992.000000,15857708.000000,41007686.000000
-144503392.000000,250183680.000000,1171024904.000000,2916352.000000,147.110000,216135090176.000000,532622512128.000000,2579166.000000,934717.000000,24104227.000000,33274471.000000,78010393600.000000,822863467520.000000,15862236.000000,41016666.000000
-196618704.000000,292552704.000000,1171024904.000000,2981888.000000,147.960000,216105725952.000000,532622512128.000000,2579167.000000,934717.000000,24104668.000000,33274620.000000,78010397696.000000,822940627968.000000,15862436.000000,41018250.000000
-197673784.000000,293806080.000000,1171024904.000000,2981888.000000,148.050000,216105734144.000000,532622512128.000000,2579172.000000,934717.000000,24104678.000000,33274698.000000,78010475520.000000,822940988416.000000,15862488.000000,41018262.000000
-321690400.000000,404094976.000000,1171024904.000000,5603328.000000,236.420000,216214724608.000000,532622512128.000000,2579172.000000,934717.000000,24109165.000000,33286373.000000,78010475520.000000,823294436352.000000,15867428.000000,41024247.000000
-200504888.000000,296976384.000000,1171024904.000000,2949120.000000,274.500000,216153882624.000000,532622512128.000000,2579173.000000,934717.000000,24111338.000000,33290076.000000,78010479616.000000,823632253952.000000,15869172.000000,41031684.000000
-201537752.000000,298123264.000000,1171024904.000000,2949120.000000,274.590000,216153882624.000000,532622512128.000000,2579173.000000,934717.000000,24111459.000000,33290132.000000,78010479616.000000,823633011712.000000,15869208.000000,41031791.000000
-202622776.000000,299270144.000000,1171024904.000000,2949120.000000,274.680000,216153886720.000000,532622512128.000000,2579173.000000,934717.000000,24111601.000000,33290167.000000,78010479616.000000,823633761280.000000,15869244.000000,41031886.000000
-292802872.000000,389914624.000000,1171024904.000000,7503872.000000,355.060000,216269213696.000000,532622512128.000000,2579173.000000,934717.000000,24115275.000000,33300171.000000,78010479616.000000,823924450304.000000,15873508.000000,41037192.000000
-183342520.000000,291782656.000000,1171024904.000000,3407872.000000,406.840000,216163098624.000000,532622512128.000000,2579173.000000,934717.000000,24118323.000000,33305384.000000,78010479616.000000,824397960192.000000,15875992.000000,41048857.000000
-184347096.000000,292814848.000000,1171024904.000000,3407872.000000,406.930000,216163098624.000000,532622512128.000000,2579173.000000,934717.000000,24118438.000000,33305447.000000,78010479616.000000,824398713856.000000,15876024.000000,41048944.000000
-185456736.000000,293994496.000000,1171024904.000000,3440640.000000,407.020000,216163106816.000000,532622512128.000000,2579173.000000,934717.000000,24118618.000000,33306301.000000,78010479616.000000,824404116480.000000,15876336.000000,41049084.000000
-450575008.000000,527925248.000000,1171024904.000000,6520832.000000,493.580000,216343662592.000000,532622512128.000000,2579173.000000,934717.000000,24122829.000000,33316912.000000,78010479616.000000,824741639168.000000,15880916.000000,41055064.000000
-254195832.000000,335667200.000000,1171024904.000000,3866624.000000,544.280000,216182599680.000000,532622512128.000000,2579173.000000,934717.000000,24125956.000000,33321736.000000,78010479616.000000,825228895232.000000,15883100.000000,41068874.000000
-255186416.000000,336691200.000000,1171024904.000000,3866624.000000,544.360000,216182599680.000000,532622512128.000000,2579173.000000,934717.000000,24126019.000000,33321791.000000,78010479616.000000,825229394944.000000,15883128.000000,41068928.000000
-255438160.000000,337059840.000000,1171024904.000000,3866624.000000,544.460000,216182599680.000000,532622512128.000000,2579173.000000,934717.000000,24126078.000000,33321821.000000,78010479616.000000,825229767680.000000,15883152.000000,41068969.000000
-492845008.000000,544358400.000000,1171024904.000000,7962624.000000,634.530000,216316932096.000000,532622512128.000000,2579173.000000,934717.000000,24130450.000000,33332769.000000,78010479616.000000,825606484992.000000,15887836.000000,41075835.000000
-159341840.000000,297156608.000000,1171024904.000000,4227072.000000,682.610000,216191127552.000000,532622512128.000000,2579173.000000,934717.000000,24133200.000000,33336633.000000,78010479616.000000,826068956160.000000,15889828.000000,41087650.000000
-160330280.000000,298188800.000000,1171024904.000000,4227072.000000,682.690000,216191127552.000000,532622512128.000000,2579173.000000,934717.000000,24133333.000000,33336702.000000,78010479616.000000,826069812224.000000,15889856.000000,41087748.000000
-161327512.000000,299442176.000000,1171024904.000000,4227072.000000,682.790000,216191131648.000000,532622512128.000000,2579173.000000,934717.000000,24133344.000000,33336734.000000,78010479616.000000,826069988352.000000,15889880.000000,41087758.000000
-488013200.000000,553967616.000000,1171024904.000000,8323072.000000,774.290000,216305545216.000000,532622512128.000000,2579173.000000,934717.000000,24137807.000000,33347582.000000,78010479616.000000,826467251200.000000,15894648.000000,41096028.000000
-247208864.000000,372301824.000000,1171024904.000000,4653056.000000,820.760000,216212951040.000000,532622512128.000000,2579173.000000,934717.000000,24140622.000000,33351724.000000,78010479616.000000,826940556288.000000,15896816.000000,41112039.000000
-248231408.000000,373309440.000000,1171024904.000000,4653056.000000,820.860000,216212979712.000000,532622512128.000000,2579173.000000,934717.000000,24140668.000000,33351796.000000,78010479616.000000,826941064192.000000,15896840.000000,41112085.000000
-249232632.000000,374538240.000000,1171024904.000000,4685824.000000,820.950000,216212983808.000000,532622512128.000000,2579173.000000,934717.000000,24140832.000000,33351861.000000,78010479616.000000,826942055424.000000,15896872.000000,41112196.000000
-417615264.000000,488087552.000000,1171287048.000000,8716288.000000,898.070000,216326787072.000000,532622512128.000000,2579173.000000,934717.000000,24144333.000000,33361662.000000,78010479616.000000,827219571712.000000,15901056.000000,41117278.000000
-226420104.000000,362766336.000000,1171287048.000000,5013504.000000,962.480000,216250486784.000000,532622512128.000000,2579173.000000,934717.000000,24148103.000000,33368077.000000,78010479616.000000,827809596416.000000,15904056.000000,41131234.000000
-261184032.000000,390455296.000000,1171287048.000000,5046272.000000,965.050000,216210780160.000000,532622512128.000000,2579173.000000,934717.000000,24148565.000000,33368253.000000,78010479616.000000,827883713536.000000,15904276.000000,41132947.000000
-262184976.000000,391438336.000000,1171287048.000000,5046272.000000,965.140000,216210780160.000000,532622512128.000000,2579173.000000,934717.000000,24148574.000000,33368275.000000,78010479616.000000,827883840512.000000,15904304.000000,41132954.000000
-604721304.000000,646250496.000000,1171549192.000000,6815744.000000,1043.320000,216356417536.000000,532622512128.000000,2579177.000000,934717.000000,24152276.000000,33378336.000000,78010496000.000000,828158329856.000000,15908588.000000,41137926.000000
-313094112.000000,447815680.000000,1171549192.000000,5505024.000000,1104.420000,216272936960.000000,532622512128.000000,2579178.000000,934717.000000,24155771.000000,33384261.000000,78010500096.000000,828726760448.000000,15911396.000000,41152748.000000
-216737216.000000,368173056.000000,1171549192.000000,5505024.000000,1107.830000,216241926144.000000,532622512128.000000,2579178.000000,934717.000000,24156168.000000,33384402.000000,78010500096.000000,828805039104.000000,15911604.000000,41154417.000000
-217728816.000000,369238016.000000,1171549192.000000,5505024.000000,1107.920000,216241926144.000000,532622512128.000000,2579178.000000,934717.000000,24156176.000000,33384427.000000,78010500096.000000,828805174272.000000,15911624.000000,41154423.000000
-380038216.000000,504356864.000000,1171549192.000000,9011200.000000,1186.700000,216343621632.000000,532622512128.000000,2579178.000000,934717.000000,24159852.000000,33394035.000000,78010500096.000000,829086954496.000000,15915792.000000,41159679.000000
-326293912.000000,461873152.000000,1171811336.000000,5832704.000000,1247.120000,216264146944.000000,532622512128.000000,2579178.000000,934717.000000,24163452.000000,33399814.000000,78010500096.000000,829657355264.000000,15918560.000000,41176808.000000
-215114168.000000,373841920.000000,1171811336.000000,5898240.000000,1250.410000,216228913152.000000,532622512128.000000,2579178.000000,934717.000000,24163888.000000,33399939.000000,78010500096.000000,829740545024.000000,15918788.000000,41178746.000000
-216095096.000000,374923264.000000,1171811336.000000,5898240.000000,1250.490000,216228913152.000000,532622512128.000000,2579178.000000,934717.000000,24163950.000000,33400010.000000,78010500096.000000,829741089792.000000,15918820.000000,41178793.000000
-435942384.000000,538640384.000000,1171811336.000000,9404416.000000,1329.010000,216359718912.000000,532622512128.000000,2579178.000000,934717.000000,24167602.000000,33409087.000000,78010500096.000000,830015144960.000000,15923008.000000,41184164.000000
-366382888.000000,503480320.000000,1171811336.000000,6225920.000000,1396.520000,216318005248.000000,532622512128.000000,2579178.000000,934717.000000,24171556.000000,33415349.000000,78010500096.000000,830678291456.000000,15926204.000000,41204060.000000
-288063024.000000,438190080.000000,1171811336.000000,6324224.000000,1400.280000,216287191040.000000,532622512128.000000,2579178.000000,934717.000000,24171987.000000,33415495.000000,78010500096.000000,830766330880.000000,15926452.000000,41206305.000000
-289049008.000000,439255040.000000,1171811336.000000,6324224.000000,1400.370000,216287199232.000000,532622512128.000000,2579178.000000,934717.000000,24172063.000000,33415519.000000,78010500096.000000,830766752768.000000,15926476.000000,41206361.000000
-592820440.000000,648617984.000000,1171811336.000000,10944512.000000,1491.500000,216416116736.000000,532622512128.000000,2579178.000000,934717.000000,24176089.000000,33426933.000000,78010500096.000000,831081718784.000000,15931292.000000,41212274.000000
-412955272.000000,549838848.000000,1171811336.000000,6717440.000000,1552.530000,216303153152.000000,532622512128.000000,2579178.000000,934717.000000,24179957.000000,33431978.000000,78010500096.000000,831786656768.000000,15933968.000000,41235608.000000
-400606016.000000,540704768.000000,1171811336.000000,6750208.000000,1554.720000,216243875840.000000,532622512128.000000,2579178.000000,934717.000000,24180509.000000,33432136.000000,78010500096.000000,831880209408.000000,15934208.000000,41237618.000000
-401658792.000000,541786112.000000,1171811336.000000,6750208.000000,1554.810000,216243879936.000000,532622512128.000000,2579178.000000,934717.000000,24180515.000000,33432154.000000,78010500096.000000,831880307712.000000,15934224.000000,41237623.000000
-705799792.000000,755605504.000000,1171811336.000000,11239424.000000,1645.060000,216447635456.000000,532622512128.000000,2579178.000000,934717.000000,24184545.000000,33443202.000000,78010500096.000000,832175440896.000000,15939052.000000,41242770.000000
-435444016.000000,572989440.000000,1171811336.000000,7208960.000000,1697.150000,216331624448.000000,532622512128.000000,2579178.000000,934717.000000,24188047.000000,33447769.000000,78010500096.000000,832808510464.000000,15941368.000000,41263377.000000
-357952816.000000,505380864.000000,1171811336.000000,7208960.000000,1698.940000,216261554176.000000,532622512128.000000,2579178.000000,934717.000000,24188613.000000,33447900.000000,78010500096.000000,832906941440.000000,15941624.000000,41266004.000000
-358981336.000000,506413056.000000,1171811336.000000,7208960.000000,1699.020000,216261554176.000000,532622512128.000000,2579178.000000,934717.000000,24188666.000000,33447919.000000,78010500096.000000,832907236352.000000,15941656.000000,41266042.000000
-645200880.000000,712359936.000000,1171811336.000000,11304960.000000,1789.820000,216432041984.000000,532622512128.000000,2579178.000000,934717.000000,24192661.000000,33459025.000000,78010500096.000000,833216009216.000000,15946280.000000,41271750.000000
-486666376.000000,629022720.000000,1171811336.000000,7569408.000000,1844.880000,216325537792.000000,532622512128.000000,2579178.000000,934717.000000,24196310.000000,33464136.000000,78010500096.000000,833855517696.000000,15948896.000000,41289795.000000
-279969736.000000,453255168.000000,1171811336.000000,7602176.000000,1847.550000,216268791808.000000,532622512128.000000,2579178.000000,934717.000000,24196803.000000,33464283.000000,78010500096.000000,833958388736.000000,15949132.000000,41292740.000000
-280966504.000000,454287360.000000,1171811336.000000,7602176.000000,1847.630000,216268800000.000000,532622512128.000000,2579178.000000,934717.000000,24197104.000000,33465131.000000,78010500096.000000,833964315648.000000,15949464.000000,41292992.000000
-579870776.000000,726736896.000000,1171811336.000000,10485760.000000,1935.420000,216453967872.000000,532622512128.000000,2579178.000000,934717.000000,24201162.000000,33475831.000000,78010500096.000000,834261988352.000000,15953888.000000,41298526.000000
-475865016.000000,630161408.000000,1171811336.000000,7995392.000000,1992.630000,216328896512.000000,532622512128.000000,2579178.000000,934717.000000,24205045.000000,33480941.000000,78010500096.000000,834951488512.000000,15956544.000000,41325944.000000
-277482128.000000,462774272.000000,1171811336.000000,7995392.000000,1995.450000,216269619200.000000,532622512128.000000,2579178.000000,934717.000000,24205505.000000,33481060.000000,78010500096.000000,835058951168.000000,15956800.000000,41329203.000000
-278469464.000000,463814656.000000,1171811336.000000,7995392.000000,1995.530000,216269619200.000000,532622512128.000000,2579178.000000,934717.000000,24205568.000000,33481121.000000,78010500096.000000,835059463168.000000,15956828.000000,41329255.000000
-527738136.000000,645808128.000000,1171811336.000000,11534336.000000,2081.020000,216436346880.000000,532622512128.000000,2579178.000000,934717.000000,24209341.000000,33491514.000000,78010500096.000000,835349738496.000000,15961140.000000,41334868.000000
-276888040.000000,477872128.000000,1171811336.000000,8355840.000000,2143.000000,216321773568.000000,532622512128.000000,2579178.000000,934717.000000,24213436.000000,33497515.000000,78010500096.000000,836062512128.000000,15964140.000000,41360952.000000
-437751880.000000,602570752.000000,1171811336.000000,8421376.000000,2146.530000,216277540864.000000,532622512128.000000,2579178.000000,934717.000000,24213996.000000,33497636.000000,78010500096.000000,836174967808.000000,15964376.000000,41367221.000000
-438791088.000000,603660288.000000,1171811336.000000,8421376.000000,2146.630000,216277544960.000000,532622512128.000000,2579178.000000,934717.000000,24214053.000000,33497694.000000,78010500096.000000,836175438848.000000,15964408.000000,41367267.000000
-618939528.000000,709967872.000000,1171811336.000000,12058624.000000,2228.890000,216448671744.000000,532622512128.000000,2579178.000000,934717.000000,24217645.000000,33507672.000000,78010500096.000000,836453663744.000000,15968716.000000,41372305.000000
-279186032.000000,487243776.000000,1171811336.000000,8781824.000000,2293.640000,216244719616.000000,532622512128.000000,2579178.000000,934717.000000,24221695.000000,33513492.000000,78010500096.000000,837132968960.000000,15971568.000000,41393505.000000
-519075632.000000,675717120.000000,1171811336.000000,8847360.000000,2298.990000,216226930688.000000,532622512128.000000,2579178.000000,934717.000000,24221990.000000,33513597.000000,78010500096.000000,837197677568.000000,15971756.000000,41395008.000000
-520077248.000000,676741120.000000,1171811336.000000,8847360.000000,2299.070000,216226930688.000000,532622512128.000000,2579178.000000,934717.000000,24221999.000000,33513608.000000,78010500096.000000,837197759488.000000,15971780.000000,41395016.000000
-724053432.000000,808714240.000000,1171811336.000000,12353536.000000,2392.520000,216420888576.000000,532622512128.000000,2579178.000000,934717.000000,24226100.000000,33524859.000000,78010500096.000000,837513647104.000000,15976660.000000,41401007.000000
-358294152.000000,547651584.000000,1171811336.000000,9273344.000000,2444.330000,216265412608.000000,532622512128.000000,2579178.000000,934717.000000,24229468.000000,33528852.000000,78010500096.000000,838106416128.000000,15978808.000000,41415078.000000
-311804984.000000,510500864.000000,1171811336.000000,9273344.000000,2449.080000,216244969472.000000,532622512128.000000,2579178.000000,934717.000000,24229823.000000,33528996.000000,78010500096.000000,838172369920.000000,15979000.000000,41416560.000000
-312764648.000000,511590400.000000,1171811336.000000,9273344.000000,2449.160000,216244973568.000000,532622512128.000000,2579178.000000,934717.000000,24229829.000000,33529014.000000,78010500096.000000,838172468224.000000,15979016.000000,41416565.000000
-529505368.000000,696999936.000000,1171811336.000000,12779520.000000,2542.110000,216430071808.000000,532622512128.000000,2579178.000000,934717.000000,24234010.000000,33540229.000000,78010500096.000000,838497436672.000000,15983972.000000,41422781.000000
-339917808.000000,540098560.000000,1171811336.000000,9666560.000000,2601.990000,216304758784.000000,532622512128.000000,2579178.000000,934717.000000,24237386.000000,33544345.000000,78010500096.000000,839106565120.000000,15986292.000000,41442663.000000
-407897520.000000,592953344.000000,1171811336.000000,9699328.000000,2607.860000,216234901504.000000,532622512128.000000,2579178.000000,934717.000000,24238024.000000,33544605.000000,78010500096.000000,839234204672.000000,15986596.000000,41446313.000000
-408902544.000000,593977344.000000,1171811336.000000,9699328.000000,2607.960000,216234901504.000000,532622512128.000000,2579178.000000,934717.000000,24238101.000000,33544625.000000,78010500096.000000,839234618368.000000,15986620.000000,41446376.000000
-775233832.000000,854188032.000000,1171811336.000000,14286848.000000,2699.790000,216443449344.000000,532622512128.000000,2579178.000000,934717.000000,24242147.000000,33555074.000000,78010500096.000000,839550624768.000000,15991248.000000,41452032.000000
-591935272.000000,772947968.000000,1171811336.000000,10125312.000000,2758.970000,216241905664.000000,532622512128.000000,2579435.000000,934898.000000,24245810.000000,33559685.000000,78014645248.000000,840164103168.000000,15993940.000000,41468601.000000
-391461864.000000,584253440.000000,1171811336.000000,10092544.000000,2765.330000,216235085824.000000,532622512128.000000,2579435.000000,934898.000000,24246117.000000,33559808.000000,78014645248.000000,840231408640.000000,15994140.000000,41470045.000000
-392425000.000000,585351168.000000,1171811336.000000,10092544.000000,2765.410000,216235089920.000000,532622512128.000000,2579435.000000,934898.000000,24246200.000000,33560870.000000,78014645248.000000,840236114944.000000,15994192.000000,41470141.000000
-557936528.000000,717930496.000000,1171811336.000000,14123008.000000,2852.760000,216390418432.000000,532622512128.000000,2579435.000000,934898.000000,24250931.000000,33571561.000000,78014645248.000000,840533263360.000000,15998636.000000,41477217.000000
-621850232.000000,789372928.000000,1171811336.000000,10518528.000000,2915.620000,216246894592.000000,532622512128.000000,2579435.000000,934898.000000,24254616.000000,33576880.000000,78014645248.000000,841151087616.000000,16001252.000000,41493824.000000
-333379000.000000,550379520.000000,1171811336.000000,10518528.000000,2922.120000,216239259648.000000,532622512128.000000,2579435.000000,934898.000000,24254923.000000,33576990.000000,78014645248.000000,841219154944.000000,16001448.000000,41495421.000000
-334343928.000000,551493632.000000,1171811336.000000,10518528.000000,2922.210000,216239263744.000000,532622512128.000000,2579435.000000,934898.000000,24255075.000000,33577054.000000,78014645248.000000,841220056064.000000,16001480.000000,41495536.000000
-858211176.000000,904830976.000000,1171811336.000000,16121856.000000,3004.990000,216371470336.000000,532622512128.000000,2579435.000000,934898.000000,24258733.000000,33587720.000000,78014645248.000000,841496740864.000000,16005828.000000,41500442.000000
-547721224.000000,733822976.000000,1171811336.000000,10878976.000000,3073.730000,216325128192.000000,532622512128.000000,2579435.000000,934898.000000,24262148.000000,33593399.000000,78014645248.000000,842056430592.000000,16008468.000000,41516397.000000
-429676408.000000,633020416.000000,1171811336.000000,10944512.000000,3084.970000,216231440384.000000,532622512128.000000,2579435.000000,934898.000000,24263028.000000,33593774.000000,78014645248.000000,842251936768.000000,16008984.000000,41521642.000000
-430700848.000000,634077184.000000,1171811336.000000,10944512.000000,3085.060000,216231440384.000000,532622512128.000000,2579435.000000,934898.000000,24263084.000000,33593796.000000,78014645248.000000,842252264448.000000,16009020.000000,41521683.000000
-923123984.000000,967335936.000000,1172073480.000000,16220160.000000,3180.160000,216383979520.000000,532622512128.000000,2579435.000000,934898.000000,24267264.000000,33605055.000000,78014645248.000000,842584101888.000000,16014136.000000,41528417.000000
-610214232.000000,804519936.000000,1172073480.000000,11304960.000000,3241.030000,216232701952.000000,532622512128.000000,2579435.000000,934898.000000,24270464.000000,33608704.000000,78014645248.000000,843160441856.000000,16016340.000000,41546342.000000
-416499096.000000,628285440.000000,1172073480.000000,11337728.000000,3247.470000,216225935360.000000,532622512128.000000,2579435.000000,934898.000000,24270824.000000,33608838.000000,78014645248.000000,843230008320.000000,16016548.000000,41548140.000000
-417482744.000000,629350400.000000,1172073480.000000,11337728.000000,3247.560000,216225935360.000000,532622512128.000000,2579435.000000,934898.000000,24270832.000000,33608854.000000,78014645248.000000,843230106624.000000,16016568.000000,41548147.000000
-896464040.000000,948518912.000000,1172073480.000000,15138816.000000,3341.310000,216402153472.000000,532622512128.000000,2579435.000000,934898.000000,24274916.000000,33619751.000000,78014645248.000000,843530368000.000000,16021204.000000,41553600.000000
-620347856.000000,801988608.000000,1172073480.000000,11730944.000000,3405.000000,216255041536.000000,532622512128.000000,2579435.000000,934898.000000,24278215.000000,33623940.000000,78014645248.000000,844132058112.000000,16023476.000000,41571012.000000
-473314064.000000,677068800.000000,1172073480.000000,11796480.000000,3414.260000,216223866880.000000,532622512128.000000,2579435.000000,934898.000000,24278840.000000,33624219.000000,78014645248.000000,844258182144.000000,16023824.000000,41573783.000000
-474333152.000000,678182912.000000,1172073480.000000,11796480.000000,3414.340000,216223875072.000000,532622512128.000000,2579435.000000,934898.000000,24279096.000000,33625048.000000,78014645248.000000,844263810048.000000,16024176.000000,41574007.000000
-1024406976.000000,1060954112.000000,1197239304.000000,14024704.000000,3505.810000,216415698944.000000,532622512128.000000,2579435.000000,934898.000000,24283355.000000,33636669.000000,78014645248.000000,844564558848.000000,16028968.000000,41579426.000000
-676390872.000000,860880896.000000,1197239304.000000,12156928.000000,3568.490000,216247361536.000000,532622512128.000000,2579435.000000,934898.000000,24286844.000000,33640802.000000,78014645248.000000,845171852288.000000,16031420.000000,41601205.000000
-648432320.000000,838860800.000000,1197239304.000000,12222464.000000,3574.790000,216240578560.000000,532622512128.000000,2579435.000000,934898.000000,24287162.000000,33640924.000000,78014645248.000000,845242795008.000000,16031620.000000,41602861.000000
-649469704.000000,839892992.000000,1197239304.000000,12222464.000000,3574.870000,216240582656.000000,532622512128.000000,2579435.000000,934898.000000,24287228.000000,33640981.000000,78014645248.000000,845243298816.000000,16031644.000000,41602914.000000
-791933504.000000,897048576.000000,1197239304.000000,16842752.000000,3667.150000,216386162688.000000,532622512128.000000,2579435.000000,934898.000000,24291301.000000,33651981.000000,78014645248.000000,845555262464.000000,16036424.000000,41608573.000000
-659822856.000000,864133120.000000,1197239304.000000,12517376.000000,3732.590000,216325578752.000000,532622512128.000000,2579435.000000,934898.000000,24294340.000000,33655899.000000,78014645248.000000,846065882112.000000,16038728.000000,41622926.000000
-508670016.000000,730415104.000000,1197239304.000000,12550144.000000,3739.550000,216233742336.000000,532622512128.000000,2579435.000000,934898.000000,24294980.000000,33656115.000000,78014645248.000000,846209262592.000000,16039044.000000,41627836.000000
-509628544.000000,731455488.000000,1197239304.000000,12582912.000000,3739.640000,216233742336.000000,532622512128.000000,2579435.000000,934898.000000,24295049.000000,33656137.000000,78014645248.000000,846209647616.000000,16039068.000000,41627884.000000
-902410432.000000,981835776.000000,1206761496.000000,15826944.000000,3829.190000,216400343040.000000,532622512128.000000,2579435.000000,934898.000000,24298890.000000,33666687.000000,78014645248.000000,846500025344.000000,16043816.000000,41633120.000000
-693899760.000000,895844352.000000,1206761496.000000,12976128.000000,3895.350000,216437198848.000000,532622512128.000000,2579435.000000,934898.000000,24301593.000000,33670927.000000,78014645248.000000,846926517248.000000,16046072.000000,41644057.000000
-570358488.000000,789594112.000000,1206761496.000000,13008896.000000,3903.870000,216244035584.000000,532622512128.000000,2579435.000000,934898.000000,24302842.000000,33671429.000000,78014645248.000000,847182058496.000000,16046652.000000,41653274.000000
-571431264.000000,790650880.000000,1206761496.000000,13008896.000000,3903.970000,216244035584.000000,532622512128.000000,2579435.000000,934898.000000,24302849.000000,33671445.000000,78014645248.000000,847182152704.000000,16046672.000000,41653280.000000
-613321872.000000,827072512.000000,1206761496.000000,16220160.000000,3991.170000,216383057920.000000,532622512128.000000,2579435.000000,934898.000000,24306615.000000,33681523.000000,78014645248.000000,847465239552.000000,16051028.000000,41658496.000000
-651780272.000000,873021440.000000,1291781160.000000,13369344.000000,4060.190000,216403828736.000000,532622512128.000000,2579436.000000,934898.000000,24309420.000000,33686092.000000,78014649344.000000,847899857920.000000,16053404.000000,41670581.000000
-689819136.000000,915087360.000000,1291781160.000000,13402112.000000,4071.990000,216237846528.000000,532622512128.000000,2579436.000000,934898.000000,24310632.000000,33686608.000000,78014649344.000000,848157824000.000000,16053992.000000,41678022.000000
-690798696.000000,916086784.000000,1291781160.000000,13402112.000000,4072.090000,216237846528.000000,532622512128.000000,2579436.000000,934898.000000,24310699.000000,33686627.000000,78014649344.000000,848158192640.000000,16054012.000000,41678075.000000
-1068923856.000000,1108983808.000000,1291781160.000000,16908288.000000,4156.900000,216353181696.000000,532622512128.000000,2579436.000000,934898.000000,24314396.000000,33696436.000000,78014649344.000000,848450978816.000000,16058316.000000,41683312.000000
-763539400.000000,972324864.000000,1316946984.000000,13795328.000000,4229.970000,216396673024.000000,532622512128.000000,2579446.000000,934898.000000,24317363.000000,33701445.000000,78014690304.000000,848902759424.000000,16060708.000000,41696371.000000
-549209112.000000,780091392.000000,1316946984.000000,13795328.000000,4241.800000,216230424576.000000,532622512128.000000,2579447.000000,934898.000000,24318667.000000,33701999.000000,78014694400.000000,849164059648.000000,16061344.000000,41702870.000000
-550267880.000000,781131776.000000,1316946984.000000,13795328.000000,4241.880000,216230428672.000000,532622512128.000000,2579447.000000,934898.000000,24318675.000000,33702012.000000,78014694400.000000,849164145664.000000,16061372.000000,41702876.000000
-835743040.000000,965910528.000000,1316946984.000000,16875520.000000,4336.600000,216434671616.000000,532622512128.000000,2579447.000000,934898.000000,24322687.000000,33713843.000000,78014694400.000000,849457705984.000000,16066176.000000,41708085.000000
-652811072.000000,873193472.000000,1316946984.000000,14221312.000000,4398.980000,216452616192.000000,532622512128.000000,2579447.000000,934898.000000,24325240.000000,33717412.000000,78014694400.000000,849878709248.000000,16068024.000000,41721215.000000
-453676944.000000,707141632.000000,1316946984.000000,14254080.000000,4407.530000,216251072512.000000,532622512128.000000,2579447.000000,934898.000000,24326422.000000,33717868.000000,78014694400.000000,850142036992.000000,16068644.000000,41728055.000000
-454674632.000000,708182016.000000,1316946984.000000,14254080.000000,4407.620000,216251076608.000000,532622512128.000000,2579447.000000,934898.000000,24326474.000000,33717922.000000,78014694400.000000,850142471168.000000,16068672.000000,41728100.000000
-713334024.000000,885342208.000000,1321141288.000000,18317312.000000,4500.960000,216382279680.000000,532622512128.000000,2579456.000000,934898.000000,24330515.000000,33728990.000000,78014731264.000000,850461058048.000000,16073496.000000,41734104.000000
-559177800.000000,811810816.000000,1321141288.000000,14647296.000000,4562.950000,216421654528.000000,532622512128.000000,2579456.000000,934898.000000,24333155.000000,33733175.000000,78014731264.000000,850870875136.000000,16075644.000000,41745997.000000
-573617280.000000,820150272.000000,1321141288.000000,14680064.000000,4571.350000,216278470656.000000,532622512128.000000,2579456.000000,934898.000000,24334080.000000,33733503.000000,78014731264.000000,851079373824.000000,16076160.000000,41751296.000000
-574631112.000000,821207040.000000,1321141288.000000,14680064.000000,4571.440000,216278470656.000000,532622512128.000000,2579456.000000,934898.000000,24334150.000000,33733524.000000,78014731264.000000,851079750656.000000,16076188.000000,41751346.000000
-747266752.000000,929832960.000000,1321141288.000000,18677760.000000,4661.930000,216389705728.000000,532622512128.000000,2579460.000000,934898.000000,24338073.000000,33744031.000000,78014747648.000000,851375182848.000000,16080996.000000,41756633.000000
-811448192.000000,1036615680.000000,1321141288.000000,15007744.000000,4728.090000,216430583808.000000,532622512128.000000,2579461.000000,934898.000000,24340751.000000,33747978.000000,78014751744.000000,851810022400.000000,16083060.000000,41768165.000000
-781961504.000000,1017454592.000000,1321141288.000000,15073280.000000,4739.130000,216245497856.000000,532622512128.000000,2579464.000000,934898.000000,24342009.000000,33748495.000000,78014764032.000000,852080100352.000000,16083668.000000,41776505.000000
-782956496.000000,1018494976.000000,1321141288.000000,15073280.000000,4739.220000,216245497856.000000,532622512128.000000,2579464.000000,934898.000000,24342081.000000,33748515.000000,78014764032.000000,852080497664.000000,16083696.000000,41776561.000000
-843682424.000000,1005879296.000000,1360023608.000000,19136512.000000,4827.440000,216408956928.000000,532622512128.000000,2579464.000000,934898.000000,24345915.000000,33759468.000000,78014764032.000000,852380787712.000000,16088200.000000,41782027.000000
-471979424.000000,744054784.000000,1372606520.000000,15433728.000000,4895.060000,216367067136.000000,532622512128.000000,2579468.000000,934898.000000,24348878.000000,33764697.000000,78014780416.000000,852849034240.000000,16090740.000000,41795652.000000
-450793784.000000,722673664.000000,1372606520.000000,15466496.000000,4905.130000,216236085248.000000,532622512128.000000,2579470.000000,934898.000000,24349923.000000,33765072.000000,78014788608.000000,853064926208.000000,16091296.000000,41801660.000000
-451720424.000000,723746816.000000,1372606520.000000,15466496.000000,4905.220000,216236085248.000000,532622512128.000000,2579470.000000,934898.000000,24349931.000000,33765088.000000,78014788608.000000,853065024512.000000,16091316.000000,41801667.000000
-774937472.000000,990445568.000000,1372606520.000000,18382848.000000,4989.800000,216410808320.000000,532622512128.000000,2579473.000000,934898.000000,24353655.000000,33775394.000000,78014800896.000000,853330379776.000000,16095656.000000,41806506.000000
-692727160.000000,935575552.000000,1372606520.000000,15826944.000000,5057.970000,216424169472.000000,532622512128.000000,2579476.000000,934898.000000,24356737.000000,33779741.000000,78014813184.000000,853790569472.000000,16098008.000000,41819919.000000
-825238232.000000,1057636352.000000,1372606520.000000,15925248.000000,5071.140000,216258383872.000000,532622512128.000000,2579478.000000,934898.000000,24358006.000000,33780225.000000,78014821376.000000,854068589568.000000,16098648.000000,41827064.000000
-826229192.000000,1058660352.000000,1372606520.000000,15925248.000000,5071.220000,216258396160.000000,532622512128.000000,2579478.000000,934898.000000,24358252.000000,33781092.000000,78014821376.000000,854074319872.000000,16099032.000000,41827279.000000
-1051856816.000000,1130823680.000000,1372606520.000000,20119552.000000,5170.900000,216427343872.000000,532622512128.000000,2579480.000000,934898.000000,24362717.000000,33793812.000000,78014829568.000000,854408086528.000000,16104248.000000,41833482.000000
-708756888.000000,952344576.000000,1436654664.000000,16252928.000000,5227.060000,216310472704.000000,532622512128.000000,2579482.000000,934898.000000,24365667.000000,33797217.000000,78014837760.000000,854921393152.000000,16106148.000000,41848371.000000
-869190960.000000,1100349440.000000,1436654664.000000,16285696.000000,5234.820000,216250499072.000000,532622512128.000000,2579484.000000,934898.000000,24366295.000000,33797432.000000,78014845952.000000,855062127616.000000,16106504.000000,41851741.000000
-870176128.000000,1101414400.000000,1436654664.000000,16285696.000000,5234.910000,216250499072.000000,532622512128.000000,2579484.000000,934898.000000,24366350.000000,33797464.000000,78014845952.000000,855062483968.000000,16106532.000000,41851788.000000
-1137455816.000000,1207549952.000000,1449237576.000000,20152320.000000,5321.490000,216388722688.000000,532622512128.000000,2579485.000000,934898.000000,24370022.000000,33807608.000000,78014850048.000000,855348302848.000000,16110928.000000,41857441.000000
-571767048.000000,844046336.000000,1449237576.000000,16711680.000000,5391.890000,216407445504.000000,532622512128.000000,2579489.000000,934898.000000,24373154.000000,33812759.000000,78014866432.000000,855829865472.000000,16113440.000000,41869770.000000
-514990208.000000,794312704.000000,1449237576.000000,16809984.000000,5406.710000,216252874752.000000,532622512128.000000,2579489.000000,934898.000000,24374484.000000,33813262.000000,78014866432.000000,856114459648.000000,16114064.000000,41880503.000000
-515922896.000000,795394048.000000,1449237576.000000,16809984.000000,5406.800000,216252874752.000000,532622512128.000000,2579489.000000,934898.000000,24374490.000000,33813278.000000,78014866432.000000,856114549760.000000,16114084.000000,41880508.000000
-1051940632.000000,1171873792.000000,1449237576.000000,20381696.000000,5494.420000,216415571968.000000,532622512128.000000,2579489.000000,934898.000000,24378305.000000,33824101.000000,78014866432.000000,856415958016.000000,16118660.000000,41886200.000000
-580243608.000000,855883776.000000,1449237576.000000,17039360.000000,5556.650000,216394838016.000000,532622512128.000000,2579489.000000,934898.000000,24381122.000000,33828867.000000,78014866432.000000,856863790080.000000,16121036.000000,41898995.000000
-613272720.000000,879353856.000000,1449237576.000000,17104896.000000,5567.490000,216267210752.000000,532622512128.000000,2579489.000000,934898.000000,24382230.000000,33829268.000000,78014866432.000000,857089094656.000000,16121576.000000,41906410.000000
-614248792.000000,880353280.000000,1449237576.000000,17104896.000000,5567.580000,216267210752.000000,532622512128.000000,2579489.000000,934898.000000,24382237.000000,33829288.000000,78014866432.000000,857089205248.000000,16121592.000000,41906416.000000
-1252014544.000000,1309556736.000000,1449237576.000000,20840448.000000,5659.490000,216431235072.000000,532622512128.000000,2579489.000000,934898.000000,24386023.000000,33839361.000000,78014866432.000000,857377813504.000000,16126184.000000,41911971.000000
-797175408.000000,1046847488.000000,1500964952.000000,17530880.000000,5720.850000,216401350656.000000,532622512128.000000,2579489.000000,934898.000000,24388988.000000,33843529.000000,78014866432.000000,857839866880.000000,16128444.000000,41924337.000000
-752652464.000000,1001832448.000000,1500964952.000000,17563648.000000,5731.530000,216268832768.000000,532622512128.000000,2579489.000000,934898.000000,24390009.000000,33843905.000000,78014866432.000000,858067031040.000000,16128964.000000,41931964.000000
-753659376.000000,1002897408.000000,1500964952.000000,17563648.000000,5731.620000,216268836864.000000,532622512128.000000,2579489.000000,934898.000000,24390069.000000,33843964.000000,78014866432.000000,858067518464.000000,16128988.000000,41932022.000000
-1007611960.000000,1216274432.000000,1500964952.000000,21561344.000000,5824.770000,216411131904.000000,532622512128.000000,2579489.000000,934898.000000,24393973.000000,33854536.000000,78014866432.000000,858377188352.000000,16133704.000000,41937748.000000
-909896688.000000,1173528576.000000,1530325080.000000,17956864.000000,5887.440000,216354177024.000000,532622512128.000000,2579489.000000,934898.000000,24396944.000000,33858743.000000,78014866432.000000,858845758464.000000,16135996.000000,41953151.000000
-601099344.000000,887398400.000000,1530325080.000000,17956864.000000,5896.570000,216265007104.000000,532622512128.000000,2579489.000000,934898.000000,24397670.000000,33858979.000000,78014866432.000000,859011412992.000000,16136400.000000,41956770.000000
-602074544.000000,888430592.000000,1530325080.000000,17956864.000000,5896.650000,216265007104.000000,532622512128.000000,2579489.000000,934898.000000,24397798.000000,33859025.000000,78014866432.000000,859012137984.000000,16136432.000000,41956888.000000
-897101312.000000,1099210752.000000,1530587224.000000,22544384.000000,5987.690000,216399233024.000000,532622512128.000000,2579489.000000,934898.000000,24401675.000000,33869678.000000,78014866432.000000,859319858176.000000,16141116.000000,41962644.000000
-779468568.000000,1037910016.000000,1530587224.000000,18382848.000000,6050.910000,216452333568.000000,532622512128.000000,2579489.000000,934898.000000,24404550.000000,33873899.000000,78014866432.000000,859770196992.000000,16143364.000000,41975158.000000
-608176264.000000,891797504.000000,1530587224.000000,18382848.000000,6065.770000,216282607616.000000,532622512128.000000,2579489.000000,934898.000000,24405905.000000,33874388.000000,78014866432.000000,860067116032.000000,16144012.000000,41984858.000000
-609149112.000000,892903424.000000,1530587224.000000,18382848.000000,6065.860000,216282611712.000000,532622512128.000000,2579489.000000,934898.000000,24405913.000000,33874409.000000,78014866432.000000,860067234816.000000,16144028.000000,41984865.000000
-1082462264.000000,1187504128.000000,1543301208.000000,22544384.000000,6162.430000,216430489600.000000,532622512128.000000,2579489.000000,934898.000000,24409870.000000,33885905.000000,78014866432.000000,860382004224.000000,16148828.000000,41990624.000000
+832385352.000000,871292928.000000,989058520.000000,6881280.000000,66.720000,212879372288.000000,532622512128.000000,2584957.000000,937549.000000,26991407.000000,37485752.000000,78096610304.000000,1030296474624.000000,17890972.000000,47956390.000000
+603740712.000000,809754624.000000,1295320104.000000,5308416.000000,135.190000,212984217600.000000,532622512128.000000,2584957.000000,937549.000000,26995450.000000,37491007.000000,78096610304.000000,1030738412544.000000,17893616.000000,47964489.000000
+189226168.000000,303505408.000000,1295320104.000000,4849664.000000,171.880000,213007212544.000000,532622512128.000000,2584957.000000,937549.000000,26998199.000000,37493821.000000,78096610304.000000,1031128146944.000000,17895276.000000,47972515.000000
+845299264.000000,920797184.000000,1295844392.000000,6815744.000000,255.730000,213068533760.000000,532622512128.000000,2584957.000000,937549.000000,27000008.000000,37495609.000000,78096610304.000000,1031406539776.000000,17896412.000000,47978274.000000
+258854352.000000,363921408.000000,1295844392.000000,3211264.000000,321.890000,213079506944.000000,532622512128.000000,2584957.000000,937549.000000,27004026.000000,37498902.000000,78096610304.000000,1032047838208.000000,17898540.000000,47993944.000000
+179251864.000000,296435712.000000,1295844392.000000,3211264.000000,322.610000,213039562752.000000,532622512128.000000,2584957.000000,937549.000000,27004269.000000,37499053.000000,78096610304.000000,1032096994304.000000,17898660.000000,47995474.000000
+179465176.000000,296747008.000000,1295844392.000000,3211264.000000,322.690000,213039566848.000000,532622512128.000000,2584957.000000,937549.000000,27004483.000000,37499949.000000,78096610304.000000,1032102708224.000000,17899036.000000,47995659.000000
+806834432.000000,924966912.000000,1295844392.000000,7766016.000000,401.680000,213109301248.000000,532622512128.000000,2584957.000000,937549.000000,27005968.000000,37501142.000000,78096610304.000000,1032338138112.000000,17899832.000000,48000323.000000
+263657720.000000,365715456.000000,1300038696.000000,3801088.000000,469.150000,213113921536.000000,532622512128.000000,2584957.000000,937549.000000,27009690.000000,37503877.000000,78096610304.000000,1032987321344.000000,17901768.000000,48017633.000000
+254502168.000000,359784448.000000,1300038696.000000,3571712.000000,473.380000,213051101184.000000,532622512128.000000,2584957.000000,937549.000000,27010436.000000,37504293.000000,78096610304.000000,1033129116672.000000,17902140.000000,48021806.000000
+255463880.000000,360923136.000000,1300038696.000000,3571712.000000,473.470000,213051105280.000000,532622512128.000000,2584957.000000,937549.000000,27010444.000000,37504310.000000,78096610304.000000,1033129219072.000000,17902164.000000,48021813.000000
+910932776.000000,996909056.000000,1312621608.000000,6291456.000000,558.390000,213126623232.000000,532622512128.000000,2584957.000000,937549.000000,27012091.000000,37505633.000000,78096610304.000000,1033414226944.000000,17903216.000000,48027420.000000
+288439680.000000,383344640.000000,1312621608.000000,3997696.000000,624.690000,213102080000.000000,532622512128.000000,2584957.000000,937549.000000,27016272.000000,37508513.000000,78096610304.000000,1034166244352.000000,17905380.000000,48049475.000000
+155663808.000000,302178304.000000,1312621608.000000,4456448.000000,628.120000,213065154560.000000,532622512128.000000,2584957.000000,937549.000000,27016792.000000,37508818.000000,78096610304.000000,1034265461760.000000,17905632.000000,48052482.000000
+156674888.000000,303243264.000000,1312621608.000000,4456448.000000,628.200000,213065158656.000000,532622512128.000000,2584957.000000,937549.000000,27016801.000000,37508832.000000,78096610304.000000,1034265555968.000000,17905652.000000,48052490.000000
+963699160.000000,1029505024.000000,1312621608.000000,8716288.000000,709.760000,213120307200.000000,532622512128.000000,2584957.000000,937549.000000,27018404.000000,37510148.000000,78096610304.000000,1034541138944.000000,17906652.000000,48059170.000000
+236425360.000000,374218752.000000,1312621608.000000,4554752.000000,779.330000,213150019584.000000,532622512128.000000,2584957.000000,937549.000000,27022349.000000,37512845.000000,78096610304.000000,1035260195840.000000,17908636.000000,48081425.000000
+327637712.000000,432250880.000000,1312621608.000000,6520832.000000,783.570000,213077688320.000000,532622512128.000000,2584957.000000,937549.000000,27023099.000000,37513272.000000,78096610304.000000,1035404166144.000000,17908980.000000,48086406.000000
+328764352.000000,433496064.000000,1312621608.000000,6520832.000000,783.660000,213077696512.000000,532622512128.000000,2584957.000000,937549.000000,27023109.000000,37513285.000000,78096610304.000000,1035404256256.000000,17909008.000000,48086415.000000
+805164336.000000,921067520.000000,1347309624.000000,7962624.000000,868.050000,213171515392.000000,532622512128.000000,2584957.000000,937549.000000,27024707.000000,37514489.000000,78096610304.000000,1035694388224.000000,17909956.000000,48094532.000000
+273228864.000000,425164800.000000,1347309624.000000,4784128.000000,934.910000,213180473344.000000,532622512128.000000,2584957.000000,937549.000000,27028810.000000,37517060.000000,78096610304.000000,1036455224320.000000,17911944.000000,48120151.000000
+348880696.000000,465330176.000000,1347309624.000000,6914048.000000,937.490000,213087895552.000000,532622512128.000000,2584957.000000,937549.000000,27029361.000000,37517353.000000,78096610304.000000,1036570080256.000000,17912204.000000,48124917.000000
+349897776.000000,466616320.000000,1347309624.000000,6914048.000000,937.580000,213087895552.000000,532622512128.000000,2584957.000000,937549.000000,27029401.000000,37517392.000000,78096610304.000000,1036570403840.000000,17912232.000000,48124955.000000
+696204584.000000,876036096.000000,1347309624.000000,8912896.000000,1019.900000,213117693952.000000,532622512128.000000,2584957.000000,937549.000000,27031088.000000,37518612.000000,78096610304.000000,1036872713216.000000,17913224.000000,48131754.000000
+281510624.000000,436453376.000000,1347309624.000000,5636096.000000,1089.080000,213148893184.000000,532622512128.000000,2584957.000000,937549.000000,27034958.000000,37521066.000000,78096610304.000000,1037590135808.000000,17915164.000000,48157076.000000
+301848552.000000,451584000.000000,1347309624.000000,5242880.000000,1094.800000,213093310464.000000,532622512128.000000,2584957.000000,937549.000000,27035716.000000,37521402.000000,78096610304.000000,1037748909056.000000,17915484.000000,48163602.000000
+302874816.000000,452722688.000000,1347309624.000000,5242880.000000,1094.870000,213093310464.000000,532622512128.000000,2584957.000000,937549.000000,27035754.000000,37521435.000000,78096610304.000000,1037749199872.000000,17915504.000000,48163637.000000
+1203195464.000000,1267998720.000000,1364611128.000000,11960320.000000,1181.070000,213155397632.000000,532622512128.000000,2584957.000000,937549.000000,27037504.000000,37522758.000000,78096610304.000000,1038068700160.000000,17916588.000000,48173026.000000
+252242320.000000,423305216.000000,1364611128.000000,5636096.000000,1248.450000,213182681088.000000,532622512128.000000,2584957.000000,937549.000000,27041449.000000,37525128.000000,78096610304.000000,1038813811712.000000,17918448.000000,48196450.000000
+341529080.000000,494559232.000000,1364611128.000000,6684672.000000,1253.240000,213103108096.000000,532622512128.000000,2584957.000000,937549.000000,27042240.000000,37525513.000000,78096610304.000000,1038981874688.000000,17918804.000000,48203869.000000
+342644704.000000,495697920.000000,1364611128.000000,6684672.000000,1253.330000,213103112192.000000,532622512128.000000,2584957.000000,937549.000000,27042248.000000,37525527.000000,78096610304.000000,1038981964800.000000,17918820.000000,48203877.000000
+1110348472.000000,1179885568.000000,1390301240.000000,11370496.000000,1335.450000,213188640768.000000,532622512128.000000,2584957.000000,937549.000000,27043807.000000,37526707.000000,78096610304.000000,1039263237120.000000,17919728.000000,48212931.000000
+305426544.000000,483442688.000000,1446222920.000000,6488064.000000,1406.190000,213179535360.000000,532622512128.000000,2584957.000000,937549.000000,27047947.000000,37529223.000000,78096610304.000000,1040052048896.000000,17921760.000000,48240384.000000
+315108320.000000,485343232.000000,1446222920.000000,7045120.000000,1414.650000,213115342848.000000,532622512128.000000,2584957.000000,937549.000000,27049065.000000,37529818.000000,78096610304.000000,1040276337664.000000,17922280.000000,48248903.000000
+316142976.000000,486465536.000000,1446222920.000000,7045120.000000,1414.730000,213115342848.000000,532622512128.000000,2584957.000000,937549.000000,27049073.000000,37529830.000000,78096610304.000000,1040276419584.000000,17922300.000000,48248911.000000
+976163312.000000,1073627136.000000,1446485064.000000,10092544.000000,1500.950000,213173391360.000000,532622512128.000000,2584957.000000,937549.000000,27050933.000000,37531194.000000,78096610304.000000,1040621282304.000000,17923444.000000,48259986.000000
+396013072.000000,567582720.000000,1446747208.000000,6520832.000000,1569.210000,213201313792.000000,532622512128.000000,2584957.000000,937549.000000,27054860.000000,37533268.000000,78096610304.000000,1041384141824.000000,17925268.000000,48287655.000000
+387304640.000000,554041344.000000,1446747208.000000,7995392.000000,1574.900000,213121208320.000000,532622512128.000000,2584957.000000,937549.000000,27055758.000000,37533685.000000,78096610304.000000,1041573614592.000000,17925684.000000,48295576.000000
+388281520.000000,555196416.000000,1446747208.000000,7995392.000000,1574.990000,213121208320.000000,532622512128.000000,2584957.000000,937549.000000,27055792.000000,37533726.000000,78096610304.000000,1041573921792.000000,17925712.000000,48295609.000000
+1227194216.000000,1265893376.000000,1446747208.000000,10584064.000000,1656.040000,213240922112.000000,532622512128.000000,2584957.000000,937549.000000,27057164.000000,37534693.000000,78096610304.000000,1041825412096.000000,17926540.000000,48302649.000000
+390729144.000000,571842560.000000,1446747208.000000,7372800.000000,1725.990000,213198561280.000000,532622512128.000000,2584957.000000,937549.000000,27061501.000000,37537074.000000,78096610304.000000,1042672514048.000000,17928524.000000,48333443.000000
+455739912.000000,621420544.000000,1446747208.000000,7897088.000000,1732.280000,213132873728.000000,532622512128.000000,2584957.000000,937549.000000,27062453.000000,37537500.000000,78096610304.000000,1042872931328.000000,17928964.000000,48341835.000000
+456725080.000000,622518272.000000,1446747208.000000,7897088.000000,1732.370000,213132873728.000000,532622512128.000000,2584957.000000,937549.000000,27062482.000000,37537530.000000,78096610304.000000,1042873172992.000000,17928980.000000,48341867.000000
+772573784.000000,942538752.000000,1498736728.000000,10420224.000000,1818.590000,213180407808.000000,532622512128.000000,2584957.000000,937549.000000,27064440.000000,37538869.000000,78096610304.000000,1043243131904.000000,17930076.000000,48353603.000000
+489507264.000000,663683072.000000,1498736728.000000,7602176.000000,1887.350000,213278978048.000000,532622512128.000000,2584957.000000,937549.000000,27068330.000000,37540899.000000,78096610304.000000,1044020618240.000000,17931936.000000,48379319.000000
+429310672.000000,599752704.000000,1498736728.000000,9437184.000000,1890.680000,213141282816.000000,532622512128.000000,2584957.000000,937549.000000,27069346.000000,37541279.000000,78096610304.000000,1044226393088.000000,17932360.000000,48390509.000000
+430383296.000000,600809472.000000,1498998872.000000,9437184.000000,1890.760000,213141270528.000000,532622512128.000000,2584957.000000,937549.000000,27069525.000000,37542185.000000,78096610304.000000,1044231959552.000000,17932692.000000,48390664.000000
+790111752.000000,965566464.000000,1498998872.000000,11927552.000000,1973.200000,213257392128.000000,532622512128.000000,2584957.000000,937549.000000,27071140.000000,37543330.000000,78096610304.000000,1044505887744.000000,17933600.000000,48402110.000000
+333354800.000000,546816000.000000,1498998872.000000,7995392.000000,2042.020000,213283586048.000000,532622512128.000000,2584958.000000,937549.000000,27075265.000000,37545469.000000,78096614400.000000,1045328368640.000000,17935536.000000,48436152.000000
+433200760.000000,625025024.000000,1498998872.000000,8716288.000000,2051.100000,213148295168.000000,532622512128.000000,2584958.000000,937549.000000,27076805.000000,37546062.000000,78096614400.000000,1045650646016.000000,17936224.000000,48452016.000000
+434168024.000000,626122752.000000,1498998872.000000,8716288.000000,2051.180000,213148299264.000000,532622512128.000000,2584958.000000,937549.000000,27076813.000000,37546070.000000,78096614400.000000,1045650711552.000000,17936240.000000,48452023.000000
+820176400.000000,985989120.000000,1546184792.000000,11239424.000000,2137.790000,213297627136.000000,532622512128.000000,2584958.000000,937549.000000,27078272.000000,37547059.000000,78096614400.000000,1045931021312.000000,17937108.000000,48464819.000000
+535398056.000000,712187904.000000,1546184792.000000,8290304.000000,2206.280000,213218934784.000000,532622512128.000000,2584958.000000,937549.000000,27083047.000000,37549494.000000,78096614400.000000,1046882227200.000000,17939364.000000,48505525.000000
+400294760.000000,604413952.000000,1546184792.000000,9109504.000000,2213.750000,213158629376.000000,532622512128.000000,2584958.000000,937549.000000,27084111.000000,37549941.000000,78096614400.000000,1047106585600.000000,17939832.000000,48517072.000000
+401284864.000000,605446144.000000,1546184792.000000,9109504.000000,2213.830000,213158629376.000000,532622512128.000000,2584958.000000,937549.000000,27084150.000000,37549973.000000,78096614400.000000,1047106876416.000000,17939864.000000,48517112.000000
+1425797120.000000,1454350336.000000,1546184792.000000,15040512.000000,2294.590000,213253611520.000000,532622512128.000000,2584958.000000,937549.000000,27085590.000000,37550950.000000,78096614400.000000,1047387091968.000000,17940720.000000,48529826.000000
+526548792.000000,726745088.000000,1568289896.000000,8847360.000000,2363.300000,213332635648.000000,532622512128.000000,2584958.000000,937549.000000,27089680.000000,37552941.000000,78096614400.000000,1048204669952.000000,17942652.000000,48561710.000000
+276337960.000000,526213120.000000,1568289896.000000,8978432.000000,2374.650000,213167476736.000000,532622512128.000000,2584958.000000,937549.000000,27091561.000000,37553759.000000,78096614400.000000,1048612553728.000000,17943536.000000,48582512.000000
+277300792.000000,527220736.000000,1568289896.000000,8978432.000000,2374.740000,213167480832.000000,532622512128.000000,2584958.000000,937549.000000,27091569.000000,37553770.000000,78096614400.000000,1048612631552.000000,17943552.000000,48582518.000000
+779288624.000000,974741504.000000,1589821288.000000,11796480.000000,2461.380000,213260767232.000000,532622512128.000000,2584958.000000,937549.000000,27093516.000000,37554894.000000,78096614400.000000,1048997135360.000000,17944556.000000,48600104.000000
+561418248.000000,752861184.000000,1589821288.000000,9175040.000000,2529.440000,213249900544.000000,532622512128.000000,2584958.000000,937549.000000,27097282.000000,37556866.000000,78096614400.000000,1049737675776.000000,17946304.000000,48622781.000000
+280267520.000000,533921792.000000,1589821288.000000,9306112.000000,2540.280000,213119832064.000000,532622512128.000000,2584958.000000,937549.000000,27098565.000000,37557485.000000,78096614400.000000,1049999717376.000000,17946900.000000,48632788.000000
+281236440.000000,534962176.000000,1589821288.000000,9306112.000000,2540.370000,213119832064.000000,532622512128.000000,2584958.000000,937549.000000,27098572.000000,37557495.000000,78096614400.000000,1049999787008.000000,17946928.000000,48632794.000000
+1376463080.000000,1413808128.000000,1589821288.000000,13697024.000000,2621.410000,213264375808.000000,532622512128.000000,2584958.000000,937549.000000,27099631.000000,37558360.000000,78096614400.000000,1050186056704.000000,17947612.000000,48637620.000000
+457567584.000000,683876352.000000,1589821288.000000,9895936.000000,2693.060000,213240422400.000000,532622512128.000000,2584958.000000,937549.000000,27103430.000000,37560425.000000,78096614400.000000,1050941473792.000000,17949380.000000,48666434.000000
+464985256.000000,693854208.000000,1589821288.000000,9535488.000000,2710.330000,213126217728.000000,532622512128.000000,2584958.000000,937549.000000,27105319.000000,37561311.000000,78096614400.000000,1051321947136.000000,17950280.000000,48679510.000000
+465970776.000000,694943744.000000,1589821288.000000,9535488.000000,2710.420000,213126221824.000000,532622512128.000000,2584958.000000,937549.000000,27105326.000000,37561322.000000,78096614400.000000,1051322020864.000000,17950296.000000,48679516.000000
+847997400.000000,1035231232.000000,1620229992.000000,12582912.000000,2796.380000,213244059648.000000,532622512128.000000,2584958.000000,937549.000000,27106733.000000,37562317.000000,78096614400.000000,1051574813696.000000,17951140.000000,48686382.000000
+501270392.000000,727310336.000000,1620229992.000000,9830400.000000,2866.440000,213305135104.000000,532622512128.000000,2584958.000000,937549.000000,27110356.000000,37564284.000000,78096614400.000000,1052273525760.000000,17952856.000000,48711756.000000
+512167368.000000,740524032.000000,1620229992.000000,10518528.000000,2875.000000,213123895296.000000,532622512128.000000,2584958.000000,937549.000000,27111578.000000,37564808.000000,78096614400.000000,1052533728256.000000,17953468.000000,48719850.000000
+513103944.000000,741588992.000000,1620229992.000000,10518528.000000,2875.090000,213123899392.000000,532622512128.000000,2584958.000000,937549.000000,27111673.000000,37564852.000000,78096614400.000000,1052534301696.000000,17953500.000000,48719954.000000
+1014280088.000000,1169571840.000000,1620229992.000000,14483456.000000,2956.870000,213264281600.000000,532622512128.000000,2584958.000000,937549.000000,27112708.000000,37565701.000000,78096614400.000000,1052722246656.000000,17954212.000000,48723523.000000
+546670400.000000,774905856.000000,1638140792.000000,10747904.000000,3027.560000,213193777152.000000,532622512128.000000,2584958.000000,937549.000000,27116646.000000,37567841.000000,78096614400.000000,1053484614656.000000,17956036.000000,48750883.000000
+589415008.000000,812007424.000000,1638140792.000000,10420224.000000,3042.030000,213127442432.000000,532622512128.000000,2584958.000000,937549.000000,27117858.000000,37568367.000000,78096614400.000000,1053744092160.000000,17956628.000000,48760152.000000
+590364696.000000,813096960.000000,1638140792.000000,10420224.000000,3042.110000,213127442432.000000,532622512128.000000,2584958.000000,937549.000000,27117902.000000,37568380.000000,78096614400.000000,1053744329728.000000,17956652.000000,48760187.000000
+940831472.000000,1112113152.000000,1638140792.000000,13565952.000000,3127.700000,213298040832.000000,532622512128.000000,2584958.000000,937549.000000,27118973.000000,37569301.000000,78096614400.000000,1053929587712.000000,17957376.000000,48765612.000000
+609484760.000000,834633728.000000,1638140792.000000,10911744.000000,3197.740000,213231878144.000000,532622512128.000000,2584958.000000,937549.000000,27122823.000000,37571380.000000,78096614400.000000,1054686884864.000000,17959112.000000,48790285.000000
+483877552.000000,727556096.000000,1638140792.000000,10944512.000000,3210.250000,213121036288.000000,532622512128.000000,2584958.000000,937549.000000,27124103.000000,37571946.000000,78096614400.000000,1054945743872.000000,17959676.000000,48798858.000000
+383129504.000000,626753536.000000,1638140792.000000,12353536.000000,3211.820000,213121040384.000000,532622512128.000000,2584958.000000,937549.000000,27124109.000000,37571957.000000,78096614400.000000,1054945813504.000000,17959696.000000,48798863.000000
+1124888424.000000,1256726528.000000,1659112312.000000,13893632.000000,3300.710000,213249196032.000000,532622512128.000000,2584958.000000,937549.000000,27125554.000000,37573092.000000,78096614400.000000,1055202845696.000000,17960520.000000,48807229.000000
+557952384.000000,784318464.000000,1659112312.000000,12189696.000000,3369.920000,213188063232.000000,532622512128.000000,2584958.000000,937549.000000,27129166.000000,37575284.000000,78096614400.000000,1055880758272.000000,17962260.000000,48832430.000000
+588663512.000000,826744832.000000,1659112312.000000,11173888.000000,3384.150000,213124669440.000000,532622512128.000000,2584958.000000,937549.000000,27130393.000000,37575853.000000,78096614400.000000,1056137425920.000000,17962844.000000,48840478.000000
+589641400.000000,827875328.000000,1659112312.000000,11173888.000000,3384.250000,213124673536.000000,532622512128.000000,2584958.000000,937549.000000,27130402.000000,37575866.000000,78096614400.000000,1056137516032.000000,17962864.000000,48840486.000000
+846977384.000000,1076748288.000000,1719752584.000000,14057472.000000,3471.560000,213213261824.000000,532622512128.000000,2584958.000000,937549.000000,27131824.000000,37576916.000000,78096614400.000000,1056386532352.000000,17963708.000000,48848634.000000
+512426808.000000,767893504.000000,1719752584.000000,11468800.000000,3541.080000,213279121408.000000,532622512128.000000,2584958.000000,937549.000000,27134710.000000,37578644.000000,78096614400.000000,1056942953472.000000,17965112.000000,48867199.000000
+584315296.000000,820273152.000000,1719752584.000000,11862016.000000,3549.690000,213119369216.000000,532622512128.000000,2584958.000000,937549.000000,27135969.000000,37579260.000000,78096614400.000000,1057187324928.000000,17965696.000000,48874623.000000
+585280368.000000,821469184.000000,1719752584.000000,11862016.000000,3549.770000,213119344640.000000,532622512128.000000,2584958.000000,937549.000000,27136272.000000,37580160.000000,78096614400.000000,1057193362432.000000,17966012.000000,48874898.000000
+1507540800.000000,1557561344.000000,1719752584.000000,16154624.000000,3632.410000,213237465088.000000,532622512128.000000,2584958.000000,937549.000000,27137313.000000,37581116.000000,78096614400.000000,1057373533184.000000,17966696.000000,48879349.000000
+411947704.000000,708337664.000000,1749112712.000000,12353536.000000,3703.730000,213210718208.000000,532622512128.000000,2584958.000000,937549.000000,27140674.000000,37582968.000000,78096614400.000000,1058001609728.000000,17968232.000000,48901530.000000
+533435640.000000,777904128.000000,1749112712.000000,12255232.000000,3717.290000,213115977728.000000,532622512128.000000,2584958.000000,937549.000000,27141857.000000,37583561.000000,78096614400.000000,1058245739520.000000,17968788.000000,48909392.000000
+534479896.000000,778977280.000000,1749112712.000000,12255232.000000,3717.370000,213115977728.000000,532622512128.000000,2584958.000000,937549.000000,27141898.000000,37583599.000000,78096614400.000000,1058246063104.000000,17968808.000000,48909439.000000
+1594241616.000000,1682595840.000000,1789043608.000000,18612224.000000,3805.450000,213255421952.000000,532622512128.000000,2584958.000000,937549.000000,27143014.000000,37584629.000000,78096614400.000000,1058426401792.000000,17969512.000000,48914580.000000
+484462088.000000,760291328.000000,1789043608.000000,12615680.000000,3875.490000,213202575360.000000,532622512128.000000,2584958.000000,937549.000000,27146229.000000,37586545.000000,78096614400.000000,1059048027136.000000,17971068.000000,48938975.000000
+495689176.000000,773890048.000000,1789043608.000000,13336576.000000,3884.310000,213124366336.000000,532622512128.000000,2584958.000000,937549.000000,27147196.000000,37586948.000000,78096614400.000000,1059231749120.000000,17971468.000000,48947116.000000
+496685136.000000,774955008.000000,1789043608.000000,13336576.000000,3884.400000,213124370432.000000,532622512128.000000,2584958.000000,937549.000000,27147235.000000,37586982.000000,78096614400.000000,1059232048128.000000,17971496.000000,48947159.000000
+809592568.000000,1090535424.000000,1793237912.000000,16056320.000000,3968.730000,213232189440.000000,532622512128.000000,2584958.000000,937549.000000,27148350.000000,37587953.000000,78096614400.000000,1059410355200.000000,17972188.000000,48951584.000000
+639680360.000000,911654912.000000,1793237912.000000,13336576.000000,4040.660000,213163569152.000000,532622512128.000000,2584958.000000,937549.000000,27151575.000000,37589893.000000,78096614400.000000,1060031759360.000000,17973788.000000,48974231.000000
+583938056.000000,864559104.000000,1793237912.000000,12812288.000000,4058.550000,213119922176.000000,532622512128.000000,2584958.000000,937549.000000,27152849.000000,37590548.000000,78096614400.000000,1060291134464.000000,17974392.000000,48982624.000000
+584902784.000000,865640448.000000,1793237912.000000,12812288.000000,4058.640000,213119930368.000000,532622512128.000000,2584958.000000,937549.000000,27152950.000000,37590567.000000,78096614400.000000,1060291658752.000000,17974424.000000,48982696.000000
+1602782544.000000,1709768704.000000,1814733720.000000,18087936.000000,4147.280000,213184581632.000000,532622512128.000000,2584958.000000,937549.000000,27154349.000000,37591679.000000,78096614400.000000,1060543132672.000000,17975260.000000,48991809.000000
+544975456.000000,829300736.000000,1841295272.000000,13369344.000000,4217.010000,213221081088.000000,532622512128.000000,2584958.000000,937549.000000,27157314.000000,37593422.000000,78096614400.000000,1061093266432.000000,17976752.000000,49011953.000000
+477050680.000000,774225920.000000,1841295272.000000,13172736.000000,4231.040000,213118361600.000000,532622512128.000000,2584958.000000,937549.000000,27158584.000000,37594085.000000,78096614400.000000,1061352989696.000000,17977408.000000,49019769.000000
+478042680.000000,775274496.000000,1841295272.000000,13172736.000000,4231.130000,213118365696.000000,532622512128.000000,2584958.000000,937549.000000,27158591.000000,37594099.000000,78096614400.000000,1061353075712.000000,17977428.000000,49019775.000000
+1636370968.000000,1700904960.000000,1841295272.000000,18677760.000000,4314.910000,213236441088.000000,532622512128.000000,2584958.000000,937549.000000,27159744.000000,37594999.000000,78096614400.000000,1061531288576.000000,17978176.000000,49024384.000000
+762786824.000000,1031340032.000000,1841295272.000000,14221312.000000,4387.580000,213152624640.000000,532622512128.000000,2584958.000000,937549.000000,27163044.000000,37597084.000000,78096614400.000000,1062162457600.000000,17979784.000000,49047153.000000
+762229976.000000,1044750336.000000,1841295272.000000,13991936.000000,4402.430000,213124526080.000000,532622512128.000000,2584958.000000,937549.000000,27164028.000000,37597541.000000,78096614400.000000,1062350550016.000000,17980256.000000,49054208.000000
+763220672.000000,1045757952.000000,1841295272.000000,13991936.000000,4402.510000,213124526080.000000,532622512128.000000,2584958.000000,937549.000000,27164076.000000,37597588.000000,78096614400.000000,1062350939136.000000,17980284.000000,49054253.000000
+1495079664.000000,1614356480.000000,1841295272.000000,17629184.000000,4491.440000,213185925120.000000,532622512128.000000,2584958.000000,937549.000000,27165482.000000,37598734.000000,78096614400.000000,1062606062592.000000,17981120.000000,49062591.000000
+531728768.000000,849780736.000000,1841295272.000000,14057472.000000,4560.520000,213200994304.000000,532622512128.000000,2584958.000000,937549.000000,27168524.000000,37600615.000000,78096614400.000000,1063167980544.000000,17982564.000000,49084213.000000
+610923376.000000,907444224.000000,1841295272.000000,14417920.000000,4571.080000,213126168576.000000,532622512128.000000,2584958.000000,937549.000000,27169455.000000,37601102.000000,78096614400.000000,1063357809664.000000,17982988.000000,49091437.000000
+611979616.000000,908525568.000000,1841295272.000000,14417920.000000,4571.170000,213126172672.000000,532622512128.000000,2584958.000000,937549.000000,27169569.000000,37601147.000000,78096614400.000000,1063358489600.000000,17983024.000000,49091539.000000
+1290111912.000000,1443192832.000000,1845489576.000000,17989632.000000,4655.660000,213234733056.000000,532622512128.000000,2584958.000000,937549.000000,27170613.000000,37602060.000000,78096614400.000000,1063540626432.000000,17983772.000000,49096416.000000
+807795752.000000,1079820288.000000,1845489576.000000,14974976.000000,4727.460000,213213007872.000000,532622512128.000000,2584958.000000,937549.000000,27173624.000000,37603866.000000,78096614400.000000,1064119714816.000000,17985188.000000,49119427.000000
+587102040.000000,886644736.000000,1845489576.000000,14581760.000000,4743.470000,213124403200.000000,532622512128.000000,2584958.000000,937549.000000,27174896.000000,37604558.000000,78096614400.000000,1064373310464.000000,17985768.000000,49129905.000000
+733268344.000000,949215232.000000,1845489576.000000,16842752.000000,4744.670000,213124411392.000000,532622512128.000000,2584958.000000,937549.000000,27174999.000000,37604576.000000,78096614400.000000,1064373847040.000000,17985800.000000,49129984.000000
+896966160.000000,1175265280.000000,1887432616.000000,17334272.000000,4833.530000,213185191936.000000,532622512128.000000,2584958.000000,937549.000000,27176436.000000,37605710.000000,78096614400.000000,1064633017344.000000,17986676.000000,49138933.000000
+899851632.000000,1138147328.000000,1887432616.000000,16711680.000000,4904.140000,213195247616.000000,532622512128.000000,2584958.000000,937549.000000,27179470.000000,37607536.000000,78096614400.000000,1065201853440.000000,17988200.000000,49159720.000000
+838782504.000000,1125818368.000000,1887432616.000000,15106048.000000,4916.350000,213131501568.000000,532622512128.000000,2584958.000000,937549.000000,27180449.000000,37608066.000000,78096614400.000000,1065399264256.000000,17988672.000000,49166234.000000
+839788640.000000,1126883328.000000,1887432616.000000,15106048.000000,4916.440000,213131501568.000000,532622512128.000000,2584958.000000,937549.000000,27180457.000000,37608077.000000,78096614400.000000,1065399342080.000000,17988692.000000,49166240.000000
+1607308800.000000,1733763072.000000,1905867704.000000,18972672.000000,5002.110000,213245870080.000000,532622512128.000000,2584958.000000,937549.000000,27181621.000000,37609038.000000,78096614400.000000,1065584686080.000000,17989392.000000,49171210.000000
+898893520.000000,1186807808.000000,1984071624.000000,16809984.000000,5075.220000,213261336576.000000,532622512128.000000,2584958.000000,937549.000000,27184422.000000,37610811.000000,78096614400.000000,1066120131584.000000,17990816.000000,49191736.000000
+735638456.000000,1029971968.000000,1984071624.000000,15630336.000000,5091.460000,213135556608.000000,532622512128.000000,2584958.000000,937549.000000,27186118.000000,37611704.000000,78096614400.000000,1066438378496.000000,17991536.000000,49201080.000000
+736672552.000000,1031045120.000000,1984071624.000000,15630336.000000,5091.540000,213135556608.000000,532622512128.000000,2584958.000000,937549.000000,27186125.000000,37611724.000000,78096614400.000000,1066438489088.000000,17991556.000000,49201085.000000
+983749280.000000,1254850560.000000,1984333768.000000,18776064.000000,5176.130000,213236269056.000000,532622512128.000000,2584958.000000,937549.000000,27187269.000000,37612682.000000,78096614400.000000,1066625618944.000000,17992272.000000,49206637.000000
+544371160.000000,887308288.000000,1984858056.000000,16252928.000000,5248.610000,213354967040.000000,532622512128.000000,2584958.000000,937549.000000,27189713.000000,37614216.000000,78096614400.000000,1067096175616.000000,17993436.000000,49223476.000000
+574493912.000000,899555328.000000,1984858056.000000,15728640.000000,5266.750000,213134065664.000000,532622512128.000000,2584958.000000,937549.000000,27191644.000000,37615245.000000,78096614400.000000,1067490014208.000000,17994336.000000,49240260.000000
+755058376.000000,970735616.000000,1984858056.000000,18055168.000000,5268.180000,213134069760.000000,532622512128.000000,2584958.000000,937549.000000,27192028.000000,37616145.000000,78096614400.000000,1067496403968.000000,17994676.000000,49240598.000000
+1219295672.000000,1421574144.000000,2018674632.000000,19267584.000000,5356.760000,213192065024.000000,532622512128.000000,2584958.000000,937549.000000,27193462.000000,37617235.000000,78096614400.000000,1067762971648.000000,17995456.000000,49249925.000000
+659311592.000000,981868544.000000,2018674632.000000,16252928.000000,5427.970000,213165461504.000000,532622512128.000000,2584958.000000,937549.000000,27196757.000000,37619259.000000,78096614400.000000,1068367131648.000000,17997008.000000,49273219.000000
+508944296.000000,857333760.000000,2018674632.000000,16220160.000000,5443.680000,213141569536.000000,532622512128.000000,2584958.000000,937549.000000,27197735.000000,37619778.000000,78096614400.000000,1068570928128.000000,17997432.000000,49279436.000000
+509938312.000000,858357760.000000,2018674632.000000,16220160.000000,5443.750000,213141577728.000000,532622512128.000000,2584958.000000,937549.000000,27197772.000000,37619825.000000,78096614400.000000,1068571272192.000000,17997464.000000,49279481.000000
+1428696320.000000,1573281792.000000,2018674632.000000,19595264.000000,5528.230000,213260247040.000000,532622512128.000000,2584958.000000,937549.000000,27198915.000000,37620767.000000,78096614400.000000,1068759733248.000000,17998144.000000,49283676.000000
+907996584.000000,1243250688.000000,2018674632.000000,16809984.000000,5601.040000,213270007808.000000,532622512128.000000,2584958.000000,937549.000000,27201751.000000,37622544.000000,78096614400.000000,1069313160192.000000,17999448.000000,49306413.000000
+896814616.000000,1189060608.000000,2018674632.000000,16744448.000000,5619.110000,213141217280.000000,532622512128.000000,2584958.000000,937549.000000,27203513.000000,37623516.000000,78096614400.000000,1069646156800.000000,18000244.000000,49315896.000000
+706255432.000000,923836416.000000,2018674632.000000,18743296.000000,5621.470000,213141221376.000000,532622512128.000000,2584958.000000,937549.000000,27203521.000000,37623538.000000,78096614400.000000,1069646279680.000000,18000272.000000,49315903.000000
+1576122624.000000,1729560576.000000,2040779736.000000,20709376.000000,5709.960000,213212585984.000000,532622512128.000000,2584958.000000,937549.000000,27205034.000000,37624662.000000,78096614400.000000,1069923337216.000000,18001128.000000,49326446.000000
+772826936.000000,1052155904.000000,2040779736.000000,17989632.000000,5780.280000,213257863168.000000,532622512128.000000,2584958.000000,937549.000000,27208231.000000,37626673.000000,78096614400.000000,1070522979328.000000,18002648.000000,49350609.000000
+583845904.000000,925614080.000000,2040779736.000000,16973824.000000,5793.720000,213139959808.000000,532622512128.000000,2584958.000000,937549.000000,27209315.000000,37627311.000000,78096614400.000000,1070732567552.000000,18003144.000000,49357525.000000
+584826720.000000,926646272.000000,2040779736.000000,16973824.000000,5793.810000,213139963904.000000,532622512128.000000,2584958.000000,937549.000000,27209425.000000,37627336.000000,78096614400.000000,1070733149184.000000,18003176.000000,49357606.000000
+1227059792.000000,1456316416.000000,2087965656.000000,20119552.000000,5880.550000,213272092672.000000,532622512128.000000,2584958.000000,937549.000000,27210520.000000,37628234.000000,78096614400.000000,1070923285504.000000,18003876.000000,49363134.000000
+560448768.000000,923934720.000000,2087965656.000000,17498112.000000,5953.560000,213266386944.000000,532622512128.000000,2584958.000000,937549.000000,27213478.000000,37629947.000000,78096614400.000000,1071487095808.000000,18005272.000000,49387586.000000
+622218816.000000,961667072.000000,2087965656.000000,17367040.000000,5973.100000,213145063424.000000,532622512128.000000,2584958.000000,937549.000000,27215178.000000,37630873.000000,78096614400.000000,1071827633152.000000,18006092.000000,49397810.000000
+623221376.000000,962723840.000000,2087965656.000000,17367040.000000,5973.190000,213145067520.000000,532622512128.000000,2584958.000000,937549.000000,27215186.000000,37630892.000000,78096614400.000000,1071827743744.000000,18006112.000000,49397816.000000
+997767720.000000,1308016640.000000,2087965656.000000,20480000.000000,6057.270000,213246947328.000000,532622512128.000000,2584958.000000,937549.000000,27216374.000000,37631795.000000,78096614400.000000,1072021554176.000000,18006788.000000,49403997.000000
+777179920.000000,1109860352.000000,2087965656.000000,18219008.000000,6129.400000,213360345088.000000,532622512128.000000,2584958.000000,937549.000000,27219145.000000,37633412.000000,78096614400.000000,1072549012480.000000,18008080.000000,49425423.000000
+920671416.000000,1226686464.000000,2087965656.000000,17825792.000000,6149.970000,213148762112.000000,532622512128.000000,2584958.000000,937549.000000,27221110.000000,37634453.000000,78096614400.000000,1072933528576.000000,18008960.000000,49440247.000000
+609622712.000000,917200896.000000,2087965656.000000,19890176.000000,6151.640000,213148766208.000000,532622512128.000000,2584958.000000,937549.000000,27221145.000000,37634494.000000,78096614400.000000,1072933839872.000000,18008984.000000,49440283.000000
+1181031056.000000,1419845632.000000,2087965656.000000,20971520.000000,6240.180000,213283016704.000000,532622512128.000000,2584958.000000,937549.000000,27222260.000000,37635461.000000,78096614400.000000,1073129845760.000000,18009676.000000,49445741.000000
+817011840.000000,1199366144.000000,2087965656.000000,18186240.000000,6313.510000,213267660800.000000,532622512128.000000,2584958.000000,937549.000000,27225362.000000,37637344.000000,78096614400.000000,1073710244864.000000,18011140.000000,49468279.000000
+840991600.000000,1166082048.000000,2087965656.000000,18219008.000000,6333.820000,213154385920.000000,532622512128.000000,2584958.000000,937549.000000,27227065.000000,37638197.000000,78096614400.000000,1074053940224.000000,18011892.000000,49478158.000000
+841949552.000000,1167327232.000000,2087965656.000000,18219008.000000,6333.910000,213154385920.000000,532622512128.000000,2584958.000000,937549.000000,27227171.000000,37638246.000000,78096614400.000000,1074054607872.000000,18011928.000000,49478250.000000
+1405811896.000000,1584586752.000000,2087965656.000000,21692416.000000,6421.360000,213269045248.000000,532622512128.000000,2584958.000000,937549.000000,27228294.000000,37639236.000000,78096614400.000000,1074252690432.000000,18012660.000000,49483317.000000
+1097496968.000000,1414053888.000000,2087965656.000000,19169280.000000,6493.200000,213354270720.000000,532622512128.000000,2584958.000000,937549.000000,27231359.000000,37641020.000000,78096614400.000000,1074831627264.000000,18014104.000000,49508319.000000
+678211800.000000,1029341184.000000,2087965656.000000,18546688.000000,6514.510000,213150515200.000000,532622512128.000000,2584958.000000,937549.000000,27233113.000000,37642033.000000,78096614400.000000,1075184956416.000000,18014916.000000,49520046.000000
+679210600.000000,1030373376.000000,2087965656.000000,18546688.000000,6514.590000,213150515200.000000,532622512128.000000,2584958.000000,937549.000000,27233216.000000,37642046.000000,78096614400.000000,1075185472512.000000,18014956.000000,49520120.000000
diff --git a/test/stress/cases/istio/testdata/metrics/intermediate.csv 
b/test/stress/cases/istio/testdata/metrics/intermediate.csv
index ff82bf9e..4fbfbe32 100644
--- a/test/stress/cases/istio/testdata/metrics/intermediate.csv
+++ b/test/stress/cases/istio/testdata/metrics/intermediate.csv
@@ -1,9 +1,9 @@
-weighted_io,rate[5m],266.9066666666667,264.54,312.63,312.74333333333334,293.16333333333336,285.31333333333333,331.4766666666667,336.87,319.20666666666665,296.89666666666665,346.0133333333333,351.11,331.19666666666666,302.68333333333334,359.6,365.92333333333335,343.1933333333333,321.7133333333333,387.70666666666665,395.1566666666667,367.77666666666664,334.1166666666667,411.74333333333334,418.0733333333333,401.15,371.7866666666667,434.76666666666665,443.5,427.05333333333334,396.67333333333
 [...]
-io_time,rate[5m],107.24,108.04,114.6,114.50666666666666,98.14666666666666,106.28,116.16,116.77333333333333,102.65333333333334,108.65333333333334,117.90666666666667,117.56,102.36,108.97333333333333,118.10666666666667,118.78666666666666,103.28,110.6,121.16,121.90666666666667,106.09333333333333,114.92,123.76,124.45333333333333,110.56,116.65333333333334,123.64,124.4,110.22666666666667,116.28,124.30666666666667,125.02666666666667,112.24,117.76,125.85333333333334,126.6,112.73333333333333,116.4
 [...]
-write_bytes,rate[5m],12175837.866666667,12012612.266666668,13333094.4,13333585.92,12158730.24,11957725.866666667,13921949.013333334,14166507.52,13197967.36,12534565.546666667,14426821.973333333,14669742.08,13545117.013333334,12860197.546666667,14759867.733333332,15035924.48,13782016,13153962.666666666,15361597.44,15654475.093333334,14331671.893333333,13803874.986666666,16151975.253333334,16460513.28,15535786.666666666,14552814.933333334,16415989.76,16743669.76,15829688.32,14964162.56,168
 [...]
-read_bytes,rate[5m],286.72,286.72,273.06666666666666,13.653333333333334,13.653333333333334,0,0,0,0,54.61333333333334,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,68.26666666666667,13.653333333333334,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,
 [...]
-merged_write_count,rate[5m],250.47333333333333,243.70333333333335,257.0133333333333,256.99333333333334,218.29333333333332,238.62,259.81666666666666,260.2866666666667,227.01333333333332,243.17333333333335,262.7133333333333,260.33666666666664,225.05,240.99666666666667,260.07666666666665,260.3933333333333,224.13666666666666,241.51333333333332,262.1566666666667,262.5366666666667,226.45666666666668,250.69666666666666,267.2733333333333,267.5833333333333,234.97333333333333,250.41666666666666,26
 [...]
-write_count,rate[5m],115.44333333333333,111.93333333333334,119.84666666666666,119.96666666666667,105.55666666666667,109.98333333333333,122.14666666666666,123.21333333333334,110.99666666666667,113.17666666666666,124.44333333333333,125.16666666666667,111.15666666666667,112.98666666666666,124.77666666666667,126.03333333333333,111.66666666666667,114.67333333333333,127.41,128.81,114.18666666666667,118.22333333333333,130.96333333333334,132.25666666666666,120.60666666666667,121.47333333333333,1
 [...]
-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.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333334,0.6033333333333
 [...]
-read_count,rate[5m],0.023333333333333334,0.023333333333333334,0.02,0.0033333333333333335,0.0033333333333333335,0,0,0,0,0.013333333333333334,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.016666666666666666,0.01666666666666
 [...]
-process_cpu_seconds_total,rate[5m],2.0383333333333336,2.0906,2.2426666666666666,2.2426999999999997,1.9484333333333337,2.0785666666666667,2.292966666666667,2.301233333333333,2.0336,2.1216,2.3249666666666666,2.336033333333333,2.0478000000000005,2.1414000000000004,2.342533333333333,2.3531666666666666,2.0532,2.1546666666666665,2.379433333333333,2.3916333333333335,2.086933333333333,2.2358000000000002,2.4389,2.4459,2.189133333333333,2.2752666666666665,2.4403333333333337,2.446,2.185666666666666
 [...]
+weighted_io,rate[5m],460.47333333333336,518.8733333333333,508.00666666666666,488.93666666666667,459.3666666666667,538.6733333333333,559.81,544.38,517.9766666666667,582.1466666666666,606.8533333333334,588.19,544.8533333333334,626.34,654.71,632.47,595.2033333333334,670.83,697.2033333333334,670.2566666666667,608.3266666666667,695.0866666666667,722.9333333333333,700.3766666666667,655.09,719.0566666666666,756.24,725.46,685.5333333333333,774.2766666666666,827.13,796.9733333333334,748.116666666
 [...]
+io_time,rate[5m],63.28,61.093333333333334,56.42666666666667,52.733333333333334,48.946666666666665,55.013333333333335,54.82666666666667,52.24,49.4,54.36,55.46666666666667,52.013333333333335,47.82666666666667,53.76,55.42666666666667,52.16,49.36,54.29333333333334,55.586666666666666,52.52,48.653333333333336,54.4,55.77333333333333,52.52,49.70666666666666,54.84,56.18666666666667,53.68,50.50666666666667,55.77333333333333,58.013333333333335,55.04,51.16,56.946666666666665,58.44,54.733333333333334
 [...]
+write_bytes,rate[5m],17993045.333333332,19056039.253333334,18139777.706666667,17212880.213333335,16082916.693333333,18310471.68,18820669.44,18036872.533333335,16937929.386666667,18948983.466666665,19508852.053333335,18559126.186666667,16989975.893333334,19288623.786666665,20035938.98666667,19117602.133333333,17870288.213333335,19933252.266666666,20564527.786666665,19598445.226666667,17900625.92,20341445.973333333,21008424.96,20001532.586666666,18843320.32,20905697.28,21590644.053333335,2
 [...]
+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,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,13.653333333333334,0,0,0,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],95.79,86.84333333333333,78.44,72.61,65.7,73.37666666666667,71.51,67.64333333333333,62.93666666666667,69.45,70.67666666666666,66.31333333333333,60.64666666666667,68.01666666666667,69.95333333333333,65.60666666666667,61.163333333333334,66.65333333333334,68,64.12333333333333,58.776666666666664,65.73666666666666,67.02666666666667,63.06,59.343333333333334,64.99,66.14666666666666,64.75666666666666,60.67333333333333,66.52,68.45,64.54333333333334,59.45333333333333,65.
 [...]
+write_count,rate[5m],111,111.2,103.87333333333333,97.97666666666667,90.20666666666666,102.29666666666667,104.11,99.28666666666666,92.71333333333334,103.37666666666667,105.98666666666666,100.52333333333333,91.78333333333333,103.85,107.54666666666667,102.23,95.28,105.87,108.83,103.61666666666666,94.51333333333334,107.13333333333334,110.17333333333333,104.64666666666666,98.27333333333333,108.71333333333334,111.97333333333333,106.73666666666666,98.97,110.08333333333333,115.19,110.02,101.0833
 [...]
+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,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.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333335,0.0033333333333333
 [...]
+process_cpu_seconds_total,rate[5m],2.6710999999999996,2.6657333333333333,2.5520333333333336,2.2728333333333333,2.3266999999999998,2.5548999999999995,2.5736999999999997,2.3106333333333327,2.3730666666666664,2.583566666666667,2.5992333333333333,2.3164666666666665,2.3691999999999998,2.593566666666667,2.6215,2.3499,2.4054,2.6188,2.637466666666667,2.356466666666667,2.403766666666667,2.6283333333333334,2.6489999999999996,2.3749,2.4316999999999998,2.641833333333333,2.6527000000000007,2.36563333
 [...]
diff --git a/test/stress/cases/istio/testdata/metrics/result.csv 
b/test/stress/cases/istio/testdata/metrics/result.csv
index 00b3dbed..10ea3b68 100644
--- a/test/stress/cases/istio/testdata/metrics/result.csv
+++ b/test/stress/cases/istio/testdata/metrics/result.csv
@@ -1,16 +1,16 @@
 Metric Name, Min, Max, Mean, Median, P90, P95, P98, P99
-go_memstats_alloc_bytes, 144503392.000000, 1252014544.000000, 
533194949.707006, 515922896.000000, 863701068.000000, 1016009468.000000, 
1060432244.000000, 1109959040.000000
-go_memstats_heap_inuse_bytes, 250183680.000000, 1309556736.000000, 
701603318.216560, 709967872.000000, 1042378752.000000, 1105199104.000000, 
1180516352.000000, 1211912192.000000
-go_memstats_sys_bytes, 1149135880.000000, 1543301208.000000, 
1250226991.337580, 1171811336.000000, 1449237576.000000, 1515645016.000000, 
1530587224.000000, 1530587224.000000
-go_memstats_stack_inuse_bytes, 2916352.000000, 22544384.000000, 
11434571.006369, 11337728.000000, 17956864.000000, 18907136.000000, 
20611072.000000, 22052864.000000
-process_cpu_seconds_total, 1.948433, 2.784233, 2.473533, 2.480917, 2.747350, 
2.762550, 2.776167, 2.780333
-disk_used, 215987310592.000000, 216453967872.000000, 216290951611.515930, 
216265412608.000000, 216422912000.000000, 216435509248.000000, 
216448153600.000000, 216452474880.000000
+go_memstats_alloc_bytes, 155663808.000000, 1636370968.000000, 
668034861.677419, 585280368.000000, 1192113260.000000, 1427246720.000000, 
1585182120.000000, 1605045672.000000
+go_memstats_heap_inuse_bytes, 296435712.000000, 1733763072.000000, 
879246831.483871, 864559104.000000, 1360912384.000000, 1565421568.000000, 
1691750400.000000, 1719664640.000000
+go_memstats_sys_bytes, 989058520.000000, 2087965656.000000, 1677694351.845161, 
1638140792.000000, 2064372696.000000, 2087965656.000000, 2087965656.000000, 
2087965656.000000
+go_memstats_stack_inuse_bytes, 3211264.000000, 21692416.000000, 
12091180.593548, 11960320.000000, 18382848.000000, 19218432.000000, 
20299776.000000, 20840448.000000
+process_cpu_seconds_total, 2.272833, 2.976800, 2.644399, 2.640417, 2.865667, 
2.911000, 2.938917, 2.953100
+disk_used, 212879372288.000000, 213360345088.000000, 213166692100.954834, 
213148762112.000000, 213265381376.000000, 213283301376.000000, 
213318885376.000000, 213354618880.000000
 disk_total, 532622512128.000000, 532622512128.000000, 532622512128.000000, 
532622512128.000000, 532622512128.000000, 532622512128.000000, 
532622512128.000000, 532622512128.000000
-read_count, 0.000000, 0.856667, 0.145556, 0.016667, 0.856667, 0.856667, 
0.856667, 0.856667
-merged_read_count, 0.000000, 0.603333, 0.083068, 0.000000, 0.603333, 0.603333, 
0.603333, 0.603333
-write_count, 105.556667, 139.776667, 126.035966, 125.403333, 135.776667, 
136.976667, 138.206667, 138.383333
-merged_write_count, 217.376667, 273.723333, 250.497319, 254.925000, 
267.046667, 269.993333, 271.331667, 272.438333
-read_bytes, 0.000000, 13817.173333, 2019.605024, 68.266667, 13817.173333, 
13817.173333, 13817.173333, 13817.173333
-write_bytes, 11957725.866667, 18027383.466667, 15585415.939710, 
15669131.946667, 16918766.933333, 17439505.066667, 17689251.840000, 
17725385.386667
-io_time, 98.146667, 126.600000, 117.581546, 117.793333, 124.973333, 
125.560000, 126.260000, 126.300000
-weighted_io, 264.540000, 536.200000, 407.836111, 410.760000, 483.176667, 
504.606667, 518.133333, 522.120000
+read_count, 0.000000, 0.003333, 0.000466, 0.000000, 0.003333, 0.003333, 
0.003333, 0.003333
+merged_read_count, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 0.000000, 
0.000000, 0.000000
+write_count, 80.750000, 120.276667, 98.128676, 96.891667, 112.166667, 
114.936667, 118.011667, 119.271667
+merged_write_count, 51.716667, 95.790000, 61.027990, 59.808333, 67.031667, 
70.315000, 75.908333, 82.641667
+read_bytes, 0.000000, 13.653333, 1.907451, 0.000000, 13.653333, 13.653333, 
13.653333, 13.653333
+write_bytes, 14877204.480000, 23755147.946667, 18499353.399216, 
18014958.933333, 22047948.800000, 22672254.293333, 23202747.733333, 
23547699.200000
+io_time, 41.626667, 63.280000, 49.824314, 48.880000, 56.306667, 57.906667, 
59.573333, 60.413333
+weighted_io, 459.366667, 969.736667, 657.124093, 619.871667, 859.045000, 
922.218333, 946.415000, 959.581667

Reply via email to