diff --git a/site/.htmltest.yml b/site/.htmltest.yml index c51da7eb55..99d5af9b82 100644 --- a/site/.htmltest.yml +++ b/site/.htmltest.yml @@ -18,5 +18,7 @@ IgnoreURLs: - "youtu.be" - "youtube.com" - "cassandra.apache.org" +- "slideshare.net" +- "meetup.com" - "github.com/m3db/m3/tree/docs-test/site/content/docs" - "github.com/m3db/m3/tree/master/site/content/docs" \ No newline at end of file diff --git a/src/aggregator/aggregation/gauge.go b/src/aggregator/aggregation/gauge.go index 4e381c81c2..ba122e115a 100644 --- a/src/aggregator/aggregation/gauge.go +++ b/src/aggregator/aggregation/gauge.go @@ -27,10 +27,6 @@ import ( "github.com/m3db/m3/src/metrics/aggregation" ) -const ( - minFloat64 = -math.MaxFloat64 -) - // Gauge aggregates gauge values. type Gauge struct { Options @@ -48,8 +44,8 @@ type Gauge struct { func NewGauge(opts Options) Gauge { return Gauge{ Options: opts, - max: minFloat64, - min: math.MaxFloat64, + max: math.NaN(), + min: math.NaN(), } } @@ -65,12 +61,17 @@ func (g *Gauge) Update(timestamp time.Time, value float64) { g.Options.Metrics.Gauge.IncValuesOutOfOrder() } - g.sum += value g.count++ - if g.max < value { + + if math.IsNaN(value) { + return + } + + g.sum += value + if math.IsNaN(g.max) || g.max < value { g.max = value } - if g.min > value { + if math.IsNaN(g.min) || g.min > value { g.min = value } @@ -108,10 +109,14 @@ func (g *Gauge) Stdev() float64 { } // Min returns the minimum gauge value. -func (g *Gauge) Min() float64 { return g.min } +func (g *Gauge) Min() float64 { + return g.min +} // Max returns the maximum gauge value. -func (g *Gauge) Max() float64 { return g.max } +func (g *Gauge) Max() float64 { + return g.max +} // ValueOf returns the value for the aggregation type. func (g *Gauge) ValueOf(aggType aggregation.Type) float64 { diff --git a/src/aggregator/aggregation/gauge_test.go b/src/aggregator/aggregation/gauge_test.go index bcf37e006f..1461bcdb21 100644 --- a/src/aggregator/aggregation/gauge_test.go +++ b/src/aggregator/aggregation/gauge_test.go @@ -21,6 +21,7 @@ package aggregation import ( + "math" "testing" "time" @@ -42,6 +43,17 @@ func TestGaugeDefaultAggregationType(t *testing.T) { require.Equal(t, 100.0, g.ValueOf(aggregation.Count)) require.Equal(t, 50.5, g.ValueOf(aggregation.Mean)) require.Equal(t, 0.0, g.ValueOf(aggregation.SumSq)) + require.Equal(t, 100.0, g.ValueOf(aggregation.Max)) + require.Equal(t, 1.0, g.ValueOf(aggregation.Min)) + + g = NewGauge(NewOptions(instrument.NewOptions())) + require.Equal(t, 0.0, g.Last()) + require.Equal(t, 0.0, g.ValueOf(aggregation.Last)) + require.Equal(t, 0.0, g.ValueOf(aggregation.Count)) + require.Equal(t, 0.0, g.ValueOf(aggregation.Mean)) + require.Equal(t, 0.0, g.ValueOf(aggregation.SumSq)) + require.True(t, math.IsNaN(g.ValueOf(aggregation.Max))) + require.True(t, math.IsNaN(g.ValueOf(aggregation.Min))) } func TestGaugeCustomAggregationType(t *testing.T) { @@ -80,6 +92,33 @@ func TestGaugeCustomAggregationType(t *testing.T) { require.False(t, aggType.IsValidForGauge()) } } + + g = NewGauge(opts) + require.Equal(t, 0.0, g.Last()) + for aggType := range aggregation.ValidTypes { + v := g.ValueOf(aggType) + switch aggType { + case aggregation.Last: + require.Equal(t, 0.0, v) + case aggregation.Min: + require.True(t, math.IsNaN(v)) + case aggregation.Max: + require.True(t, math.IsNaN(v)) + case aggregation.Mean: + require.Equal(t, 0.0, v) + case aggregation.Count: + require.Equal(t, 0.0, v) + case aggregation.Sum: + require.Equal(t, 0.0, v) + case aggregation.SumSq: + require.Equal(t, 0.0, v) + case aggregation.Stdev: + require.InDelta(t, 0.0, v, 0.0) + default: + require.Equal(t, 0.0, v) + require.False(t, aggType.IsValidForGauge()) + } + } } func TestGaugeLastOutOfOrderValues(t *testing.T) { diff --git a/src/aggregator/client/queue.go b/src/aggregator/client/queue.go index 355c16d937..b3aadeb2b3 100644 --- a/src/aggregator/client/queue.go +++ b/src/aggregator/client/queue.go @@ -218,6 +218,7 @@ func (q *queue) writeAndReset() { if err := q.writeFn(q.buf); err != nil { q.log.Error("error writing data", zap.Int("buffer_size", len(q.buf)), + zap.String("target_instance_id", q.instance.ID()), zap.String("target_instance", q.instance.Endpoint()), zap.Error(err), ) diff --git a/src/cmd/tools/dtest/docker/harness/carbon_test.go b/src/cmd/tools/dtest/docker/harness/carbon_test.go index 6bc35a9ee7..c353ed0b5f 100644 --- a/src/cmd/tools/dtest/docker/harness/carbon_test.go +++ b/src/cmd/tools/dtest/docker/harness/carbon_test.go @@ -32,7 +32,7 @@ import ( ) func findVerifier(expected string) resources.ResponseVerifier { - return func(status int, s string, err error) error { + return func(status int, _ map[string][]string, s string, err error) error { if err != nil { return err } @@ -55,7 +55,7 @@ func renderVerifier(v float64) resources.ResponseVerifier { Datapoints [][]float64 `json:"datapoints"` } - return func(status int, s string, err error) error { + return func(status int, _ map[string][]string, s string, err error) error { if err != nil { return err } diff --git a/src/cmd/tools/dtest/docker/harness/query_api_test.go b/src/cmd/tools/dtest/docker/harness/query_api_test.go index dd389afd66..5ef751b2ec 100644 --- a/src/cmd/tools/dtest/docker/harness/query_api_test.go +++ b/src/cmd/tools/dtest/docker/harness/query_api_test.go @@ -71,7 +71,7 @@ func testInvalidQueryReturns400(t *testing.T, tests []urlTest) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - assert.NoError(t, coord.RunQuery(verifyStatus(400), tt.url), "for query '%v'", tt.url) + assert.NoError(t, coord.RunQuery(verifyResponse(400), tt.url), "for query '%v'", tt.url) }) } } @@ -129,8 +129,8 @@ func queryString(params map[string]string) string { return strings.Join(p, "&") } -func verifyStatus(expectedStatus int) resources.ResponseVerifier { - return func(status int, resp string, err error) error { +func verifyResponse(expectedStatus int) resources.ResponseVerifier { + return func(status int, headers map[string][]string, resp string, err error) error { if err != nil { return err } @@ -139,6 +139,12 @@ func verifyStatus(expectedStatus int) resources.ResponseVerifier { return fmt.Errorf("expeceted %v status code, got %v", expectedStatus, status) } + if contentType, ok := headers["Content-Type"]; !ok { + return fmt.Errorf("missing Content-Type header") + } else if len(contentType) != 1 || contentType[0] != "application/json" { + return fmt.Errorf("expected json content type, got %v", contentType) + } + return nil } } diff --git a/src/cmd/tools/dtest/docker/harness/resources/coordinator.go b/src/cmd/tools/dtest/docker/harness/resources/coordinator.go index d2d320a446..2bafb8506f 100644 --- a/src/cmd/tools/dtest/docker/harness/resources/coordinator.go +++ b/src/cmd/tools/dtest/docker/harness/resources/coordinator.go @@ -55,7 +55,7 @@ var ( ) // ResponseVerifier is a function that checks if the query response is valid. -type ResponseVerifier func(int, string, error) error +type ResponseVerifier func(int, map[string][]string, string, error) error // Coordinator is a wrapper for a coordinator. It provides a wrapper on HTTP // endpoints that expose cluster management APIs as well as read and write @@ -363,7 +363,7 @@ func (c *coordinator) query( defer resp.Body.Close() b, err := ioutil.ReadAll(resp.Body) - return verifier(resp.StatusCode, string(b), err) + return verifier(resp.StatusCode, resp.Header, string(b), err) } func (c *coordinator) RunQuery( diff --git a/src/dbnode/generated/mocks/generate.go b/src/dbnode/generated/mocks/generate.go index 3869d17577..de23a9efbf 100644 --- a/src/dbnode/generated/mocks/generate.go +++ b/src/dbnode/generated/mocks/generate.go @@ -20,7 +20,7 @@ // mockgen rules for generating mocks for exported interfaces (reflection mode) -//go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs CrossBlockReader,CrossBlockIterator,DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/fs_mock.go" +//go:generate sh -c "mockgen -package=fs $PACKAGE/src/dbnode/persist/fs DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter | genclean -pkg $PACKAGE/src/dbnode/persist/fs -out $GOPATH/src/$PACKAGE/src/dbnode/persist/fs/fs_mock.go" //go:generate sh -c "mockgen -package=xio $PACKAGE/src/dbnode/x/xio SegmentReader,SegmentReaderPool | genclean -pkg $PACKAGE/src/dbnode/x/xio -out $GOPATH/src/$PACKAGE/src/dbnode/x/xio/io_mock.go" //go:generate sh -c "mockgen -package=digest -destination=$GOPATH/src/$PACKAGE/src/dbnode/digest/digest_mock.go $PACKAGE/src/dbnode/digest ReaderWithDigest" //go:generate sh -c "mockgen -package=series $PACKAGE/src/dbnode/storage/series DatabaseSeries,QueryableBlockRetriever | genclean -pkg $PACKAGE/src/dbnode/storage/series -out $GOPATH/src/$PACKAGE/src/dbnode/storage/series/series_mock.go" diff --git a/src/dbnode/persist/fs/cross_block_iterator.go b/src/dbnode/persist/fs/cross_block_iterator.go deleted file mode 100644 index fcae8851cf..0000000000 --- a/src/dbnode/persist/fs/cross_block_iterator.go +++ /dev/null @@ -1,94 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package fs - -import ( - "bytes" - - "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/ts" - xtime "github.com/m3db/m3/src/x/time" -) - -type crossBlockIterator struct { - idx int - exhausted bool - current encoding.ReaderIterator - byteReader *bytes.Reader - records []BlockRecord -} - -// NewCrossBlockIterator creates a new CrossBlockIterator. -func NewCrossBlockIterator(pool encoding.ReaderIteratorPool) CrossBlockIterator { - c := &crossBlockIterator{current: pool.Get(), byteReader: bytes.NewReader(nil)} - c.Reset(nil) - return c -} - -func (c *crossBlockIterator) Next() bool { - if c.exhausted { - return false - } - - if c.idx >= 0 && c.current.Next() { - return true - } - - // NB: clear previous. - if c.idx >= 0 { - if c.current.Err() != nil { - c.exhausted = true - return false - } - } - - c.idx++ - if c.idx >= len(c.records) { - c.exhausted = true - return false - } - - c.byteReader.Reset(c.records[c.idx].Data) - c.current.Reset(c.byteReader, nil) - - // NB: rerun using the next record. - return c.Next() -} - -func (c *crossBlockIterator) Current() (ts.Datapoint, xtime.Unit, ts.Annotation) { - return c.current.Current() -} - -func (c *crossBlockIterator) Reset(records []BlockRecord) { - c.idx = -1 - c.records = records - c.exhausted = false - c.byteReader.Reset(nil) -} - -func (c *crossBlockIterator) Close() { - c.Reset(nil) - c.current.Close() -} - -func (c *crossBlockIterator) Err() error { - return c.current.Err() -} diff --git a/src/dbnode/persist/fs/cross_block_iterator_test.go b/src/dbnode/persist/fs/cross_block_iterator_test.go deleted file mode 100644 index 111580580c..0000000000 --- a/src/dbnode/persist/fs/cross_block_iterator_test.go +++ /dev/null @@ -1,177 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package fs - -import ( - "fmt" - "io" - "io/ioutil" - "testing" - "time" - - "github.com/m3db/m3/src/dbnode/encoding" - "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/dbnode/ts" - xtest "github.com/m3db/m3/src/x/test" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/golang/mock/gomock" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func TestCrossBlockIterator(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - reader := encoding.NewMockReaderIterator(ctrl) - - iterPool := encoding.NewMockReaderIteratorPool(ctrl) - iterPool.EXPECT().Get().Return(reader) - - iter := NewCrossBlockIterator(iterPool) - assert.False(t, iter.Next()) - - count := 3 - iterCount := 5 - startTime := time.Now().Truncate(time.Hour) - start := startTime - records := make([]BlockRecord, 0, count) - for i := 0; i < count; i++ { - byteString := fmt.Sprint(i) - records = append(records, BlockRecord{ - Data: []byte(byteString), - }) - - reader.EXPECT().Reset(gomock.Any(), nil).Do( - func(r io.Reader, _ namespace.SchemaDescr) { - b, err := ioutil.ReadAll(r) - require.NoError(t, err) - assert.Equal(t, byteString, string(b)) - }) - - for j := 0; j < iterCount; j++ { - reader.EXPECT().Next().Return(true) - reader.EXPECT().Current().Return(ts.Datapoint{ - Value: float64(j), - Timestamp: start, - }, xtime.Second, nil) - start = start.Add(time.Minute) - } - - reader.EXPECT().Next().Return(false) - reader.EXPECT().Err().Return(nil) - } - - iter.Reset(records) - i := 0 - for iter.Next() { - dp, _, _ := iter.Current() - // NB: iterator values should go [0,1,...,iterCount] for each block record. - assert.Equal(t, float64(i%iterCount), dp.Value) - // NB: time should be constantly increasing per value. - assert.Equal(t, startTime.Add(time.Minute*time.Duration(i)), dp.Timestamp) - i++ - } - - assert.Equal(t, count*iterCount, i) - - reader.EXPECT().Err().Return(errExpected) - assert.Equal(t, errExpected, iter.Err()) - reader.EXPECT().Close() - iter.Close() -} - -func TestFailingCrossBlockIterator(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - reader := encoding.NewMockReaderIterator(ctrl) - - iterPool := encoding.NewMockReaderIteratorPool(ctrl) - iterPool.EXPECT().Get().Return(reader) - - iter := NewCrossBlockIterator(iterPool) - assert.False(t, iter.Next()) - - count := 4 - iterCount := 5 - remaining := 12 - startTime := time.Now().Truncate(time.Hour) - start := startTime - records := make([]BlockRecord, 0, count) - for i := 0; i < count; i++ { - byteString := fmt.Sprint(i) - data := []byte(byteString) - - if remaining == 0 { - records = append(records, BlockRecord{ - Data: data, - }) - continue - } - - records = append(records, BlockRecord{ - Data: data, - }) - - reader.EXPECT().Reset(gomock.Any(), nil).Do( - func(r io.Reader, _ namespace.SchemaDescr) { - b, err := ioutil.ReadAll(r) - require.NoError(t, err) - assert.Equal(t, byteString, string(b)) - }) - - for j := 0; remaining > 0 && j < iterCount; j++ { - reader.EXPECT().Next().Return(true) - reader.EXPECT().Current().Return(ts.Datapoint{ - Value: float64(j), - Timestamp: start, - }, xtime.Second, nil) - start = start.Add(time.Minute) - remaining-- - } - - reader.EXPECT().Next().Return(false) - if remaining == 0 { - reader.EXPECT().Err().Return(errExpected).Times(2) - } else { - reader.EXPECT().Err().Return(nil) - } - } - - iter.Reset(records) - i := 0 - for iter.Next() { - dp, _, _ := iter.Current() - // NB: iterator values should go [0,1,...,iterCount] for each block record. - assert.Equal(t, float64(i%iterCount), dp.Value) - // NB: time should be constantly increasing per value. - assert.Equal(t, startTime.Add(time.Minute*time.Duration(i)), dp.Timestamp) - i++ - } - - assert.Equal(t, 12, i) - - assert.Equal(t, errExpected, iter.Err()) - reader.EXPECT().Close() - iter.Close() -} diff --git a/src/dbnode/persist/fs/cross_block_reader.go b/src/dbnode/persist/fs/cross_block_reader.go deleted file mode 100644 index 6184bde98c..0000000000 --- a/src/dbnode/persist/fs/cross_block_reader.go +++ /dev/null @@ -1,234 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package fs - -import ( - "bytes" - "container/heap" - "errors" - "io" - "time" - - "github.com/m3db/m3/src/dbnode/ts" - xerrors "github.com/m3db/m3/src/x/errors" - "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" -) - -var ( - errReaderNotOrderedByIndex = errors.New("crossBlockReader can only use DataFileSetReaders ordered by index") - errUnorderedDataFileSetReaders = errors.New("dataFileSetReaders are not ordered by time") -) - -type crossBlockReader struct { - dataFileSetReaders []DataFileSetReader - pendingReaderIndices []int - preAllocatedEntries []*minHeapEntry - minHeap minHeap - - iOpts instrument.Options - - id ident.BytesID - encodedTags ts.EncodedTags - records []BlockRecord - err error -} - -// NewCrossBlockReader constructs a new CrossBlockReader based on given DataFileSetReaders. -// DataFileSetReaders must be configured to return the data in the order of index, and must be -// provided in a slice sorted by block start time. -// Callers are responsible for closing the DataFileSetReaders. -func NewCrossBlockReader(dataFileSetReaders []DataFileSetReader, iOpts instrument.Options) (CrossBlockReader, error) { - var previousStart time.Time - for _, dataFileSetReader := range dataFileSetReaders { - if !dataFileSetReader.StreamingEnabled() { - return nil, errReaderNotOrderedByIndex - } - currentStart := dataFileSetReader.Range().Start - if !currentStart.After(previousStart) { - return nil, errUnorderedDataFileSetReaders - } - previousStart = currentStart - } - - pendingReaderIndices := make([]int, len(dataFileSetReaders)) - preAllocatedEntries := make([]*minHeapEntry, len(dataFileSetReaders)) - for i := range dataFileSetReaders { - pendingReaderIndices[i] = i - preAllocatedEntries[i] = &minHeapEntry{} - } - - return &crossBlockReader{ - dataFileSetReaders: dataFileSetReaders, - pendingReaderIndices: pendingReaderIndices, - minHeap: make([]*minHeapEntry, 0, len(dataFileSetReaders)), - preAllocatedEntries: preAllocatedEntries, - records: make([]BlockRecord, 0, len(dataFileSetReaders)), - iOpts: iOpts, - }, nil -} - -func (r *crossBlockReader) Next() bool { - if r.err != nil { - return false - } - - // use empty var in inner loop with "for i := range" to have compiler use memclr optimization - // see: https://codereview.appspot.com/137880043 - var emptyRecord BlockRecord - for i := range r.records { - r.records[i] = emptyRecord - } - r.records = r.records[:0] - - if len(r.pendingReaderIndices) == 0 { - return false - } - - var err error - for _, readerIndex := range r.pendingReaderIndices { - *r.preAllocatedEntries[readerIndex], err = r.readFromDataFileSet(readerIndex) - if err == io.EOF { - // Will no longer read from this one. - continue - } else if err != nil { - r.err = err - return false - } else { - heap.Push(&r.minHeap, r.preAllocatedEntries[readerIndex]) - } - } - - r.pendingReaderIndices = r.pendingReaderIndices[:0] - - if len(r.minHeap) == 0 { - return false - } - - firstEntry := heap.Pop(&r.minHeap).(*minHeapEntry) - - r.id = firstEntry.id - r.encodedTags = firstEntry.encodedTags - - r.records = append(r.records, BlockRecord{firstEntry.data, firstEntry.checksum}) - - // We have consumed an entry from this dataFileSetReader, so need to schedule a read from it on the next Next(). - r.pendingReaderIndices = append(r.pendingReaderIndices, firstEntry.dataFileSetReaderIndex) - - // As long as id stays the same across the blocks, accumulate records for this id/tags. - for len(r.minHeap) > 0 && bytes.Equal(r.minHeap[0].id.Bytes(), firstEntry.id.Bytes()) { - nextEntry := heap.Pop(&r.minHeap).(*minHeapEntry) - - r.records = append(r.records, BlockRecord{nextEntry.data, nextEntry.checksum}) - - // We have consumed an entry from this dataFileSetReader, so need to schedule a read from it on the next Next(). - r.pendingReaderIndices = append(r.pendingReaderIndices, nextEntry.dataFileSetReaderIndex) - } - - return true -} - -func (r *crossBlockReader) Current() (ident.BytesID, ts.EncodedTags, []BlockRecord) { - return r.id, r.encodedTags, r.records -} - -func (r *crossBlockReader) readFromDataFileSet(index int) (minHeapEntry, error) { - id, encodedTags, data, checksum, err := r.dataFileSetReaders[index].StreamingRead() - - if err == io.EOF { - return minHeapEntry{}, err - } - - if err != nil { - multiErr := xerrors.NewMultiError(). - Add(err). - Add(r.Close()) - return minHeapEntry{}, multiErr.FinalError() - } - - return minHeapEntry{ - dataFileSetReaderIndex: index, - id: id, - encodedTags: encodedTags, - data: data, - checksum: checksum, - }, nil -} - -func (r *crossBlockReader) Err() error { - return r.err -} - -func (r *crossBlockReader) Close() error { - var emptyRecord BlockRecord - for i := range r.records { - r.records[i] = emptyRecord - } - r.records = r.records[:0] - - for i := range r.minHeap { - r.minHeap[i] = nil - } - r.minHeap = r.minHeap[:0] - - return nil -} - -type minHeapEntry struct { - id ident.BytesID - encodedTags []byte - data []byte - dataFileSetReaderIndex int - checksum uint32 -} - -var _ heap.Interface = (*minHeap)(nil) - -type minHeap []*minHeapEntry - -func (h minHeap) Len() int { - return len(h) -} - -func (h minHeap) Less(i, j int) bool { - idsCmp := bytes.Compare(h[i].id.Bytes(), h[j].id.Bytes()) - if idsCmp == 0 { - return h[i].dataFileSetReaderIndex < h[j].dataFileSetReaderIndex - } - return idsCmp < 0 -} - -func (h minHeap) Swap(i, j int) { - h[i], h[j] = h[j], h[i] -} - -func (h *minHeap) Push(x interface{}) { - *h = append(*h, x.(*minHeapEntry)) -} - -func (h *minHeap) Pop() interface{} { - old := *h - n := len(old) - x := old[n-1] - old[n-1] = nil - *h = old[0 : n-1] - return x -} diff --git a/src/dbnode/persist/fs/cross_block_reader_test.go b/src/dbnode/persist/fs/cross_block_reader_test.go deleted file mode 100644 index c8cb52c7cb..0000000000 --- a/src/dbnode/persist/fs/cross_block_reader_test.go +++ /dev/null @@ -1,305 +0,0 @@ -// Copyright (c) 2020 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package fs - -import ( - "errors" - "fmt" - "io" - "strconv" - "strings" - "testing" - "time" - - "github.com/m3db/m3/src/dbnode/ts" - "github.com/m3db/m3/src/x/ident" - "github.com/m3db/m3/src/x/instrument" - xtest "github.com/m3db/m3/src/x/test" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -var errExpected = errors.New("expected error") - -func TestCrossBlockReaderRejectMisconfiguredInputs(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - dfsReader := NewMockDataFileSetReader(ctrl) - dfsReader.EXPECT().StreamingEnabled().Return(false) - - _, err := NewCrossBlockReader([]DataFileSetReader{dfsReader}, instrument.NewTestOptions(t)) - - assert.Equal(t, errReaderNotOrderedByIndex, err) -} - -func TestCrossBlockReaderRejectMisorderedInputs(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - now := time.Now().Truncate(time.Hour) - dfsReader1 := NewMockDataFileSetReader(ctrl) - dfsReader1.EXPECT().StreamingEnabled().Return(true) - dfsReader1.EXPECT().Range().Return(xtime.Range{Start: now}) - - later := now.Add(time.Hour) - dfsReader2 := NewMockDataFileSetReader(ctrl) - dfsReader2.EXPECT().StreamingEnabled().Return(true) - dfsReader2.EXPECT().Range().Return(xtime.Range{Start: later}) - - _, err := NewCrossBlockReader([]DataFileSetReader{dfsReader2, dfsReader1}, instrument.NewTestOptions(t)) - - assert.Equal(t, errUnorderedDataFileSetReaders, err) -} - -func TestCrossBlockReader(t *testing.T) { - tests := []struct { - name string - blockSeriesIDs [][]string - expectedIDs []string - }{ - { - name: "no readers", - blockSeriesIDs: [][]string{}, - expectedIDs: []string{}, - }, - { - name: "empty readers", - blockSeriesIDs: [][]string{{}, {}, {}}, - expectedIDs: []string{}, - }, - { - name: "one reader, one series", - blockSeriesIDs: [][]string{{"id1"}}, - expectedIDs: []string{"id1"}, - }, - { - name: "one reader, many series", - blockSeriesIDs: [][]string{{"id1", "id2", "id3"}}, - expectedIDs: []string{"id1", "id2", "id3"}, - }, - { - name: "many readers with same series", - blockSeriesIDs: [][]string{{"id1"}, {"id1"}, {"id1"}}, - expectedIDs: []string{"id1"}, - }, - { - name: "many readers with different series", - blockSeriesIDs: [][]string{{"id1"}, {"id2"}, {"id3"}}, - expectedIDs: []string{"id1", "id2", "id3"}, - }, - { - name: "many readers with unordered series", - blockSeriesIDs: [][]string{{"id3"}, {"id1"}, {"id2"}}, - expectedIDs: []string{"id1", "id2", "id3"}, - }, - { - name: "complex case", - blockSeriesIDs: [][]string{{"id2", "id3", "id5"}, {"id1", "id2", "id4"}, {"id1", "id4"}}, - expectedIDs: []string{"id1", "id2", "id3", "id4", "id5"}, - }, - { - name: "duplicate ids within a block", - blockSeriesIDs: [][]string{{"id1", "id2"}, {"id2", "id2"}}, - expectedIDs: []string{"id1", "id2", "id2"}, - }, - { - name: "immediate reader error", - blockSeriesIDs: [][]string{{"error"}}, - expectedIDs: []string{}, - }, - { - name: "reader error later", - blockSeriesIDs: [][]string{{"id1", "id2"}, {"id1", "error"}}, - expectedIDs: []string{"id1"}, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - testCrossBlockReader(t, tt.blockSeriesIDs, tt.expectedIDs) - }) - } -} - -func testCrossBlockReader(t *testing.T, blockSeriesIds [][]string, expectedIDs []string) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - now := time.Now().Truncate(time.Hour) - var dfsReaders []DataFileSetReader - - expectedBlockCount := 0 - - for blockIndex, ids := range blockSeriesIds { - dfsReader := NewMockDataFileSetReader(ctrl) - dfsReader.EXPECT().StreamingEnabled().Return(true) - dfsReader.EXPECT().Range().Return(xtime.Range{Start: now.Add(time.Hour * time.Duration(blockIndex))}).AnyTimes() - - blockHasError := false - for i, strID := range ids { - if strID == "error" { - dfsReader.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), errExpected) - blockHasError = true - } else { - id := ident.BytesID(strID) - tags := ts.EncodedTags(fmt.Sprintf("tags for %s", strID)) - data := []byte(strconv.Itoa(i)) - - checksum := uint32(blockIndex) // somewhat hacky - using checksum to propagate block index value for assertions - - dfsReader.EXPECT().StreamingRead().Return(id, tags, data, checksum, nil) - } - } - - if !blockHasError { - dfsReader.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), io.EOF).MaxTimes(1) - } - - dfsReaders = append(dfsReaders, dfsReader) - expectedBlockCount += len(ids) - } - - cbReader, err := NewCrossBlockReader(dfsReaders, instrument.NewTestOptions(t)) - require.NoError(t, err) - defer cbReader.Close() - - blockCount := 0 - seriesCount := 0 - for cbReader.Next() { - id, tags, records := cbReader.Current() - - assert.Equal(t, expectedIDs[seriesCount], string(id)) - assert.Equal(t, fmt.Sprintf("tags for %s", expectedIDs[seriesCount]), string(tags)) - - previousBlockIndex := -1 - for _, record := range records { - blockIndex := int(record.DataChecksum) // see the comment above - assert.True(t, blockIndex > previousBlockIndex, "same id blocks must be read in temporal order") - - previousBlockIndex = blockIndex - assert.NotNil(t, record.Data) - } - - blockCount += len(records) - seriesCount++ - } - - assert.Equal(t, len(expectedIDs), seriesCount, "count of series read") - - err = cbReader.Err() - if err == nil || (err.Error() != errExpected.Error() && !strings.HasPrefix(err.Error(), "duplicate id")) { - require.NoError(t, cbReader.Err()) - assert.Equal(t, expectedBlockCount, blockCount, "count of blocks read") - } - - for _, dfsReader := range dfsReaders { - assert.NotNil(t, dfsReader) - } -} - -func TestSkippingReader(t *testing.T) { - ctrl := xtest.NewController(t) - defer ctrl.Finish() - - now := time.Now().Truncate(time.Hour) - var dfsReaders []DataFileSetReader - - expectedBlockCount := 0 - - blockSeriesIDs := [][]string{{"id1"}, {"id2"}, {"id3"}} - expectedIDs := []string{"id3"} - for blockIndex, ids := range blockSeriesIDs { - dfsReader := NewMockDataFileSetReader(ctrl) - dfsReader.EXPECT().StreamingEnabled().Return(true) - dfsReader.EXPECT().Range().Return(xtime.Range{Start: now.Add(time.Hour * time.Duration(blockIndex))}).AnyTimes() - - blockHasError := false - for j, id := range ids { - if id == "error" { - dfsReader.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), errExpected) - blockHasError = true - } else { - tags := []byte(fmt.Sprintf("tags for %s", id)) - data := []byte{byte(j)} - - checksum := uint32(blockIndex) // somewhat hacky - using checksum to propagate block index value for assertions - - dfsReader.EXPECT().StreamingRead().Return(ident.BytesID(id), tags, data, checksum, nil) - } - } - - if !blockHasError { - dfsReader.EXPECT().StreamingRead().Return(nil, nil, nil, uint32(0), io.EOF).MaxTimes(1) - } - - dfsReaders = append(dfsReaders, dfsReader) - expectedBlockCount += len(ids) - } - - cbReader, err := NewCrossBlockReader(dfsReaders, instrument.NewTestOptions(t)) - require.NoError(t, err) - defer cbReader.Close() - - blockCount := 0 - seriesCount := 0 - - // NB: skip first two - expectedBlockCount -= 2 - require.True(t, cbReader.Next()) - require.True(t, cbReader.Next()) - for cbReader.Next() { - // NB: call Current twice to ensure it does not mutate values. - id, _, _ := cbReader.Current() - assert.Equal(t, expectedIDs[seriesCount], id.String()) - _, tags, records := cbReader.Current() - - strID := string(id) - assert.Equal(t, expectedIDs[seriesCount], strID) - assert.Equal(t, fmt.Sprintf("tags for %s", strID), string(tags)) - - previousBlockIndex := -1 - for _, record := range records { - blockIndex := int(record.DataChecksum) // see the comment above - assert.True(t, blockIndex > previousBlockIndex, "same id blocks must be read in temporal order") - - previousBlockIndex = blockIndex - assert.NotNil(t, record.Data) - } - - blockCount += len(records) - seriesCount++ - } - - assert.Equal(t, len(expectedIDs), seriesCount, "count of series read") - - err = cbReader.Err() - if err == nil || (err.Error() != errExpected.Error() && !strings.HasPrefix(err.Error(), "duplicate id")) { - require.NoError(t, cbReader.Err()) - assert.Equal(t, expectedBlockCount, blockCount, "count of blocks read") - } - - for _, dfsReader := range dfsReaders { - assert.NotNil(t, dfsReader) - } -} diff --git a/src/dbnode/persist/fs/fs_mock.go b/src/dbnode/persist/fs/fs_mock.go index 861c2659a1..25c89d94ca 100644 --- a/src/dbnode/persist/fs/fs_mock.go +++ b/src/dbnode/persist/fs/fs_mock.go @@ -1,5 +1,5 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: CrossBlockReader,CrossBlockIterator,DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter) +// Source: github.com/m3db/m3/src/dbnode/persist/fs (interfaces: DataFileSetWriter,DataFileSetReader,DataFileSetSeeker,IndexFileSetWriter,IndexFileSetReader,IndexSegmentFileSetWriter,IndexSegmentFileSet,IndexSegmentFile,SnapshotMetadataFileWriter,DataFileSetSeekerManager,ConcurrentDataFileSetSeeker,MergeWith,StreamingWriter) // Copyright (c) 2020 Uber Technologies, Inc. // @@ -43,178 +43,6 @@ import ( "github.com/golang/mock/gomock" ) -// MockCrossBlockReader is a mock of CrossBlockReader interface -type MockCrossBlockReader struct { - ctrl *gomock.Controller - recorder *MockCrossBlockReaderMockRecorder -} - -// MockCrossBlockReaderMockRecorder is the mock recorder for MockCrossBlockReader -type MockCrossBlockReaderMockRecorder struct { - mock *MockCrossBlockReader -} - -// NewMockCrossBlockReader creates a new mock instance -func NewMockCrossBlockReader(ctrl *gomock.Controller) *MockCrossBlockReader { - mock := &MockCrossBlockReader{ctrl: ctrl} - mock.recorder = &MockCrossBlockReaderMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockCrossBlockReader) EXPECT() *MockCrossBlockReaderMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockCrossBlockReader) Close() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Close") - ret0, _ := ret[0].(error) - return ret0 -} - -// Close indicates an expected call of Close -func (mr *MockCrossBlockReaderMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockReader)(nil).Close)) -} - -// Current mocks base method -func (m *MockCrossBlockReader) Current() (ident.BytesID, ts.EncodedTags, []BlockRecord) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Current") - ret0, _ := ret[0].(ident.BytesID) - ret1, _ := ret[1].(ts.EncodedTags) - ret2, _ := ret[2].([]BlockRecord) - return ret0, ret1, ret2 -} - -// Current indicates an expected call of Current -func (mr *MockCrossBlockReaderMockRecorder) Current() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockReader)(nil).Current)) -} - -// Err mocks base method -func (m *MockCrossBlockReader) Err() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Err") - ret0, _ := ret[0].(error) - return ret0 -} - -// Err indicates an expected call of Err -func (mr *MockCrossBlockReaderMockRecorder) Err() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockReader)(nil).Err)) -} - -// Next mocks base method -func (m *MockCrossBlockReader) Next() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Next") - ret0, _ := ret[0].(bool) - return ret0 -} - -// Next indicates an expected call of Next -func (mr *MockCrossBlockReaderMockRecorder) Next() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockReader)(nil).Next)) -} - -// MockCrossBlockIterator is a mock of CrossBlockIterator interface -type MockCrossBlockIterator struct { - ctrl *gomock.Controller - recorder *MockCrossBlockIteratorMockRecorder -} - -// MockCrossBlockIteratorMockRecorder is the mock recorder for MockCrossBlockIterator -type MockCrossBlockIteratorMockRecorder struct { - mock *MockCrossBlockIterator -} - -// NewMockCrossBlockIterator creates a new mock instance -func NewMockCrossBlockIterator(ctrl *gomock.Controller) *MockCrossBlockIterator { - mock := &MockCrossBlockIterator{ctrl: ctrl} - mock.recorder = &MockCrossBlockIteratorMockRecorder{mock} - return mock -} - -// EXPECT returns an object that allows the caller to indicate expected use -func (m *MockCrossBlockIterator) EXPECT() *MockCrossBlockIteratorMockRecorder { - return m.recorder -} - -// Close mocks base method -func (m *MockCrossBlockIterator) Close() { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Close") -} - -// Close indicates an expected call of Close -func (mr *MockCrossBlockIteratorMockRecorder) Close() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockCrossBlockIterator)(nil).Close)) -} - -// Current mocks base method -func (m *MockCrossBlockIterator) Current() (ts.Datapoint, time0.Unit, ts.Annotation) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Current") - ret0, _ := ret[0].(ts.Datapoint) - ret1, _ := ret[1].(time0.Unit) - ret2, _ := ret[2].(ts.Annotation) - return ret0, ret1, ret2 -} - -// Current indicates an expected call of Current -func (mr *MockCrossBlockIteratorMockRecorder) Current() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Current", reflect.TypeOf((*MockCrossBlockIterator)(nil).Current)) -} - -// Err mocks base method -func (m *MockCrossBlockIterator) Err() error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Err") - ret0, _ := ret[0].(error) - return ret0 -} - -// Err indicates an expected call of Err -func (mr *MockCrossBlockIteratorMockRecorder) Err() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Err", reflect.TypeOf((*MockCrossBlockIterator)(nil).Err)) -} - -// Next mocks base method -func (m *MockCrossBlockIterator) Next() bool { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Next") - ret0, _ := ret[0].(bool) - return ret0 -} - -// Next indicates an expected call of Next -func (mr *MockCrossBlockIteratorMockRecorder) Next() *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Next", reflect.TypeOf((*MockCrossBlockIterator)(nil).Next)) -} - -// Reset mocks base method -func (m *MockCrossBlockIterator) Reset(arg0 []BlockRecord) { - m.ctrl.T.Helper() - m.ctrl.Call(m, "Reset", arg0) -} - -// Reset indicates an expected call of Reset -func (mr *MockCrossBlockIteratorMockRecorder) Reset(arg0 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockCrossBlockIterator)(nil).Reset), arg0) -} - // MockDataFileSetWriter is a mock of DataFileSetWriter interface type MockDataFileSetWriter struct { ctrl *gomock.Controller diff --git a/src/dbnode/persist/fs/index_read.go b/src/dbnode/persist/fs/index_read.go index 5b93a34225..773c7b5291 100644 --- a/src/dbnode/persist/fs/index_read.go +++ b/src/dbnode/persist/fs/index_read.go @@ -22,7 +22,6 @@ package fs import ( "bytes" - "errors" "fmt" "io" "io/ioutil" @@ -42,9 +41,6 @@ const ( mmapPersistFsIndexName = "mmap.persist.fs.index" ) -// ErrIndexReaderValidationFailed is returned for corrupt index segemnts. -var ErrIndexReaderValidationFailed = errors.New("validation failed") - type indexReader struct { opts Options filePathPrefix string @@ -309,16 +305,16 @@ func (r *indexReader) Validate() error { func (r *indexReader) validateDigestsFileDigest() error { if r.readDigests.digestsFileDigest != r.expectedDigestOfDigest { - return fmt.Errorf("(%w) read digests file checksum bad: expected=%d, actual=%d", - ErrIndexReaderValidationFailed, r.expectedDigestOfDigest, r.readDigests.digestsFileDigest) + return fmt.Errorf("read digests file checksum bad: expected=%d, actual=%d", + r.expectedDigestOfDigest, r.readDigests.digestsFileDigest) } return nil } func (r *indexReader) validateInfoFileDigest() error { if r.readDigests.infoFileDigest != r.expectedDigest.InfoDigest { - return fmt.Errorf("(%w) read info file checksum bad: expected=%d, actual=%d", - ErrIndexReaderValidationFailed, r.expectedDigest.InfoDigest, r.readDigests.infoFileDigest) + return fmt.Errorf("read info file checksum bad: expected=%d, actual=%d", + r.expectedDigest.InfoDigest, r.readDigests.infoFileDigest) } return nil } @@ -326,37 +322,35 @@ func (r *indexReader) validateInfoFileDigest() error { func (r *indexReader) validateSegmentFileDigest(segmentIdx, fileIdx int) error { if segmentIdx >= len(r.readDigests.segments) { return fmt.Errorf( - "(%w) have not read correct number of segments to validate segment %d checksums: "+ + "have not read correct number of segments to validate segment %d checksums: "+ "need=%d, actual=%d", - ErrIndexReaderValidationFailed, segmentIdx, segmentIdx+1, len(r.readDigests.segments)) + segmentIdx, segmentIdx+1, len(r.readDigests.segments)) } if segmentIdx >= len(r.expectedDigest.SegmentDigests) { return fmt.Errorf( - "(%w) have not read digest files correctly to validate segment %d checksums: "+ + "have not read digest files correctly to validate segment %d checksums: "+ "need=%d, actual=%d", - ErrIndexReaderValidationFailed, segmentIdx, segmentIdx+1, len(r.expectedDigest.SegmentDigests)) + segmentIdx, segmentIdx+1, len(r.expectedDigest.SegmentDigests)) } if fileIdx >= len(r.readDigests.segments[segmentIdx].files) { return fmt.Errorf( - "(%w) have not read correct number of segment files to validate segment %d checksums: "+ + "have not read correct number of segment files to validate segment %d checksums: "+ "need=%d, actual=%d", - ErrIndexReaderValidationFailed, segmentIdx, fileIdx+1, - len(r.readDigests.segments[segmentIdx].files)) + segmentIdx, fileIdx+1, len(r.readDigests.segments[segmentIdx].files)) } if fileIdx >= len(r.expectedDigest.SegmentDigests[segmentIdx].Files) { return fmt.Errorf( - "(%w) have not read correct number of segment files to validate segment %d checksums: "+ + "have not read correct number of segment files to validate segment %d checksums: "+ "need=%d, actual=%d", - ErrIndexReaderValidationFailed, segmentIdx, fileIdx+1, - len(r.expectedDigest.SegmentDigests[segmentIdx].Files)) + segmentIdx, fileIdx+1, len(r.expectedDigest.SegmentDigests[segmentIdx].Files)) } expected := r.expectedDigest.SegmentDigests[segmentIdx].Files[fileIdx].Digest actual := r.readDigests.segments[segmentIdx].files[fileIdx].digest if actual != expected { - return fmt.Errorf("(%w) read segment file %d for segment %d checksum bad: expected=%d, actual=%d", - ErrIndexReaderValidationFailed, segmentIdx, fileIdx, expected, actual) + return fmt.Errorf("read segment file %d for segment %d checksum bad: expected=%d, actual=%d", + segmentIdx, fileIdx, expected, actual) } return nil } diff --git a/src/dbnode/persist/fs/merger.go b/src/dbnode/persist/fs/merger.go index 555bbcc6d8..a6e78dc496 100644 --- a/src/dbnode/persist/fs/merger.go +++ b/src/dbnode/persist/fs/merger.go @@ -249,10 +249,15 @@ func (m *merger) Merge( if err == nil { err = onFlush.OnFlushNewSeries(persist.OnFlushNewSeriesEvent{ - Shard: shard, - BlockStart: startTime, - FirstWrite: mergeWithData.FirstWrite, - SeriesMetadata: seriesMetadata, + Shard: shard, + BlockStart: startTime, + FirstWrite: mergeWithData.FirstWrite, + SeriesMetadata: persist.SeriesMetadata{ + Type: persist.SeriesDocumentType, + Document: seriesMetadata, + // The lifetime of the shard series metadata is longly lived. + LifeTime: persist.SeriesLifeTimeLong, + }, }) } diff --git a/src/dbnode/persist/fs/types.go b/src/dbnode/persist/fs/types.go index ecce353743..9c89e6566a 100644 --- a/src/dbnode/persist/fs/types.go +++ b/src/dbnode/persist/fs/types.go @@ -660,38 +660,6 @@ type Segments interface { BlockStart() time.Time } -// BlockRecord wraps together M3TSZ data bytes with their checksum. -type BlockRecord struct { - Data []byte - DataChecksum uint32 -} - -// CrossBlockReader allows reading data (encoded bytes) from multiple -// DataFileSetReaders of the same shard, ordered lexographically by series ID, -// then by block time. -type CrossBlockReader interface { - io.Closer - - // Next advances to the next data record, returning true if it exists. - Next() bool - - // Err returns the last error encountered (if any). - Err() error - - // Current returns distinct series id and encodedTags, plus a slice with data - // and checksums from all blocks corresponding to that series (in temporal order). - // ID, encodedTags, records, and underlying data are invalidated on each call to Next(). - Current() (id ident.BytesID, encodedTags ts.EncodedTags, records []BlockRecord) -} - -// CrossBlockIterator iterates across BlockRecords. -type CrossBlockIterator interface { - encoding.Iterator - - // Reset resets the iterator to the given block records. - Reset(records []BlockRecord) -} - // IndexClaimsManager manages concurrent claims to volume indices per ns and block start. // This allows multiple threads to safely increment the volume index. type IndexClaimsManager interface { @@ -700,6 +668,3 @@ type IndexClaimsManager interface { blockStart time.Time, ) (int, error) } - -// DeleteFilesFn deletes files passed in as arg. -type DeleteFilesFn func(files []string) error diff --git a/src/dbnode/persist/types.go b/src/dbnode/persist/types.go index f93f28bb56..c01b4aa8ca 100644 --- a/src/dbnode/persist/types.go +++ b/src/dbnode/persist/types.go @@ -35,9 +35,7 @@ import ( "github.com/pborman/uuid" ) -var ( - errReuseableTagIteratorRequired = errors.New("reuseable tags iterator is required") -) +var errReuseableTagIteratorRequired = errors.New("reuseable tags iterator is required") // Metadata is metadata for a time series, it can // have several underlying sources. @@ -322,12 +320,49 @@ const ( FileSetIndexContentType ) +// SeriesMetadataLifeTime describes the memory life time type. +type SeriesMetadataLifeTime uint8 + +const ( + // SeriesLifeTimeLong means the underlying memory's life time is long lived and exceeds + // the execution duration of the series metadata receiver. + SeriesLifeTimeLong SeriesMetadataLifeTime = iota + // SeriesLifeTimeShort means that the underlying memory is only valid for the duration + // of the OnFlushNewSeries call. Must clone the underlying bytes in order to extend the life time. + SeriesLifeTimeShort +) + +// SeriesMetadataType describes the type of series metadata. +type SeriesMetadataType uint8 + +const ( + // SeriesDocumentType means the metadata is in doc.Document form. + SeriesDocumentType SeriesMetadataType = iota + // SeriesIDAndEncodedTagsType means the metadata is in IDAndEncodedTags form. + SeriesIDAndEncodedTagsType +) + +// IDAndEncodedTags contains a series ID and encoded tags. +type IDAndEncodedTags struct { + ID ident.BytesID + EncodedTags ts.EncodedTags +} + +// SeriesMetadata captures different representations of series metadata and +// the ownership status of the underlying memory. +type SeriesMetadata struct { + Document doc.Document + IDAndEncodedTags IDAndEncodedTags + Type SeriesMetadataType + LifeTime SeriesMetadataLifeTime +} + // OnFlushNewSeriesEvent is the fields related to a flush of a new series. type OnFlushNewSeriesEvent struct { Shard uint32 BlockStart time.Time FirstWrite time.Time - SeriesMetadata doc.Document + SeriesMetadata SeriesMetadata } // OnFlushSeries performs work on a per series level. diff --git a/src/dbnode/storage/bootstrap/bootstrap_mock.go b/src/dbnode/storage/bootstrap/bootstrap_mock.go index 680d2c98a4..7ecbc95f65 100644 --- a/src/dbnode/storage/bootstrap/bootstrap_mock.go +++ b/src/dbnode/storage/bootstrap/bootstrap_mock.go @@ -703,21 +703,6 @@ func (mr *MockCacheMockRecorder) InfoFilesForShard(ns, shard interface{}) *gomoc return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InfoFilesForShard", reflect.TypeOf((*MockCache)(nil).InfoFilesForShard), ns, shard) } -// IndexInfoFilesForNamespace mocks base method -func (m *MockCache) IndexInfoFilesForNamespace(ns namespace.Metadata) ([]fs.ReadIndexInfoFileResult, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "IndexInfoFilesForNamespace", ns) - ret0, _ := ret[0].([]fs.ReadIndexInfoFileResult) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// IndexInfoFilesForNamespace indicates an expected call of IndexInfoFilesForNamespace -func (mr *MockCacheMockRecorder) IndexInfoFilesForNamespace(ns interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IndexInfoFilesForNamespace", reflect.TypeOf((*MockCache)(nil).IndexInfoFilesForNamespace), ns) -} - // ReadInfoFiles mocks base method func (m *MockCache) ReadInfoFiles() InfoFilesByNamespace { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/bootstrap/bootstrapper/README.md b/src/dbnode/storage/bootstrap/bootstrapper/README.md index 947ea7f10a..b062aaac3d 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/README.md +++ b/src/dbnode/storage/bootstrap/bootstrapper/README.md @@ -6,7 +6,6 @@ The collection of bootstrappers comprise the task executed when bootstrapping a - `fs`: The filesystem bootstrapper, used to bootstrap as much data as possible from the local filesystem. - `peers`: The peers bootstrapper, used to bootstrap any remaining data from peers. This is used for a full node join too. - - *NOTE*: For the node leave case, the peers bs will persist default volume type index filesets to disk with non-overlapping shard time ranges to avoid re-building the entire index segment w/ new shards. - `commitlog`: The commit log bootstrapper, currently only used in the case that peers bootstrapping fails. Once the current block is being snapshotted frequently to disk it might be faster and make more sense to not actively use the peers bootstrapper and just use a combination of the filesystem bootstrapper and the minimal time range required from the commit log bootstrapper. - *NOTE*: the commitlog bootstrapper is special cased in that it runs for the *entire* bootstrappable range per shard whereas other bootstrappers fill in the unfulfilled gaps as bootstrapping progresses. diff --git a/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go b/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go index 7d97ff487f..a9a790d15f 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/fs/source.go @@ -21,12 +21,10 @@ package fs import ( - "errors" "fmt" "sync" "time" - indexpb "github.com/m3db/m3/src/dbnode/generated/proto/index" "github.com/m3db/m3/src/dbnode/namespace" "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" @@ -49,7 +47,6 @@ import ( "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/context" - xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" "github.com/m3db/m3/src/x/instrument" "github.com/m3db/m3/src/x/pool" @@ -82,14 +79,12 @@ type fileSystemSource struct { idPool ident.Pool newReaderFn newDataFileSetReaderFn newReaderPoolOpts bootstrapper.NewReaderPoolOptions - deleteFilesFn fs.DeleteFilesFn metrics fileSystemSourceMetrics } type fileSystemSourceMetrics struct { - persistedIndexBlocksRead tally.Counter - persistedIndexBlocksWrite tally.Counter - indexBlocksFailedValidation tally.Counter + persistedIndexBlocksRead tally.Counter + persistedIndexBlocksWrite tally.Counter } func newFileSystemSource(opts Options) (bootstrap.Source, error) { @@ -103,17 +98,15 @@ func newFileSystemSource(opts Options) (bootstrap.Source, error) { opts = opts.SetInstrumentOptions(iopts) s := &fileSystemSource{ - opts: opts, - fsopts: opts.FilesystemOptions(), - log: iopts.Logger().With(zap.String("bootstrapper", "filesystem")), - nowFn: opts.ResultOptions().ClockOptions().NowFn(), - idPool: opts.IdentifierPool(), - newReaderFn: fs.NewReader, - deleteFilesFn: fs.DeleteFiles, + opts: opts, + fsopts: opts.FilesystemOptions(), + log: iopts.Logger().With(zap.String("bootstrapper", "filesystem")), + nowFn: opts.ResultOptions().ClockOptions().NowFn(), + idPool: opts.IdentifierPool(), + newReaderFn: fs.NewReader, metrics: fileSystemSourceMetrics{ - persistedIndexBlocksRead: scope.Counter("persist-index-blocks-read"), - persistedIndexBlocksWrite: scope.Counter("persist-index-blocks-write"), - indexBlocksFailedValidation: scope.Counter("index-blocks-failed-validation"), + persistedIndexBlocksRead: scope.Counter("persist-index-blocks-read"), + persistedIndexBlocksWrite: scope.Counter("persist-index-blocks-write"), }, } s.newReaderPoolOpts.Alloc = s.newReader @@ -335,7 +328,6 @@ func (s *fileSystemSource) bootstrapFromReaders( builder *result.IndexBuilder, persistManager *bootstrapper.SharedPersistManager, compactor *bootstrapper.SharedCompactor, - cache bootstrap.Cache, ) { resultOpts := s.opts.ResultOptions() @@ -346,7 +338,7 @@ func (s *fileSystemSource) bootstrapFromReaders( s.loadShardReadersDataIntoShardResult(run, ns, accumulator, runOpts, runResult, resultOpts, timeWindowReaders, readerPool, - builder, persistManager, compactor, cache) + builder, persistManager, compactor) } } @@ -400,7 +392,6 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( builder *result.IndexBuilder, persistManager *bootstrapper.SharedPersistManager, compactor *bootstrapper.SharedCompactor, - cache bootstrap.Cache, ) { var ( blockPool = ropts.DatabaseBlockOptions().DatabaseBlockPool() @@ -587,36 +578,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( } if shouldFlush && satisifiedFlushRanges { - // NB(bodu): If we are persisting an index segment to disk, we need to delete any existing - // index filesets at this block start. The newly persisted index segments becomes the new source of truth. - var ( - filesToDelete = []string{} - persistSuccess bool - ) - defer func() { - if persistSuccess { - if err := s.deleteFilesFn(filesToDelete); err != nil { - instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { - l.Error("failed to delete non default index filesets", - zap.Error(err), - zap.Stringer("namespace", ns.ID()), - zap.Stringer("requestedRanges", requestedRanges)) - }) - } - } - }() - filesToDelete = s.appendIndexFilesetFilesToDelete(ns, blockStart, cache, filesToDelete, runResult, iopts) - - // Use debug level with full log fidelity. s.log.Debug("building file set index segment", buildIndexLogFields...) - // Use info log with more high level attributes. - s.log.Info("rebuilding file set index segment", - zap.Stringer("namespace", ns.ID()), - zap.Int("totalEntries", totalEntries), - zap.Time("blockStart", blockStart), - zap.Time("blockEnd", blockEnd)) - // NB(bodu): The index claims manager ensures that we properly advance the volume index - // past existing volume indices. indexBlock, err = bootstrapper.PersistBootstrapIndexSegment( ns, requestedRanges, @@ -628,11 +590,7 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( blockStart, blockEnd, ) - if err == nil { - // Track success. - s.metrics.persistedIndexBlocksWrite.Inc(1) - persistSuccess = true - } else { + if err != nil { instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { l.Error("persist fs index bootstrap failed", zap.Error(err), @@ -640,6 +598,8 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( zap.Stringer("requestedRanges", requestedRanges)) }) } + // Track success. + s.metrics.persistedIndexBlocksWrite.Inc(1) } else { s.log.Info("building in-memory index segment", buildIndexLogFields...) indexBlock, err = bootstrapper.BuildBootstrapIndexSegment( @@ -690,74 +650,6 @@ func (s *fileSystemSource) loadShardReadersDataIntoShardResult( remainingRanges, timesWithErrors) } -// appendIndexFilesetFilesToDelete appends all index filesets at a block start for deletion. -// Also removes index results from given block start since results are not complete and/or corrupted -// and require an index segment rebuild. -func (s *fileSystemSource) appendIndexFilesetFilesToDelete( - ns namespace.Metadata, - blockStart time.Time, - cache bootstrap.Cache, - filesToDelete []string, - runResult *runResult, - iopts instrument.Options, -) []string { - infoFiles, err := cache.IndexInfoFilesForNamespace(ns) - if err != nil { - instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { - l.Error("failed to get index info files from cache", - zap.Error(err), - zap.Time("blockStart", blockStart), - zap.Stringer("namespace", ns.ID())) - }) - } - for i := range infoFiles { - if err := infoFiles[i].Err.Error(); err != nil { - // We already log errors once when bootstrapping from persisted - // index blocks just continue here. - continue - } - - info := infoFiles[i].Info - indexBlockStart := xtime.UnixNano(info.BlockStart).ToTime() - if blockStart.Equal(indexBlockStart) { - filesToDelete = append(filesToDelete, infoFiles[i].AbsoluteFilePaths...) - } - } - - // Remove index results for the block we're deleting. - if err := removeIndexResults(ns, blockStart, runResult); err != nil { - instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { - l.Error("error removing partial/corrupted index results", - zap.Error(err), - zap.Time("blockStart", blockStart), - zap.Stringer("namespace", ns.ID())) - }) - } - - return filesToDelete -} - -func removeIndexResults( - ns namespace.Metadata, - blockStart time.Time, - runResult *runResult, -) error { - runResult.Lock() - defer runResult.Unlock() - - multiErr := xerrors.NewMultiError() - results, ok := runResult.index.IndexResults()[xtime.ToUnixNano(blockStart)] - if ok { - for volumeType, indexBlock := range results.Iter() { - for _, seg := range indexBlock.Segments() { - multiErr = multiErr.Add(seg.Segment().Close()) - } - results.DeleteBlock(volumeType) - } - } - return multiErr.FinalError() -} - func (s *fileSystemSource) readNextEntryAndRecordBlock( nsCtx namespace.Context, accumulator bootstrap.NamespaceDataAccumulator, @@ -855,17 +747,21 @@ func (s *fileSystemSource) read( ) (*runResult, error) { var ( seriesCachePolicy = s.opts.ResultOptions().SeriesCachePolicy() - res = newRunResult() + res *runResult ) if shardTimeRanges.IsEmpty() { return newRunResult(), nil } - mergeResult := func(newResult *runResult) { + setOrMergeResult := func(newResult *runResult) { if newResult == nil { return } - res = res.mergedResult(newResult) + if res == nil { + res = newResult + } else { + res = res.mergedResult(newResult) + } } if run == bootstrapDataRunType { @@ -889,7 +785,7 @@ func (s *fileSystemSource) read( // subtract the shard + time ranges from what we intend to bootstrap // for those we found. r, err := s.bootstrapFromIndexPersistedBlocks(md, - shardTimeRanges, cache) + shardTimeRanges) if err != nil { s.log.Warn("filesystem bootstrapped failed to read persisted index blocks") } else { @@ -897,7 +793,7 @@ func (s *fileSystemSource) read( shardTimeRanges = shardTimeRanges.Copy() shardTimeRanges.Subtract(r.fulfilled) // Set or merge result. - mergeResult(r.result) + setOrMergeResult(r.result) } logSpan("bootstrap_from_index_persisted_blocks_done") } @@ -936,6 +832,8 @@ func (s *fileSystemSource) read( Cache: cache, }) + bootstrapFromReadersRunResult := newRunResult() + var buildWg sync.WaitGroup for i := 0; i < indexSegmentConcurrency; i++ { alloc := s.opts.ResultOptions().IndexDocumentsBuilderAllocator() @@ -971,17 +869,19 @@ func (s *fileSystemSource) read( buildWg.Add(1) go func() { s.bootstrapFromReaders(run, md, - accumulator, runOpts, res, + accumulator, runOpts, bootstrapFromReadersRunResult, readerPool, readersCh, builder, &bootstrapper.SharedPersistManager{Mgr: persistManager}, - &bootstrapper.SharedCompactor{Compactor: compactor}, - cache) + &bootstrapper.SharedCompactor{Compactor: compactor}) buildWg.Done() }() } buildWg.Wait() + // Merge any existing results if necessary. + setOrMergeResult(bootstrapFromReadersRunResult) + return res, nil } @@ -1028,21 +928,15 @@ type bootstrapFromIndexPersistedBlocksResult struct { func (s *fileSystemSource) bootstrapFromIndexPersistedBlocks( ns namespace.Metadata, shardTimeRanges result.ShardTimeRanges, - cache bootstrap.Cache, ) (bootstrapFromIndexPersistedBlocksResult, error) { res := bootstrapFromIndexPersistedBlocksResult{ fulfilled: result.NewShardTimeRanges(), } indexBlockSize := ns.Options().IndexOptions().BlockSize() - infoFiles, err := cache.IndexInfoFilesForNamespace(ns) - if err != nil { - return bootstrapFromIndexPersistedBlocksResult{}, err - } + infoFiles := fs.ReadIndexInfoFiles(s.fsopts.FilePathPrefix(), ns.ID(), + s.fsopts.InfoReaderBufferSize()) - // Track corrupted block starts as we will attempt to later recover - // from corruption by building an index segment from TSDB data. - corruptedBlockStarts := make(map[xtime.UnixNano]struct{}) for _, infoFile := range infoFiles { if err := infoFile.Err.Error(); err != nil { s.log.Error("unable to read index info file", @@ -1055,18 +949,7 @@ func (s *fileSystemSource) bootstrapFromIndexPersistedBlocks( } info := infoFile.Info - indexBlockStartUnixNanos := xtime.UnixNano(info.BlockStart) - indexBlockStart := indexBlockStartUnixNanos.ToTime() - - if _, ok := corruptedBlockStarts[indexBlockStartUnixNanos]; ok { - s.log.Info("index block corrupted skipping index info file", - zap.Stringer("namespace", ns.ID()), - zap.Stringer("blockStart", indexBlockStart), - zap.String("filepath", infoFile.Err.Filepath()), - ) - continue - } - + indexBlockStart := xtime.UnixNano(info.BlockStart).ToTime() indexBlockRange := xtime.Range{ Start: indexBlockStart, End: indexBlockStart.Add(indexBlockSize), @@ -1122,19 +1005,6 @@ func (s *fileSystemSource) bootstrapFromIndexPersistedBlocks( zap.Time("blockStart", indexBlockStart), zap.Int("volumeIndex", infoFile.ID.VolumeIndex), ) - if errors.Is(err, fs.ErrIndexReaderValidationFailed) { - // Emit a metric for failed validations. - s.metrics.indexBlocksFailedValidation.Inc(1) - // Track corrupted blocks and remove any loaded results. - corruptedBlockStarts[indexBlockStartUnixNanos] = struct{}{} - if err := removeIndexResults(ns, indexBlockStart, res.result); err != nil { - s.log.Error("error removing partial/corrupted index results", - zap.Error(err), - zap.Stringer("namespace", ns.ID()), - zap.Time("blockStart", indexBlockStart), - ) - } - } continue } @@ -1142,30 +1012,26 @@ func (s *fileSystemSource) bootstrapFromIndexPersistedBlocks( s.metrics.persistedIndexBlocksRead.Inc(1) // Record result. + if res.result == nil { + res.result = newRunResult() + } segmentsFulfilled := willFulfill // NB(bodu): All segments read from disk are already persisted. persistedSegments := make([]result.Segment, 0, len(readResult.Segments)) for _, segment := range readResult.Segments { persistedSegments = append(persistedSegments, result.NewSegment(segment, true)) } + volumeType := idxpersist.DefaultIndexVolumeType + if info.IndexVolumeType != nil { + volumeType = idxpersist.IndexVolumeType(info.IndexVolumeType.Value) + } indexBlockByVolumeType := result.NewIndexBlockByVolumeType(indexBlockStart) - volumeType := volumeTypeFromInfo(&info) indexBlockByVolumeType.SetBlock(volumeType, result.NewIndexBlock(persistedSegments, segmentsFulfilled)) - - if res.result == nil { - res.result = newRunResult() - } // NB(r): Don't need to call MarkFulfilled on the IndexResults here // as we've already passed the ranges fulfilled to the block that // we place in the IndexResuts with the call to Add(...). res.result.index.Add(indexBlockByVolumeType, nil) - - // NB(bodu): We only mark ranges as fulfilled for the default index volume type. - // It's possible to have other index volume types but the default type is required to - // fulfill bootstrappable ranges. - if volumeType == idxpersist.DefaultIndexVolumeType { - res.fulfilled.AddRanges(segmentsFulfilled) - } + res.fulfilled.AddRanges(segmentsFulfilled) } return res, nil @@ -1208,11 +1074,3 @@ func (r *runResult) mergedResult(other *runResult) *runResult { index: result.MergedIndexBootstrapResult(r.index, other.index), } } - -func volumeTypeFromInfo(info *indexpb.IndexVolumeInfo) idxpersist.IndexVolumeType { - volumeType := idxpersist.DefaultIndexVolumeType - if info.IndexVolumeType != nil { - volumeType = idxpersist.IndexVolumeType(info.IndexVolumeType.Value) - } - return volumeType -} diff --git a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go index 754bec8de9..14f83aa76d 100644 --- a/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go +++ b/src/dbnode/storage/bootstrap/bootstrapper/fs/source_index_test.go @@ -35,7 +35,6 @@ import ( "github.com/m3db/m3/src/dbnode/storage/index/convert" "github.com/m3db/m3/src/m3ninx/index/segment/mem" idxpersist "github.com/m3db/m3/src/m3ninx/persist" - xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/ident" xtime "github.com/m3db/m3/src/x/time" "github.com/stretchr/testify/require" @@ -305,7 +304,7 @@ func validateGoodTaggedSeries( } } -func TestBootstrapIndexAndUnfulfilledRanges(t *testing.T) { +func TestBootstrapIndex(t *testing.T) { dir := createTempDir(t) defer os.RemoveAll(dir) @@ -335,46 +334,9 @@ func TestBootstrapIndexAndUnfulfilledRanges(t *testing.T) { times.shardTimeRanges, opts.FilesystemOptions(), nsMD) defer tester.Finish() - // Write out non default type index volume type index block and ensure - // that it gets deleted and is not loaded into the index results to test - // the unfulfilled shard time ranges case (missing default index volume type - // and/or index segments failed validation). - var ( - notDefaultIndexVolumeType = idxpersist.IndexVolumeType("not-default") - shards = map[uint32]struct{}{testShard: struct{}{}} - filesToDelete []string - ) - idxWriter, err := fs.NewIndexWriter(src.fsopts) - require.NoError(t, err) - require.NoError(t, idxWriter.Open(fs.IndexWriterOpenOptions{ - Identifier: fs.FileSetFileIdentifier{ - FileSetContentType: persist.FileSetIndexContentType, - Namespace: nsMD.ID(), - BlockStart: times.start, - VolumeIndex: 1, - }, - BlockSize: nsMD.Options().IndexOptions().BlockSize(), - FileSetType: persist.FileSetFlushType, - Shards: shards, - IndexVolumeType: notDefaultIndexVolumeType, - })) - // Don't need to write any actual data. - require.NoError(t, idxWriter.Close()) - src.deleteFilesFn = func(files []string) error { - filesToDelete = append(filesToDelete, files...) - multiErr := xerrors.NewMultiError() - for _, f := range files { - multiErr = multiErr.Add(os.Remove(f)) - } - return multiErr.FinalError() - } - tester.TestReadWith(src) indexResults := tester.ResultForNamespace(nsMD.ID()).IndexResult.IndexResults() - // Ensure we are attempting to delete a single index fileset (in this case w/ no data). - require.Len(t, filesToDelete, 3) - // Check that single persisted segment got written out infoFiles := fs.ReadIndexInfoFiles(src.fsopts.FilePathPrefix(), testNs1ID, src.fsopts.InfoReaderBufferSize()) @@ -400,11 +362,6 @@ func TestBootstrapIndexAndUnfulfilledRanges(t *testing.T) { require.True(t, ok) require.True(t, segment.IsPersisted()) - // Check that the non default index volume type (missing default index volume type) - // was not added to the index results. - _, ok = blockByVolumeType.GetBlock(notDefaultIndexVolumeType) - require.False(t, ok) - // Check that the second segment is mutable and was not written out blockByVolumeType, ok = indexResults[xtime.ToUnixNano(times.start.Add(testIndexBlockSize))] require.True(t, ok) @@ -421,7 +378,7 @@ func TestBootstrapIndexAndUnfulfilledRanges(t *testing.T) { // Validate that wrote the block out (and no index blocks // were read as existing index blocks on disk) counters := scope.Snapshot().Counters() - require.Equal(t, int64(1), counters["fs-bootstrapper.persist-index-blocks-read+"].Value()) + require.Equal(t, int64(0), counters["fs-bootstrapper.persist-index-blocks-read+"].Value()) require.Equal(t, int64(1), counters["fs-bootstrapper.persist-index-blocks-write+"].Value()) } diff --git a/src/dbnode/storage/bootstrap/cache.go b/src/dbnode/storage/bootstrap/cache.go index bd4c91d613..50ebad8349 100644 --- a/src/dbnode/storage/bootstrap/cache.go +++ b/src/dbnode/storage/bootstrap/cache.go @@ -39,13 +39,11 @@ var ( type cache struct { sync.Mutex - fsOpts fs.Options - namespaceDetails []NamespaceDetails - infoFilesByNamespace InfoFilesByNamespace - indexInfoFilesByNamespace IndexInfoFilesByNamespace - iOpts instrument.Options - hasPopulatedInfo bool - hasPopulatedIndexInfo bool + fsOpts fs.Options + namespaceDetails []NamespaceDetails + infoFilesByNamespace InfoFilesByNamespace + iOpts instrument.Options + hasPopulatedInfo bool } // NewCache creates a cache specifically to be used during the bootstrap process. @@ -56,11 +54,10 @@ func NewCache(options CacheOptions) (Cache, error) { return nil, err } return &cache{ - fsOpts: options.FilesystemOptions(), - namespaceDetails: options.NamespaceDetails(), - infoFilesByNamespace: make(InfoFilesByNamespace, len(options.NamespaceDetails())), - indexInfoFilesByNamespace: make(IndexInfoFilesByNamespace, len(options.NamespaceDetails())), - iOpts: options.InstrumentOptions(), + fsOpts: options.FilesystemOptions(), + namespaceDetails: options.NamespaceDetails(), + infoFilesByNamespace: make(InfoFilesByNamespace, len(options.NamespaceDetails())), + iOpts: options.InstrumentOptions(), }, nil } @@ -88,24 +85,10 @@ func (c *cache) InfoFilesForShard(ns namespace.Metadata, shard uint32) ([]fs.Rea return infoFileResults, nil } -func (c *cache) IndexInfoFilesForNamespace(ns namespace.Metadata) ( - []fs.ReadIndexInfoFileResult, - error, -) { - infoFiles, ok := c.readIndexInfoFiles()[ns] - // This should never happen as Cache object is initialized with all namespaces to bootstrap. - if !ok { - return nil, fmt.Errorf("attempting to read index info files for namespace %v not "+ - "specified at bootstrap startup", ns.ID().String()) - } - return infoFiles, nil -} - func (c *cache) Evict() { c.Lock() defer c.Unlock() c.hasPopulatedInfo = false - c.hasPopulatedIndexInfo = false } func (c *cache) ReadInfoFiles() InfoFilesByNamespace { @@ -128,33 +111,14 @@ func (c *cache) populateInfoFilesByNamespaceWithLock() { } for _, shard := range finder.Shards { result[shard] = fs.ReadInfoFiles(c.fsOpts.FilePathPrefix(), - finder.Namespace.ID(), shard, c.fsOpts.InfoReaderBufferSize(), - c.fsOpts.DecodingOptions(), persist.FileSetFlushType) + finder.Namespace.ID(), shard, c.fsOpts.InfoReaderBufferSize(), c.fsOpts.DecodingOptions(), + persist.FileSetFlushType) } c.infoFilesByNamespace[finder.Namespace] = result } } -func (c *cache) readIndexInfoFiles() IndexInfoFilesByNamespace { - c.Lock() - defer c.Unlock() - if !c.hasPopulatedIndexInfo { - c.populateIndexInfoFilesByNamespaceWithLock() - c.hasPopulatedIndexInfo = true - } - return c.indexInfoFilesByNamespace -} - -func (c *cache) populateIndexInfoFilesByNamespaceWithLock() { - for i := range c.namespaceDetails { - finder := c.namespaceDetails[i] - c.indexInfoFilesByNamespace[finder.Namespace] = fs.ReadIndexInfoFiles( - c.fsOpts.FilePathPrefix(), finder.Namespace.ID(), - c.fsOpts.InfoReaderBufferSize()) - } -} - type cacheOptions struct { fsOpts fs.Options namespaceDetails []NamespaceDetails diff --git a/src/dbnode/storage/bootstrap/cache_test.go b/src/dbnode/storage/bootstrap/cache_test.go index 88699bbb93..9b7a29114c 100644 --- a/src/dbnode/storage/bootstrap/cache_test.go +++ b/src/dbnode/storage/bootstrap/cache_test.go @@ -32,7 +32,6 @@ import ( "github.com/m3db/m3/src/dbnode/persist" "github.com/m3db/m3/src/dbnode/persist/fs" "github.com/m3db/m3/src/dbnode/retention" - idxpersist "github.com/m3db/m3/src/m3ninx/persist" "github.com/m3db/m3/src/x/checked" "github.com/m3db/m3/src/x/ident" @@ -134,54 +133,6 @@ func TestCacheReadInfoFilesInvariantViolation(t *testing.T) { require.Error(t, err) } -func TestCacheReadIndexInfoFiles(t *testing.T) { - dir := createTempDir(t) - defer os.RemoveAll(dir) - - md1 := testNamespaceMetadata(t, ident.StringID("ns1")) - md2 := testNamespaceMetadata(t, ident.StringID("ns2")) - - fsOpts := testFilesystemOptions.SetFilePathPrefix(dir) - - shards := map[uint32]struct{}{ - 0: struct{}{}, - 1: struct{}{}, - } - writeIndexFilesets(t, md1.ID(), shards, fsOpts) - writeIndexFilesets(t, md2.ID(), shards, fsOpts) - - opts := NewCacheOptions(). - SetFilesystemOptions(fsOpts). - SetInstrumentOptions(fsOpts.InstrumentOptions()). - SetNamespaceDetails([]NamespaceDetails{ - { - Namespace: md1, - Shards: []uint32{0, 1}, - }, - { - Namespace: md2, - Shards: []uint32{0, 1}, - }, - }) - cache, err := NewCache(opts) - require.NoError(t, err) - - infoFilesByNamespace := cache.ReadInfoFiles() - require.NotEmpty(t, infoFilesByNamespace) - - // Ensure we have two namespaces. - require.Equal(t, 2, len(infoFilesByNamespace)) - - // Ensure each shard has three info files (one for each fileset written). - infoFiles, err := cache.IndexInfoFilesForNamespace(md1) - require.NoError(t, err) - require.Equal(t, 3, len(infoFiles)) - - infoFiles, err = cache.IndexInfoFilesForNamespace(md2) - require.NoError(t, err) - require.Equal(t, 3, len(infoFiles)) -} - func testNamespaceMetadata(t *testing.T, nsID ident.ID) namespace.Metadata { rOpts := testRetentionOptions.SetBlockSize(testBlockSize) md, err := namespace.NewMetadata(nsID, testNamespaceOptions. @@ -203,45 +154,6 @@ type testSeries struct { data []byte } -func writeIndexFilesets( - t *testing.T, - namespace ident.ID, - shards map[uint32]struct{}, - fsOpts fs.Options, -) { - blockStart := testStart - blockSize := 10 * time.Hour - numBlocks := 3 - for i := 0; i < numBlocks; i++ { - writeIndexFiles(t, namespace, shards, blockStart.Add(time.Duration(i)*blockSize), - blockSize, fsOpts) - } -} - -func writeIndexFiles( - t *testing.T, - namespace ident.ID, - shards map[uint32]struct{}, - blockStart time.Time, - blockSize time.Duration, - fsOpts fs.Options, -) { - idxWriter, err := fs.NewIndexWriter(fsOpts) - require.NoError(t, err) - require.NoError(t, idxWriter.Open(fs.IndexWriterOpenOptions{ - Identifier: fs.FileSetFileIdentifier{ - FileSetContentType: persist.FileSetIndexContentType, - Namespace: namespace, - BlockStart: blockStart, - }, - BlockSize: blockSize, - FileSetType: persist.FileSetFlushType, - Shards: shards, - IndexVolumeType: idxpersist.DefaultIndexVolumeType, - })) - require.NoError(t, idxWriter.Close()) -} - func writeFilesets(t *testing.T, namespace ident.ID, shard uint32, fsOpts fs.Options) { inputs := []struct { start time.Time diff --git a/src/dbnode/storage/bootstrap/result/result_index.go b/src/dbnode/storage/bootstrap/result/result_index.go index 0bfb7ad4ed..1a398139a1 100644 --- a/src/dbnode/storage/bootstrap/result/result_index.go +++ b/src/dbnode/storage/bootstrap/result/result_index.go @@ -312,11 +312,6 @@ func (b IndexBlockByVolumeType) SetBlock(volumeType persist.IndexVolumeType, blo b.data[volumeType] = block } -// DeleteBlock deletes an IndexBlock for volumeType. -func (b IndexBlockByVolumeType) DeleteBlock(volumeType persist.IndexVolumeType) { - delete(b.data, volumeType) -} - // Iter returns the underlying iterable map data. func (b IndexBlockByVolumeType) Iter() map[persist.IndexVolumeType]IndexBlock { return b.data diff --git a/src/dbnode/storage/bootstrap/types.go b/src/dbnode/storage/bootstrap/types.go index afdcc53450..0a64face66 100644 --- a/src/dbnode/storage/bootstrap/types.go +++ b/src/dbnode/storage/bootstrap/types.go @@ -430,9 +430,6 @@ type InfoFileResultsPerShard map[uint32][]fs.ReadInfoFileResult // InfoFilesByNamespace maps a namespace to info files grouped by shard. type InfoFilesByNamespace map[namespace.Metadata]InfoFileResultsPerShard -// IndexInfoFilesByNamespace maps a namespace to index info files. -type IndexInfoFilesByNamespace map[namespace.Metadata][]fs.ReadIndexInfoFileResult - // Cache provides a snapshot of info files for use throughout all stages of the bootstrap. type Cache interface { // InfoFilesForNamespace returns the info files grouped by namespace. @@ -441,9 +438,6 @@ type Cache interface { // InfoFilesForShard returns the info files grouped by shard for the provided namespace. InfoFilesForShard(ns namespace.Metadata, shard uint32) ([]fs.ReadInfoFileResult, error) - // IndexInfoFilesForNamespace returns the index info files. - IndexInfoFilesForNamespace(ns namespace.Metadata) ([]fs.ReadIndexInfoFileResult, error) - // ReadInfoFiles returns info file results for each shard grouped by namespace. A cached copy // is returned if the info files have already been read. ReadInfoFiles() InfoFilesByNamespace diff --git a/src/dbnode/storage/cleanup.go b/src/dbnode/storage/cleanup.go index 6ce83e1048..96dde48304 100644 --- a/src/dbnode/storage/cleanup.go +++ b/src/dbnode/storage/cleanup.go @@ -39,21 +39,13 @@ import ( "go.uber.org/zap" ) -type ( - commitLogFilesFn func(commitlog.Options) ( - persist.CommitLogFiles, - []commitlog.ErrorWithPath, - error, - ) - snapshotMetadataFilesFn func(fs.Options) ( - []fs.SnapshotMetadata, - []fs.SnapshotMetadataErrorWithPaths, - error, - ) -) +type commitLogFilesFn func(commitlog.Options) (persist.CommitLogFiles, []commitlog.ErrorWithPath, error) +type snapshotMetadataFilesFn func(fs.Options) ([]fs.SnapshotMetadata, []fs.SnapshotMetadataErrorWithPaths, error) type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error) +type deleteFilesFn func(files []string) error + type deleteInactiveDirectoriesFn func(parentDirPath string, activeDirNames []string) error // Narrow interface so as not to expose all the functionality of the commitlog @@ -76,7 +68,7 @@ type cleanupManager struct { snapshotMetadataFilesFn snapshotMetadataFilesFn snapshotFilesFn snapshotFilesFn - deleteFilesFn fs.DeleteFilesFn + deleteFilesFn deleteFilesFn deleteInactiveDirectoriesFn deleteInactiveDirectoriesFn warmFlushCleanupInProgress bool coldFlushCleanupInProgress bool @@ -221,7 +213,6 @@ func (m *cleanupManager) ColdFlushCleanup(t time.Time, isBootstrapped bool) erro return multiErr.FinalError() } - func (m *cleanupManager) Report() { m.RLock() coldFlushCleanupInProgress := m.coldFlushCleanupInProgress diff --git a/src/dbnode/storage/index.go b/src/dbnode/storage/index.go index 6fb04ba061..c811175b59 100644 --- a/src/dbnode/storage/index.go +++ b/src/dbnode/storage/index.go @@ -87,7 +87,9 @@ const ( defaultFlushDocsBatchSize = 8192 ) -var allQuery = idx.NewAllQuery() +var ( + allQuery = idx.NewAllQuery() +) // nolint: maligned type nsIndex struct { @@ -107,7 +109,7 @@ type nsIndex struct { namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager indexFilesetsBeforeFn indexFilesetsBeforeFn - deleteFilesFn fs.DeleteFilesFn + deleteFilesFn deleteFilesFn readIndexInfoFilesFn readIndexInfoFilesFn newBlockFn index.NewBlockFn @@ -1994,10 +1996,6 @@ func (i *nsIndex) CleanupExpiredFileSets(t time.Time) error { return i.deleteFilesFn(filesets) } -// CleanupDuplicateFileSets only considers an index fileset of the same index volume type -// that covers a superset of shard time ranges as a dupe. We can have index filesets -// of the default volume type that have non-overlapping shard time ranges in the node leave -// case where we accept new shards and a index fileset is persisted to disk w/ the new shards. func (i *nsIndex) CleanupDuplicateFileSets() error { fsOpts := i.opts.CommitLogOptions().FilesystemOptions() infoFiles := i.readIndexInfoFilesFn( diff --git a/src/dbnode/storage/namespace.go b/src/dbnode/storage/namespace.go index 6b07920c19..739af94825 100644 --- a/src/dbnode/storage/namespace.go +++ b/src/dbnode/storage/namespace.go @@ -41,6 +41,7 @@ import ( "github.com/m3db/m3/src/dbnode/ts" "github.com/m3db/m3/src/dbnode/ts/writes" "github.com/m3db/m3/src/dbnode/x/xio" + "github.com/m3db/m3/src/m3ninx/doc" "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/context" xerrors "github.com/m3db/m3/src/x/errors" @@ -1793,3 +1794,11 @@ func (n *dbNamespace) aggregateTiles( return processedTileCount, nil } + +func (n *dbNamespace) DocRef(id ident.ID) (doc.Document, bool, error) { + shard, _, err := n.readableShardFor(id) + if err != nil { + return doc.Document{}, false, err + } + return shard.DocRef(id) +} diff --git a/src/dbnode/storage/shard.go b/src/dbnode/storage/shard.go index 16fb3f4ae9..061df21615 100644 --- a/src/dbnode/storage/shard.go +++ b/src/dbnode/storage/shard.go @@ -171,7 +171,7 @@ type dbShard struct { newFSMergeWithMemFn newFSMergeWithMemFn filesetsFn filesetsFn filesetPathsBeforeFn filesetPathsBeforeFn - deleteFilesFn fs.DeleteFilesFn + deleteFilesFn deleteFilesFn snapshotFilesFn snapshotFilesFn sleepFn func(time.Duration) identifierPool ident.Pool @@ -1325,6 +1325,7 @@ func (s *dbShard) insertSeriesForIndexingAsyncBatched( entryRefCountIncremented: true, }, }) + // i.e. unable to enqueue into shard insert queue if err != nil { entry.OnIndexFinalize(indexBlockStart) // release any reference's we've held for indexing diff --git a/src/dbnode/storage/storage_mock.go b/src/dbnode/storage/storage_mock.go index ab66619c1d..ccc6a21831 100644 --- a/src/dbnode/storage/storage_mock.go +++ b/src/dbnode/storage/storage_mock.go @@ -1150,6 +1150,22 @@ func (mr *MockNamespaceMockRecorder) SetReadOnly(value interface{}) *gomock.Call return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetReadOnly", reflect.TypeOf((*MockNamespace)(nil).SetReadOnly), value) } +// DocRef mocks base method +func (m *MockNamespace) DocRef(id ident.ID) (doc.Document, bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DocRef", id) + ret0, _ := ret[0].(doc.Document) + ret1, _ := ret[1].(bool) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// DocRef indicates an expected call of DocRef +func (mr *MockNamespaceMockRecorder) DocRef(id interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DocRef", reflect.TypeOf((*MockNamespace)(nil).DocRef), id) +} + // MockdatabaseNamespace is a mock of databaseNamespace interface type MockdatabaseNamespace struct { ctrl *gomock.Controller @@ -1326,6 +1342,22 @@ func (mr *MockdatabaseNamespaceMockRecorder) SetReadOnly(value interface{}) *gom return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetReadOnly", reflect.TypeOf((*MockdatabaseNamespace)(nil).SetReadOnly), value) } +// DocRef mocks base method +func (m *MockdatabaseNamespace) DocRef(id ident.ID) (doc.Document, bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DocRef", id) + ret0, _ := ret[0].(doc.Document) + ret1, _ := ret[1].(bool) + ret2, _ := ret[2].(error) + return ret0, ret1, ret2 +} + +// DocRef indicates an expected call of DocRef +func (mr *MockdatabaseNamespaceMockRecorder) DocRef(id interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DocRef", reflect.TypeOf((*MockdatabaseNamespace)(nil).DocRef), id) +} + // Close mocks base method func (m *MockdatabaseNamespace) Close() error { m.ctrl.T.Helper() diff --git a/src/dbnode/storage/types.go b/src/dbnode/storage/types.go index 579cae8c7d..87aa6b3da1 100644 --- a/src/dbnode/storage/types.go +++ b/src/dbnode/storage/types.go @@ -285,6 +285,9 @@ type Namespace interface { // SetReadOnly sets the value of ReadOnly option. SetReadOnly(value bool) + + // DocRef returns the doc if already present in a namespace shard. + DocRef(id ident.ID) (doc.Document, bool, error) } // NamespacesByID is a sortable slice of namespaces by ID. diff --git a/src/query/api/v1/handler/database/common.go b/src/query/api/v1/handler/database/common.go index 950bed6396..525268e4d8 100644 --- a/src/query/api/v1/handler/database/common.go +++ b/src/query/api/v1/handler/database/common.go @@ -24,8 +24,8 @@ import ( clusterclient "github.com/m3db/m3/src/cluster/client" dbconfig "github.com/m3db/m3/src/cmd/services/m3dbnode/config" "github.com/m3db/m3/src/cmd/services/m3query/config" - "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" + "github.com/m3db/m3/src/query/util/queryhttp" "github.com/m3db/m3/src/x/instrument" ) @@ -39,7 +39,7 @@ type Handler struct { // RegisterRoutes registers the namespace routes func RegisterRoutes( - addRoute handler.AddRouteFn, + r *queryhttp.EndpointRegistry, client clusterclient.Client, cfg config.Configuration, embeddedDbCfg *dbconfig.DBConfiguration, @@ -54,10 +54,18 @@ func RegisterRoutes( // Register the same handler under two different endpoints. This just makes explaining things in // our documentation easier so we can separate out concepts, but share the underlying code. - if err := addRoute(CreateURL, createHandler, CreateHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: CreateURL, + Handler: createHandler, + Methods: []string{CreateHTTPMethod}, + }); err != nil { return err } - if err := addRoute(CreateNamespaceURL, createHandler, CreateNamespaceHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: CreateNamespaceURL, + Handler: createHandler, + Methods: []string{CreateNamespaceHTTPMethod}, + }); err != nil { return err } diff --git a/src/query/api/v1/handler/graphite/find.go b/src/query/api/v1/handler/graphite/find.go index 81b2aca985..a919031f18 100644 --- a/src/query/api/v1/handler/graphite/find.go +++ b/src/query/api/v1/handler/graphite/find.go @@ -168,7 +168,7 @@ func (h *grahiteFindHandler) ServeHTTP( prefix += "." } - handleroptions.AddWarningHeaders(w, meta) + handleroptions.AddResponseHeaders(w, meta, opts) // TODO: Support multiple result types if err = findResultsJSON(w, prefix, seenMap); err != nil { logger.Error("unable to print find results", zap.Error(err)) diff --git a/src/query/api/v1/handler/graphite/find_test.go b/src/query/api/v1/handler/graphite/find_test.go index f7651ff976..bc352f65a0 100644 --- a/src/query/api/v1/handler/graphite/find_test.go +++ b/src/query/api/v1/handler/graphite/find_test.go @@ -227,8 +227,11 @@ func testFind(t *testing.T, httpMethod string, ex bool, ex2 bool, header string) // setup storage and handler store := setupStorage(ctrl, ex, ex2) - builder := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + builder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetFetchOptionsBuilder(builder). SetStorage(store) diff --git a/src/query/api/v1/handler/graphite/render.go b/src/query/api/v1/handler/graphite/render.go index 489531f7ad..2ebe317b21 100644 --- a/src/query/api/v1/handler/graphite/render.go +++ b/src/query/api/v1/handler/graphite/render.go @@ -55,6 +55,7 @@ var ( // A renderHandler implements the graphite /render endpoint, including full // support for executing functions. It only works against data in M3. type renderHandler struct { + opts options.HandlerOptions engine *native.Engine queryContextOpts models.QueryContextOptions graphiteOpts graphite.M3WrappedStorageOptions @@ -70,6 +71,7 @@ func NewRenderHandler(opts options.HandlerOptions) http.Handler { wrappedStore := graphite.NewM3WrappedStorage(opts.Storage(), opts.M3DBOptions(), opts.InstrumentOpts(), opts.GraphiteStorageOptions()) return &renderHandler{ + opts: opts, engine: native.NewEngine(wrappedStore), queryContextOpts: opts.QueryContextOptions(), graphiteOpts: opts.GraphiteStorageOptions(), @@ -95,7 +97,7 @@ func (h *renderHandler) serveHTTP( r *http.Request, ) error { reqCtx := context.WithValue(r.Context(), handler.HeaderKey, r.Header) - p, err := ParseRenderRequest(r) + p, fetchOpts, err := ParseRenderRequest(r, h.opts) if err != nil { return xhttp.NewError(err, http.StatusBadRequest) } @@ -211,7 +213,7 @@ func (h *renderHandler) serveHTTP( SortApplied: true, } - handleroptions.AddWarningHeaders(w, meta) + handleroptions.AddResponseHeaders(w, meta, fetchOpts) return WriteRenderResponse(w, response, p.Format, renderResultsJSONOptions{ renderSeriesAllNaNs: h.graphiteOpts.RenderSeriesAllNaNs, diff --git a/src/query/api/v1/handler/graphite/render_parser.go b/src/query/api/v1/handler/graphite/render_parser.go index 2b8124e74c..2659fd4969 100644 --- a/src/query/api/v1/handler/graphite/render_parser.go +++ b/src/query/api/v1/handler/graphite/render_parser.go @@ -29,9 +29,11 @@ import ( "time" "github.com/m3db/m3/src/query/api/v1/handler/graphite/pickle" + "github.com/m3db/m3/src/query/api/v1/options" "github.com/m3db/m3/src/query/graphite/errors" "github.com/m3db/m3/src/query/graphite/graphite" "github.com/m3db/m3/src/query/graphite/ts" + "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/util/json" xhttp "github.com/m3db/m3/src/x/net/http" ) @@ -68,7 +70,6 @@ func WriteRenderResponse( const ( tzOffsetForAbsoluteTime = time.Duration(0) maxTimeout = time.Minute - defaultTimeout = time.Second * 5 ) // RenderRequest are the arguments to a render call. @@ -83,21 +84,28 @@ type RenderRequest struct { } // ParseRenderRequest parses the arguments to a render call from an incoming request. -func ParseRenderRequest(r *http.Request) (RenderRequest, error) { - var ( - p RenderRequest - err error - now = time.Now() - ) +func ParseRenderRequest( + r *http.Request, + opts options.HandlerOptions, +) (RenderRequest, *storage.FetchOptions, error) { + fetchOpts, err := opts.FetchOptionsBuilder().NewFetchOptions(r) + if err != nil { + return RenderRequest{}, nil, err + } - if err = r.ParseForm(); err != nil { - return p, err + if err := r.ParseForm(); err != nil { + return RenderRequest{}, nil, err } + var ( + p = RenderRequest{ + Timeout: fetchOpts.Timeout, + } + now = time.Now() + ) p.Targets = r.Form["target"] - if len(p.Targets) == 0 { - return p, errNoTarget + return p, nil, errNoTarget } fromString, untilString := r.FormValue("from"), r.FormValue("until") @@ -114,7 +122,7 @@ func ParseRenderRequest(r *http.Request) (RenderRequest, error) { now, tzOffsetForAbsoluteTime, ); err != nil { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'from': %s", fromString)) + return p, nil, errors.NewInvalidParamsError(fmt.Errorf("invalid 'from': %s", fromString)) } if p.Until, err = graphite.ParseTime( @@ -122,11 +130,11 @@ func ParseRenderRequest(r *http.Request) (RenderRequest, error) { now, tzOffsetForAbsoluteTime, ); err != nil { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'until': %s", untilString)) + return p, nil, errors.NewInvalidParamsError(fmt.Errorf("invalid 'until': %s", untilString)) } if !p.From.Before(p.Until) { - return p, errFromNotBeforeUntil + return p, nil, errFromNotBeforeUntil } // If this is a real-time query, and the query range is large enough, we shift the query @@ -147,7 +155,7 @@ func ParseRenderRequest(r *http.Request) (RenderRequest, error) { dur, err := graphite.ParseDuration(offset) if err != nil { err = errors.NewInvalidParamsError(err) - return p, errors.NewRenamedError(err, fmt.Errorf("invalid 'offset': %s", err)) + return p, nil, errors.NewRenamedError(err, fmt.Errorf("invalid 'offset': %s", err)) } p.Until = p.Until.Add(dur) @@ -159,7 +167,7 @@ func ParseRenderRequest(r *http.Request) (RenderRequest, error) { p.MaxDataPoints, err = strconv.ParseInt(maxDataPointsString, 10, 64) if err != nil || p.MaxDataPoints < 1 { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'maxDataPoints': %s", maxDataPointsString)) + return p, nil, errors.NewInvalidParamsError(fmt.Errorf("invalid 'maxDataPoints': %s", maxDataPointsString)) } } else { p.MaxDataPoints = math.MaxInt64 @@ -172,28 +180,14 @@ func ParseRenderRequest(r *http.Request) (RenderRequest, error) { p.From, tzOffsetForAbsoluteTime, ); err != nil && len(compareString) != 0 { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'compare': %s", compareString)) + return p, nil, errors.NewInvalidParamsError(fmt.Errorf("invalid 'compare': %s", compareString)) } else if p.From.Before(compareFrom) { - return p, errors.NewInvalidParamsError(fmt.Errorf("'compare' must be in the past")) + return p, nil, errors.NewInvalidParamsError(fmt.Errorf("'compare' must be in the past")) } else { p.Compare = compareFrom.Sub(p.From) } - timeout := r.FormValue("timeout") - if timeout != "" { - duration, err := time.ParseDuration(timeout) - if err != nil { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'timeout': %v", err)) - } - if duration > maxTimeout { - return p, errors.NewInvalidParamsError(fmt.Errorf("invalid 'timeout': greater than %v", maxTimeout)) - } - p.Timeout = duration - } else { - p.Timeout = defaultTimeout - } - - return p, nil + return p, fetchOpts, nil } type renderResultsJSONOptions struct { diff --git a/src/query/api/v1/handler/graphite/render_test.go b/src/query/api/v1/handler/graphite/render_test.go index eb1ebc2cfa..de764d2539 100644 --- a/src/query/api/v1/handler/graphite/render_test.go +++ b/src/query/api/v1/handler/graphite/render_test.go @@ -23,12 +23,13 @@ package graphite import ( "fmt" "io/ioutil" + "math" "net/http" "net/http/httptest" - "math" "testing" "time" + "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/options" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/graphite/graphite" @@ -45,6 +46,18 @@ import ( "github.com/stretchr/testify/require" ) +func testHandlerOptions(t *testing.T) options.HandlerOptions { + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) + + return options.EmptyHandlerOptions(). + SetQueryContextOptions(models.QueryContextOptions{}). + SetFetchOptionsBuilder(fetchOptsBuilder) +} + func makeBlockResult( ctrl *gomock.Controller, results *storage.FetchResult, @@ -78,9 +91,7 @@ func makeBlockResult( func TestParseNoQuery(t *testing.T) { mockStorage := mock.NewMockStorage() - opts := options.EmptyHandlerOptions(). - SetStorage(mockStorage). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(mockStorage) handler := NewRenderHandler(opts) recorder := httptest.NewRecorder() @@ -99,9 +110,7 @@ func TestParseQueryNoResults(t *testing.T) { store.EXPECT().FetchBlocks(gomock.Any(), gomock.Any(), gomock.Any()). Return(blockResult, nil) - opts := options.EmptyHandlerOptions(). - SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(store) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) @@ -144,9 +153,7 @@ func TestParseQueryResults(t *testing.T) { store.EXPECT().FetchBlocks(gomock.Any(), gomock.Any(), gomock.Any()). Return(blockResult, nil) - opts := options.EmptyHandlerOptions(). - SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(store) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) @@ -198,9 +205,7 @@ func TestParseQueryResultsMaxDatapoints(t *testing.T) { store.EXPECT().FetchBlocks(gomock.Any(), gomock.Any(), gomock.Any()). Return(blockResult, nil) - opts := options.EmptyHandlerOptions(). - SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(store) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) @@ -254,9 +259,7 @@ func TestParseQueryResultsMultiTarget(t *testing.T) { store.EXPECT().FetchBlocks(gomock.Any(), gomock.Any(), gomock.Any()). Return(makeBlockResult(ctrl, fr), nil) - opts := options.EmptyHandlerOptions(). - SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(store) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) @@ -317,9 +320,7 @@ func TestParseQueryResultsMultiTargetWithLimits(t *testing.T) { store.EXPECT().FetchBlocks(gomock.Any(), gomock.Any(), gomock.Any()). Return(makeBlockResult(ctrl, frTwo), nil) - opts := options.EmptyHandlerOptions(). - SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}) + opts := testHandlerOptions(t).SetStorage(store) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) @@ -364,9 +365,9 @@ func TestParseQueryResultsAllNaN(t *testing.T) { graphiteStorageOpts := graphiteStorage.M3WrappedStorageOptions{ RenderSeriesAllNaNs: true, } - opts := options.EmptyHandlerOptions(). + opts := testHandlerOptions(t). SetStorage(store). - SetQueryContextOptions(models.QueryContextOptions{}).SetGraphiteStorageOptions(graphiteStorageOpts) + SetGraphiteStorageOptions(graphiteStorageOpts) handler := NewRenderHandler(opts) req := newGraphiteReadHTTPRequest(t) diff --git a/src/query/api/v1/handler/namespace/common.go b/src/query/api/v1/handler/namespace/common.go index 8290490395..f58ccf05d2 100644 --- a/src/query/api/v1/handler/namespace/common.go +++ b/src/query/api/v1/handler/namespace/common.go @@ -31,9 +31,9 @@ import ( "github.com/m3db/m3/src/cluster/kv" nsproto "github.com/m3db/m3/src/dbnode/generated/proto/namespace" "github.com/m3db/m3/src/dbnode/namespace" - "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/storage/m3" + "github.com/m3db/m3/src/query/util/queryhttp" "github.com/m3db/m3/src/x/instrument" xhttp "github.com/m3db/m3/src/x/net/http" ) @@ -106,89 +106,94 @@ type applyMiddlewareFn func( r *http.Request, ) -type addRouteFn func( - path string, - applyMiddlewareFn applyMiddlewareFn, - methods ...string, -) error - // RegisterRoutes registers the namespace routes. func RegisterRoutes( - addRouteFn handler.AddRouteFn, + r *queryhttp.EndpointRegistry, client clusterclient.Client, clusters m3.Clusters, defaults []handleroptions.ServiceOptionsDefault, instrumentOpts instrument.Options, ) error { - addRoute := applyMiddlewareToRoute(addRouteFn, defaults) + applyMiddleware := func( + f func(svc handleroptions.ServiceNameAndDefaults, + w http.ResponseWriter, r *http.Request), + defaults []handleroptions.ServiceOptionsDefault, + ) http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + svc := handleroptions.ServiceNameAndDefaults{ + ServiceName: handleroptions.M3DBServiceName, + Defaults: defaults, + } + f(svc, w, r) + }) + } // Get M3DB namespaces. - getHandler := NewGetHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBGetURL, getHandler, GetHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBGetURL, + Handler: applyMiddleware(NewGetHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{GetHTTPMethod}, + }); err != nil { return err } // Add M3DB namespaces. - addHandler := NewAddHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBAddURL, addHandler, AddHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBAddURL, + Handler: applyMiddleware(NewAddHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{AddHTTPMethod}, + }); err != nil { return err } // Update M3DB namespaces. - updateHandler := NewUpdateHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBUpdateURL, updateHandler, UpdateHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBUpdateURL, + Handler: applyMiddleware(NewUpdateHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{UpdateHTTPMethod}, + }); err != nil { return err } // Delete M3DB namespaces. - deleteHandler := NewDeleteHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBDeleteURL, deleteHandler, DeleteHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBDeleteURL, + Handler: applyMiddleware(NewDeleteHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{DeleteHTTPMethod}, + }); err != nil { return err } // Deploy M3DB schemas. - schemaHandler := NewSchemaHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBSchemaURL, schemaHandler, SchemaDeployHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBSchemaURL, + Handler: applyMiddleware(NewSchemaHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{SchemaDeployHTTPMethod}, + }); err != nil { return err } // Reset M3DB schemas. - schemaResetHandler := NewSchemaResetHandler(client, instrumentOpts).ServeHTTP - if err := addRoute(M3DBSchemaURL, schemaResetHandler, DeleteHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBSchemaURL, + Handler: applyMiddleware(NewSchemaResetHandler(client, instrumentOpts).ServeHTTP, defaults), + Methods: []string{DeleteHTTPMethod}, + }); err != nil { return err } // Mark M3DB namespace as ready. - readyHandler := NewReadyHandler(client, clusters, instrumentOpts).ServeHTTP - if err := addRoute(M3DBReadyURL, readyHandler, ReadyHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: M3DBReadyURL, + Handler: applyMiddleware(NewReadyHandler(client, clusters, instrumentOpts).ServeHTTP, defaults), + Methods: []string{ReadyHTTPMethod}, + }); err != nil { return err } return nil } -func applyMiddlewareToRoute( - addRouteFn handler.AddRouteFn, - defaults []handleroptions.ServiceOptionsDefault, -) addRouteFn { - applyMiddleware := func( - applyMiddlewareFn applyMiddlewareFn, - defaults []handleroptions.ServiceOptionsDefault, - ) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - svc := handleroptions.ServiceNameAndDefaults{ - ServiceName: handleroptions.M3DBServiceName, - Defaults: defaults, - } - applyMiddlewareFn(svc, w, r) - }) - } - - return func(path string, f applyMiddlewareFn, methods ...string) error { - return addRouteFn(path, applyMiddleware(f, defaults), methods...) - } -} - func validateNamespaceAggregationOptions(mds []namespace.Metadata) error { resolutionRetentionMap := make(map[resolutionRetentionKey]bool, len(mds)) diff --git a/src/query/api/v1/handler/placement/common.go b/src/query/api/v1/handler/placement/common.go index 4afba97187..d28ec4096e 100644 --- a/src/query/api/v1/handler/placement/common.go +++ b/src/query/api/v1/handler/placement/common.go @@ -37,9 +37,8 @@ import ( "github.com/m3db/m3/src/cluster/services" "github.com/m3db/m3/src/cluster/shard" "github.com/m3db/m3/src/cmd/services/m3query/config" - "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" - "github.com/m3db/m3/src/query/util/logging" + "github.com/m3db/m3/src/query/util/queryhttp" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/instrument" xhttp "github.com/m3db/m3/src/x/net/http" @@ -221,113 +220,119 @@ func ConvertInstancesProto(instancesProto []*placementpb.Instance) ([]placement. // RegisterRoutes registers the placement routes func RegisterRoutes( - addRoute handler.AddRouteFn, + r *queryhttp.EndpointRegistry, defaults []handleroptions.ServiceOptionsDefault, opts HandlerOptions, ) error { // Init var ( initHandler = NewInitHandler(opts) - initFn = applyMiddleware(initHandler.ServeHTTP, defaults, opts.instrumentOptions) + initFn = applyMiddleware(initHandler.ServeHTTP, defaults) ) - - if err := addRoute(M3DBInitURL, initFn, InitHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggInitURL, initFn, InitHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorInitURL, initFn, InitHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBInitURL, + M3AggInitURL, + M3CoordinatorInitURL, + }, queryhttp.RegisterPathsOptions{ + Handler: initFn, + Methods: []string{InitHTTPMethod}, + }); err != nil { return err } // Get var ( getHandler = NewGetHandler(opts) - getFn = applyMiddleware(getHandler.ServeHTTP, defaults, opts.instrumentOptions) + getFn = applyMiddleware(getHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBGetURL, getFn, GetHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggGetURL, getFn, GetHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorGetURL, getFn, GetHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBGetURL, + M3AggGetURL, + M3CoordinatorGetURL, + }, queryhttp.RegisterPathsOptions{ + Handler: getFn, + Methods: []string{GetHTTPMethod}, + }); err != nil { return err } // Delete all var ( deleteAllHandler = NewDeleteAllHandler(opts) - deleteAllFn = applyMiddleware(deleteAllHandler.ServeHTTP, defaults, opts.instrumentOptions) + deleteAllFn = applyMiddleware(deleteAllHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBDeleteAllURL, deleteAllFn, DeleteAllHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggDeleteAllURL, deleteAllFn, DeleteAllHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorDeleteAllURL, deleteAllFn, DeleteAllHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBDeleteAllURL, + M3AggDeleteAllURL, + M3CoordinatorDeleteAllURL, + }, queryhttp.RegisterPathsOptions{ + Handler: deleteAllFn, + Methods: []string{DeleteAllHTTPMethod}, + }); err != nil { return err } // Add var ( addHandler = NewAddHandler(opts) - addFn = applyMiddleware(addHandler.ServeHTTP, defaults, opts.instrumentOptions) + addFn = applyMiddleware(addHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBAddURL, addFn, AddHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggAddURL, addFn, AddHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorAddURL, addFn, AddHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBAddURL, + M3AggAddURL, + M3CoordinatorAddURL, + }, queryhttp.RegisterPathsOptions{ + Handler: addFn, + Methods: []string{AddHTTPMethod}, + }); err != nil { return err } // Delete var ( deleteHandler = NewDeleteHandler(opts) - deleteFn = applyMiddleware(deleteHandler.ServeHTTP, defaults, opts.instrumentOptions) + deleteFn = applyMiddleware(deleteHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBDeleteURL, deleteFn, DeleteHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggDeleteURL, deleteFn, DeleteHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorDeleteURL, deleteFn, DeleteHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBDeleteURL, + M3AggDeleteURL, + M3CoordinatorDeleteURL, + }, queryhttp.RegisterPathsOptions{ + Handler: deleteFn, + Methods: []string{DeleteHTTPMethod}, + }); err != nil { return err } // Replace var ( replaceHandler = NewReplaceHandler(opts) - replaceFn = applyMiddleware(replaceHandler.ServeHTTP, defaults, opts.instrumentOptions) + replaceFn = applyMiddleware(replaceHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBReplaceURL, replaceFn, ReplaceHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggReplaceURL, replaceFn, ReplaceHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorReplaceURL, replaceFn, ReplaceHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBReplaceURL, + M3AggReplaceURL, + M3CoordinatorReplaceURL, + }, queryhttp.RegisterPathsOptions{ + Handler: replaceFn, + Methods: []string{ReplaceHTTPMethod}, + }); err != nil { return err } // Set var ( setHandler = NewSetHandler(opts) - setFn = applyMiddleware(setHandler.ServeHTTP, defaults, opts.instrumentOptions) + setFn = applyMiddleware(setHandler.ServeHTTP, defaults) ) - if err := addRoute(M3DBSetURL, setFn, SetHTTPMethod); err != nil { - return err - } - if err := addRoute(M3AggSetURL, setFn, SetHTTPMethod); err != nil { - return err - } - if err := addRoute(M3CoordinatorSetURL, setFn, SetHTTPMethod); err != nil { + if err := r.RegisterPaths([]string{ + M3DBSetURL, + M3AggSetURL, + M3CoordinatorSetURL, + }, queryhttp.RegisterPathsOptions{ + Handler: setFn, + Methods: []string{SetHTTPMethod}, + }); err != nil { return err } @@ -427,21 +432,10 @@ func validateAllAvailable(p placement.Placement) error { } func applyMiddleware( - f func(svc handleroptions.ServiceNameAndDefaults, w http.ResponseWriter, r *http.Request), - defaults []handleroptions.ServiceOptionsDefault, - instrumentOpts instrument.Options, -) http.Handler { - return logging.WithResponseTimeAndPanicErrorLoggingFunc( - parseServiceMiddleware(f, defaults), - instrumentOpts, - ) -} - -func parseServiceMiddleware( next func(svc handleroptions.ServiceNameAndDefaults, w http.ResponseWriter, r *http.Request), defaults []handleroptions.ServiceOptionsDefault, -) func(w http.ResponseWriter, r *http.Request) { - return func(w http.ResponseWriter, r *http.Request) { +) http.Handler { + return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { var ( svc = handleroptions.ServiceNameAndDefaults{Defaults: defaults} err error @@ -453,7 +447,7 @@ func parseServiceMiddleware( } next(svc, w, r) - } + }) } func parseServiceFromRequest(r *http.Request) (string, error) { diff --git a/src/query/api/v1/handler/prom/prom.go b/src/query/api/v1/handler/prom/prom.go index 3d11e4f133..9e04ddbfe5 100644 --- a/src/query/api/v1/handler/prom/prom.go +++ b/src/query/api/v1/handler/prom/prom.go @@ -84,22 +84,13 @@ func newDefaultOptions(hOpts options.HandlerOptions) opts { // NewReadHandler creates a handler to handle PromQL requests. func NewReadHandler(hOpts options.HandlerOptions, options ...Option) (http.Handler, error) { - return NewReadHandlerWithHooks(hOpts, &noopReadHandlerHooks{}, options...) -} - -// NewReadHandlerWithHooks creates a handler for PromQL requests that accepts ReadHandlerHooks. -func NewReadHandlerWithHooks( - hOpts options.HandlerOptions, - hooks ReadHandlerHooks, - options ...Option, -) (http.Handler, error) { queryable := prometheus.NewPrometheusQueryable( prometheus.PrometheusOptions{ Storage: hOpts.Storage(), InstrumentOptions: hOpts.InstrumentOpts(), }) - return newReadHandler(hOpts, hooks, queryable, options...) + return newReadHandler(hOpts, queryable, options...) } // ApplyRangeWarnings applies warnings encountered during execution. diff --git a/src/query/api/v1/handler/prom/read.go b/src/query/api/v1/handler/prom/read.go index 525cb33c26..726e43d495 100644 --- a/src/query/api/v1/handler/prom/read.go +++ b/src/query/api/v1/handler/prom/read.go @@ -41,16 +41,6 @@ import ( "go.uber.org/zap" ) -// ReadHandlerHooks allows dynamic plugging into read request processing. -type ReadHandlerHooks interface { - // OnParsedRequest gets invoked after parsing request arguments. - OnParsedRequest( - context.Context, - *http.Request, - models.RequestParams, - ) (models.RequestParams, error) -} - // NewQueryFn creates a new promql Query. type NewQueryFn func( engine *promql.Engine, @@ -86,7 +76,6 @@ var ( type readHandler struct { engine *promql.Engine - hooks ReadHandlerHooks queryable promstorage.Queryable hOpts options.HandlerOptions scope tally.Scope @@ -96,7 +85,6 @@ type readHandler struct { func newReadHandler( hOpts options.HandlerOptions, - hooks ReadHandlerHooks, queryable promstorage.Queryable, options ...Option, ) (http.Handler, error) { @@ -112,7 +100,6 @@ func newReadHandler( return &readHandler{ engine: opts.promQLEngine, - hooks: hooks, queryable: queryable, hOpts: hOpts, opts: opts, @@ -136,11 +123,7 @@ func (h *readHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - params, err := h.hooks.OnParsedRequest(ctx, r, request.Params) - if err != nil { - RespondError(w, err) - return - } + params := request.Params // NB (@shreyas): We put the FetchOptions in context so it can be // retrieved in the queryable object as there is no other way to pass @@ -186,19 +169,9 @@ func (h *readHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { zap.Bool("instant", h.opts.instant)) } - handleroptions.AddWarningHeaders(w, resultMetadata) + handleroptions.AddResponseHeaders(w, resultMetadata, fetchOptions) Respond(w, &QueryData{ Result: res.Value, ResultType: res.Value.Type(), }, res.Warnings) } - -type noopReadHandlerHooks struct{} - -func (h *noopReadHandlerHooks) OnParsedRequest( - _ context.Context, - _ *http.Request, - params models.RequestParams, -) (models.RequestParams, error) { - return params, nil -} diff --git a/src/query/api/v1/handler/prom/read_test.go b/src/query/api/v1/handler/prom/read_test.go index fe34542684..8dcc9d15d4 100644 --- a/src/query/api/v1/handler/prom/read_test.go +++ b/src/query/api/v1/handler/prom/read_test.go @@ -29,7 +29,6 @@ import ( "testing" "time" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/native" "github.com/m3db/m3/src/query/api/v1/options" @@ -59,12 +58,11 @@ type testHandlers struct { } func setupTest(t *testing.T) testHandlers { - timeoutOpts := &prometheus.TimeoutOpts{ - FetchTimeout: 15 * time.Second, + fetchOptsBuilderCfg := handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, } - - fetchOptsBuilderCfg := handleroptions.FetchOptionsBuilderOptions{} - fetchOptsBuilder := handleroptions.NewFetchOptionsBuilder(fetchOptsBuilderCfg) + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder(fetchOptsBuilderCfg) + require.NoError(t, err) instrumentOpts := instrument.NewOptions() engineOpts := executor.NewEngineOptions(). SetLookbackDuration(time.Minute). @@ -72,13 +70,12 @@ func setupTest(t *testing.T) testHandlers { engine := executor.NewEngine(engineOpts) hOpts := options.EmptyHandlerOptions(). SetFetchOptionsBuilder(fetchOptsBuilder). - SetEngine(engine). - SetTimeoutOpts(timeoutOpts) + SetEngine(engine) queryable := &mockQueryable{} - readHandler, err := newReadHandler(hOpts, &noopReadHandlerHooks{}, queryable, WithEngine(testPromQLEngine)) + readHandler, err := newReadHandler(hOpts, queryable, WithEngine(testPromQLEngine)) require.NoError(t, err) readInstantHandler, err := newReadHandler(hOpts, - &noopReadHandlerHooks{}, queryable, WithInstantEngine(testPromQLEngine)) + queryable, WithInstantEngine(testPromQLEngine)) require.NoError(t, err) return testHandlers{ queryable: queryable, diff --git a/src/query/api/v1/handler/prometheus/common.go b/src/query/api/v1/handler/prometheus/common.go index 5192b82a11..ded4732681 100644 --- a/src/query/api/v1/handler/prometheus/common.go +++ b/src/query/api/v1/handler/prometheus/common.go @@ -53,11 +53,6 @@ var ( roleName = []byte("role") ) -// TimeoutOpts stores options related to various timeout configurations. -type TimeoutOpts struct { - FetchTimeout time.Duration -} - // ParsePromCompressedRequestResult is the result of a // ParsePromCompressedRequest call. type ParsePromCompressedRequestResult struct { @@ -95,38 +90,6 @@ func ParsePromCompressedRequest( }, nil } -// ParseRequestTimeout parses the input request timeout with a default. -func ParseRequestTimeout( - r *http.Request, - configFetchTimeout time.Duration, -) (time.Duration, error) { - var timeout string - if v := r.FormValue("timeout"); v != "" { - timeout = v - } - // Note: Header should take precedence. - if v := r.Header.Get("timeout"); v != "" { - timeout = v - } - - if timeout == "" { - return configFetchTimeout, nil - } - - duration, err := time.ParseDuration(timeout) - if err != nil { - return 0, xerrors.NewInvalidParamsError( - fmt.Errorf("invalid 'timeout': %v", err)) - } - - if duration > maxTimeout { - return 0, xerrors.NewInvalidParamsError( - fmt.Errorf("invalid 'timeout': greater than %v", maxTimeout)) - } - - return duration, nil -} - // TagCompletionQueries are tag completion queries. type TagCompletionQueries struct { // Queries are the tag completion queries. diff --git a/src/query/api/v1/handler/prometheus/common_test.go b/src/query/api/v1/handler/prometheus/common_test.go index cacbfbfc44..7e7e0d091d 100644 --- a/src/query/api/v1/handler/prometheus/common_test.go +++ b/src/query/api/v1/handler/prometheus/common_test.go @@ -23,20 +23,15 @@ package prometheus import ( "bytes" "fmt" - "mime/multipart" "net/http/httptest" - "net/url" "strings" "testing" - "time" "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/test" xerrors "github.com/m3db/m3/src/x/errors" - xhttp "github.com/m3db/m3/src/x/net/http" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestPromCompressedReadSuccess(t *testing.T) { @@ -66,53 +61,6 @@ func TestPromCompressedReadInvalidEncoding(t *testing.T) { assert.True(t, xerrors.IsInvalidParams(err)) } -func TestTimeoutParseWithHeader(t *testing.T) { - req := httptest.NewRequest("POST", "/dummy", nil) - req.Header.Add("timeout", "1ms") - - timeout, err := ParseRequestTimeout(req, time.Second) - assert.NoError(t, err) - assert.Equal(t, timeout, time.Millisecond) - - req.Header.Del("timeout") - timeout, err = ParseRequestTimeout(req, 2*time.Minute) - assert.NoError(t, err) - assert.Equal(t, timeout, 2*time.Minute) - - req.Header.Add("timeout", "invalid") - _, err = ParseRequestTimeout(req, 15*time.Second) - assert.Error(t, err) - assert.True(t, xerrors.IsInvalidParams(err)) -} - -func TestTimeoutParseWithPostRequestParam(t *testing.T) { - params := url.Values{} - params.Add("timeout", "1ms") - - buff := bytes.NewBuffer(nil) - form := multipart.NewWriter(buff) - form.WriteField("timeout", "1ms") - require.NoError(t, form.Close()) - - req := httptest.NewRequest("POST", "/dummy", buff) - req.Header.Set(xhttp.HeaderContentType, form.FormDataContentType()) - - timeout, err := ParseRequestTimeout(req, time.Second) - assert.NoError(t, err) - assert.Equal(t, timeout, time.Millisecond) -} - -func TestTimeoutParseWithGetRequestParam(t *testing.T) { - params := url.Values{} - params.Add("timeout", "1ms") - - req := httptest.NewRequest("GET", "/dummy?"+params.Encode(), nil) - - timeout, err := ParseRequestTimeout(req, time.Second) - assert.NoError(t, err) - assert.Equal(t, timeout, time.Millisecond) -} - type writer struct { value string } diff --git a/src/query/api/v1/handler/prometheus/handleroptions/fetch_options.go b/src/query/api/v1/handler/prometheus/handleroptions/fetch_options.go index ac34a43f05..1e75b9f9c1 100644 --- a/src/query/api/v1/handler/prometheus/handleroptions/fetch_options.go +++ b/src/query/api/v1/handler/prometheus/handleroptions/fetch_options.go @@ -42,8 +42,11 @@ const ( StepParam = "step" // LookbackParam is the lookback parameter. LookbackParam = "lookback" - maxInt64 = float64(math.MaxInt64) - minInt64 = float64(math.MinInt64) + // TimeoutParam is the timeout parameter. + TimeoutParam = "timeout" + maxInt64 = float64(math.MaxInt64) + minInt64 = float64(math.MinInt64) + maxTimeout = 10 * time.Minute ) // FetchOptionsBuilder builds fetch options based on a request and default @@ -58,6 +61,12 @@ type FetchOptionsBuilder interface { type FetchOptionsBuilderOptions struct { Limits FetchOptionsBuilderLimitsOptions RestrictByTag *storage.RestrictByTag + Timeout time.Duration +} + +// Validate validates the fetch options builder options. +func (o FetchOptionsBuilderOptions) Validate() error { + return validateTimeout(o.Timeout) } // FetchOptionsBuilderLimitsOptions provides limits options to use when @@ -75,8 +84,11 @@ type fetchOptionsBuilder struct { // NewFetchOptionsBuilder returns a new fetch options builder. func NewFetchOptionsBuilder( opts FetchOptionsBuilderOptions, -) FetchOptionsBuilder { - return fetchOptionsBuilder{opts: opts} +) (FetchOptionsBuilder, error) { + if err := opts.Validate(); err != nil { + return nil, err + } + return fetchOptionsBuilder{opts: opts}, nil } // ParseLimit parses request limit from either header or query string. @@ -247,6 +259,11 @@ func (b fetchOptionsBuilder) newFetchOptions( fetchOpts.LookbackDuration = &lookback } + fetchOpts.Timeout, err = ParseRequestTimeout(req, b.opts.Timeout) + if err != nil { + return nil, fmt.Errorf("could not parse timeout: err=%v", err) + } + return fetchOpts, nil } @@ -345,3 +362,46 @@ func ParseDuration(r *http.Request, key string) (time.Duration, error) { return 0, fmt.Errorf("cannot parse duration='%s': as_duration_err=%s, as_float_err=%s", str, durationErr, floatErr) } + +// ParseRequestTimeout parses the input request timeout with a default. +func ParseRequestTimeout( + r *http.Request, + configFetchTimeout time.Duration, +) (time.Duration, error) { + var timeout string + if v := r.FormValue(TimeoutParam); v != "" { + timeout = v + } + // Note: Header should take precedence. + if v := r.Header.Get(TimeoutParam); v != "" { + timeout = v + } + + if timeout == "" { + return configFetchTimeout, nil + } + + duration, err := time.ParseDuration(timeout) + if err != nil { + return 0, xerrors.NewInvalidParamsError( + fmt.Errorf("invalid 'timeout': %v", err)) + } + + if err := validateTimeout(duration); err != nil { + return 0, err + } + + return duration, nil +} + +func validateTimeout(v time.Duration) error { + if v <= 0 { + return xerrors.NewInvalidParamsError( + fmt.Errorf("invalid 'timeout': less than or equal to zero %v", v)) + } + if v > maxTimeout { + return xerrors.NewInvalidParamsError( + fmt.Errorf("invalid 'timeout': %v greater than max %v", v, maxTimeout)) + } + return nil +} diff --git a/src/query/api/v1/handler/prometheus/handleroptions/fetch_options_test.go b/src/query/api/v1/handler/prometheus/handleroptions/fetch_options_test.go index 427c78ad1a..4ccbb1490d 100644 --- a/src/query/api/v1/handler/prometheus/handleroptions/fetch_options_test.go +++ b/src/query/api/v1/handler/prometheus/handleroptions/fetch_options_test.go @@ -21,8 +21,10 @@ package handleroptions import ( + "bytes" "fmt" "math" + "mime/multipart" "net/http" "net/http/httptest" "net/url" @@ -34,7 +36,9 @@ import ( "github.com/m3db/m3/src/query/models" "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/storage/m3/storagemetadata" + xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/headers" + xhttp "github.com/m3db/m3/src/x/net/http" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -214,12 +218,14 @@ func TestFetchOptionsBuilder(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - builder := NewFetchOptionsBuilder(FetchOptionsBuilderOptions{ + builder, err := NewFetchOptionsBuilder(FetchOptionsBuilderOptions{ Limits: FetchOptionsBuilderLimitsOptions{ SeriesLimit: test.defaultLimit, }, RestrictByTag: test.defaultRestrictByTag, + Timeout: 10 * time.Second, }) + require.NoError(t, err) url := "/foo" if test.query != "" { @@ -231,7 +237,6 @@ func TestFetchOptionsBuilder(t *testing.T) { } opts, err := builder.NewFetchOptions(req) - if !test.expectedErr { require.NoError(t, err) require.Equal(t, test.expectedLimit, opts.SeriesLimit) @@ -247,6 +252,7 @@ func TestFetchOptionsBuilder(t *testing.T) { require.NotNil(t, opts.LookbackDuration) require.Equal(t, test.expectedLookback.value, *opts.LookbackDuration) } + require.Equal(t, 10*time.Second, opts.Timeout) } else { require.Error(t, err) } @@ -360,11 +366,14 @@ func TestFetchOptionsWithHeader(t *testing.T) { }`, } - builder := NewFetchOptionsBuilder(FetchOptionsBuilderOptions{ + builder, err := NewFetchOptionsBuilder(FetchOptionsBuilderOptions{ Limits: FetchOptionsBuilderLimitsOptions{ SeriesLimit: 5, }, + Timeout: 10 * time.Second, }) + require.NoError(t, err) + req := httptest.NewRequest("GET", "/", nil) for k, v := range headers { req.Header.Add(k, v) @@ -397,3 +406,57 @@ func TestFetchOptionsWithHeader(t *testing.T) { func stripSpace(str string) string { return regexp.MustCompile(`\s+`).ReplaceAllString(str, "") } + +func TestParseRequestTimeout(t *testing.T) { + req := httptest.NewRequest("GET", "/read?timeout=2m", nil) + dur, err := ParseRequestTimeout(req, time.Second) + require.NoError(t, err) + assert.Equal(t, 2*time.Minute, dur) +} + +func TestTimeoutParseWithHeader(t *testing.T) { + req := httptest.NewRequest("POST", "/dummy", nil) + req.Header.Add("timeout", "1ms") + + timeout, err := ParseRequestTimeout(req, time.Second) + assert.NoError(t, err) + assert.Equal(t, timeout, time.Millisecond) + + req.Header.Del("timeout") + timeout, err = ParseRequestTimeout(req, 2*time.Minute) + assert.NoError(t, err) + assert.Equal(t, timeout, 2*time.Minute) + + req.Header.Add("timeout", "invalid") + _, err = ParseRequestTimeout(req, 15*time.Second) + assert.Error(t, err) + assert.True(t, xerrors.IsInvalidParams(err)) +} + +func TestTimeoutParseWithPostRequestParam(t *testing.T) { + params := url.Values{} + params.Add("timeout", "1ms") + + buff := bytes.NewBuffer(nil) + form := multipart.NewWriter(buff) + form.WriteField("timeout", "1ms") + require.NoError(t, form.Close()) + + req := httptest.NewRequest("POST", "/dummy", buff) + req.Header.Set(xhttp.HeaderContentType, form.FormDataContentType()) + + timeout, err := ParseRequestTimeout(req, time.Second) + assert.NoError(t, err) + assert.Equal(t, timeout, time.Millisecond) +} + +func TestTimeoutParseWithGetRequestParam(t *testing.T) { + params := url.Values{} + params.Add("timeout", "1ms") + + req := httptest.NewRequest("GET", "/dummy?"+params.Encode(), nil) + + timeout, err := ParseRequestTimeout(req, time.Second) + assert.NoError(t, err) + assert.Equal(t, timeout, time.Millisecond) +} diff --git a/src/query/api/v1/handler/prometheus/handleroptions/header_test.go b/src/query/api/v1/handler/prometheus/handleroptions/header_test.go index 89d90a5778..4961d47727 100644 --- a/src/query/api/v1/handler/prometheus/handleroptions/header_test.go +++ b/src/query/api/v1/handler/prometheus/handleroptions/header_test.go @@ -24,37 +24,45 @@ import ( "fmt" "net/http/httptest" "testing" + "time" "github.com/m3db/m3/src/query/block" + "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/x/headers" "github.com/stretchr/testify/assert" ) -func TestAddWarningHeaders(t *testing.T) { +func TestAddResponseHeaders(t *testing.T) { recorder := httptest.NewRecorder() meta := block.NewResultMetadata() - AddWarningHeaders(recorder, meta) + AddResponseHeaders(recorder, meta, nil) assert.Equal(t, 0, len(recorder.Header())) recorder = httptest.NewRecorder() meta.Exhaustive = false ex := headers.LimitHeaderSeriesLimitApplied - AddWarningHeaders(recorder, meta) + AddResponseHeaders(recorder, meta, nil) assert.Equal(t, 1, len(recorder.Header())) assert.Equal(t, ex, recorder.Header().Get(headers.LimitHeader)) recorder = httptest.NewRecorder() meta.AddWarning("foo", "bar") ex = fmt.Sprintf("%s,%s_%s", headers.LimitHeaderSeriesLimitApplied, "foo", "bar") - AddWarningHeaders(recorder, meta) + AddResponseHeaders(recorder, meta, nil) assert.Equal(t, 1, len(recorder.Header())) assert.Equal(t, ex, recorder.Header().Get(headers.LimitHeader)) recorder = httptest.NewRecorder() meta.Exhaustive = true ex = "foo_bar" - AddWarningHeaders(recorder, meta) + AddResponseHeaders(recorder, meta, nil) assert.Equal(t, 1, len(recorder.Header())) assert.Equal(t, ex, recorder.Header().Get(headers.LimitHeader)) + + recorder = httptest.NewRecorder() + meta = block.NewResultMetadata() + AddResponseHeaders(recorder, meta, &storage.FetchOptions{Timeout: 5 * time.Second}) + assert.Equal(t, 1, len(recorder.Header())) + assert.Equal(t, "5s", recorder.Header().Get(headers.TimeoutHeader)) } diff --git a/src/query/api/v1/handler/prometheus/handleroptions/headers.go b/src/query/api/v1/handler/prometheus/handleroptions/headers.go index 0694db73ab..96de80ea81 100644 --- a/src/query/api/v1/handler/prometheus/handleroptions/headers.go +++ b/src/query/api/v1/handler/prometheus/handleroptions/headers.go @@ -25,12 +25,22 @@ import ( "strings" "github.com/m3db/m3/src/query/block" + "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/x/headers" ) -// AddWarningHeaders adds any warning headers present in the result's metadata. -// No-op if no warnings encountered. -func AddWarningHeaders(w http.ResponseWriter, meta block.ResultMetadata) { +// AddResponseHeaders adds any warning headers present in the result's metadata, +// and also effective parameters relative to the request such as effective +// timeout in use. +func AddResponseHeaders( + w http.ResponseWriter, + meta block.ResultMetadata, + fetchOpts *storage.FetchOptions, +) { + if fetchOpts != nil { + w.Header().Set(headers.TimeoutHeader, fetchOpts.Timeout.String()) + } + ex := meta.Exhaustive warns := len(meta.Warnings) if !ex { diff --git a/src/query/api/v1/handler/prometheus/native/common.go b/src/query/api/v1/handler/prometheus/native/common.go index 59ec8630e0..ce3e7db828 100644 --- a/src/query/api/v1/handler/prometheus/native/common.go +++ b/src/query/api/v1/handler/prometheus/native/common.go @@ -28,7 +28,6 @@ import ( "strconv" "time" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/errors" @@ -69,7 +68,6 @@ func parseTime(r *http.Request, key string, now time.Time) (time.Time, error) { func parseParams( r *http.Request, engineOpts executor.EngineOptions, - timeoutOpts *prometheus.TimeoutOpts, fetchOpts *storage.FetchOptions, ) (models.RequestParams, error) { var params models.RequestParams @@ -89,12 +87,6 @@ func parseParams( } } - t, err := prometheus.ParseRequestTimeout(r, timeoutOpts.FetchTimeout) - if err != nil { - return params, xerrors.NewInvalidParamsError(err) - } - params.Timeout = t - start, err := parseTime(r, startParam, params.Now) if err != nil { err = fmt.Errorf(formatErrStr, startParam, err) @@ -113,6 +105,14 @@ func parseParams( } params.End = end + timeout := fetchOpts.Timeout + if timeout <= 0 { + err := fmt.Errorf("expected positive timeout, instead got: %d", timeout) + return params, xerrors.NewInvalidParamsError( + fmt.Errorf(formatErrStr, handleroptions.TimeoutParam, err)) + } + params.Timeout = timeout + step := fetchOpts.Step if step <= 0 { err := fmt.Errorf("expected positive step size, instead got: %d", step) @@ -180,7 +180,6 @@ func parseParams( func parseInstantaneousParams( r *http.Request, engineOpts executor.EngineOptions, - timeoutOpts *prometheus.TimeoutOpts, fetchOpts *storage.FetchOptions, ) (models.RequestParams, error) { if err := r.ParseForm(); err != nil { @@ -193,7 +192,7 @@ func parseInstantaneousParams( r.Form.Set(startParam, nowTimeValue) r.Form.Set(endParam, nowTimeValue) - params, err := parseParams(r, engineOpts, timeoutOpts, fetchOpts) + params, err := parseParams(r, engineOpts, fetchOpts) if err != nil { return params, err } diff --git a/src/query/api/v1/handler/prometheus/native/common_test.go b/src/query/api/v1/handler/prometheus/native/common_test.go index 8b1a1888fc..3d675559c4 100644 --- a/src/query/api/v1/handler/prometheus/native/common_test.go +++ b/src/query/api/v1/handler/prometheus/native/common_test.go @@ -30,12 +30,10 @@ import ( "testing" "time" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/block" "github.com/m3db/m3/src/query/executor" "github.com/m3db/m3/src/query/models" - "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/test" "github.com/m3db/m3/src/query/ts" xerrors "github.com/m3db/m3/src/x/errors" @@ -52,12 +50,6 @@ const ( promQuery = `http_requests_total{job="prometheus",group="canary"}` ) -var ( - timeoutOpts = &prometheus.TimeoutOpts{ - FetchTimeout: 15 * time.Second, - } -) - func defaultParams() url.Values { vals := url.Values{} now := time.Now() @@ -69,14 +61,20 @@ func defaultParams() url.Values { } func testParseParams(req *http.Request) (models.RequestParams, error) { - fetchOpts, err := handleroptions. - NewFetchOptionsBuilder(handleroptions.FetchOptionsBuilderOptions{}). - NewFetchOptions(req) + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) if err != nil { return models.RequestParams{}, err } - return parseParams(req, executor.NewEngineOptions(), timeoutOpts, fetchOpts) + fetchOpts, err := fetchOptsBuilder.NewFetchOptions(req) + if err != nil { + return models.RequestParams{}, err + } + + return parseParams(req, executor.NewEngineOptions(), fetchOpts) } func TestParamParsing(t *testing.T) { @@ -105,10 +103,17 @@ func TestInstantaneousParamParsing(t *testing.T) { params.Add(queryParam, promQuery) params.Add(timeParam, now.Format(time.RFC3339)) req.URL.RawQuery = params.Encode() + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 10 * time.Second, + }) + require.NoError(t, err) + fetchOpts, err := fetchOptsBuilder.NewFetchOptions(req) + require.NoError(t, err) r, err := parseInstantaneousParams(req, executor.NewEngineOptions(), - timeoutOpts, storage.NewFetchOptions()) - require.Nil(t, err, "unable to parse request") + fetchOpts) + require.NoError(t, err, "unable to parse request") require.Equal(t, promQuery, r.Query) } diff --git a/src/query/api/v1/handler/prometheus/native/complete_tags.go b/src/query/api/v1/handler/prometheus/native/complete_tags.go index 85815900f6..afc4289985 100644 --- a/src/query/api/v1/handler/prometheus/native/complete_tags.go +++ b/src/query/api/v1/handler/prometheus/native/complete_tags.go @@ -125,7 +125,7 @@ func (h *CompleteTagsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) return } - handleroptions.AddWarningHeaders(w, meta) + handleroptions.AddResponseHeaders(w, meta, opts) result := resultBuilder.Build() if err := prometheus.RenderTagCompletionResultsJSON(w, result); err != nil { logger.Error("unable to render results", zap.Error(err)) diff --git a/src/query/api/v1/handler/prometheus/native/complete_tags_test.go b/src/query/api/v1/handler/prometheus/native/complete_tags_test.go index 843bd86bff..c5ba68c5b5 100644 --- a/src/query/api/v1/handler/prometheus/native/complete_tags_test.go +++ b/src/query/api/v1/handler/prometheus/native/complete_tags_test.go @@ -25,6 +25,7 @@ import ( "io/ioutil" "net/http/httptest" "testing" + "time" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/options" @@ -88,8 +89,9 @@ func testCompleteTags(t *testing.T, meta block.ResultMetadata, header string) { Metadata: meta, } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{Timeout: 15 * time.Second}) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetFetchOptionsBuilder(fb) @@ -169,8 +171,9 @@ func TestMultiCompleteTags(t *testing.T) { Metadata: barMeta, } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{Timeout: 15 * time.Second}) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetFetchOptionsBuilder(fb) diff --git a/src/query/api/v1/handler/prometheus/native/list_tags.go b/src/query/api/v1/handler/prometheus/native/list_tags.go index 8f100d4a06..7841a0dca7 100644 --- a/src/query/api/v1/handler/prometheus/native/list_tags.go +++ b/src/query/api/v1/handler/prometheus/native/list_tags.go @@ -94,7 +94,7 @@ func (h *ListTagsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - handleroptions.AddWarningHeaders(w, result.Metadata) + handleroptions.AddResponseHeaders(w, result.Metadata, opts) if err = prometheus.RenderListTagResultsJSON(w, result); err != nil { logger.Error("unable to render results", zap.Error(err)) xhttp.WriteError(w, err) diff --git a/src/query/api/v1/handler/prometheus/native/list_tags_test.go b/src/query/api/v1/handler/prometheus/native/list_tags_test.go index 8ef8a9c32f..0ecc6dfa04 100644 --- a/src/query/api/v1/handler/prometheus/native/list_tags_test.go +++ b/src/query/api/v1/handler/prometheus/native/list_tags_test.go @@ -111,8 +111,9 @@ func testListTags(t *testing.T, meta block.ResultMetadata, header string) { return now } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{Timeout: 15 * time.Second}) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetFetchOptionsBuilder(fb). @@ -155,8 +156,9 @@ func TestListErrorTags(t *testing.T) { return now } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{Timeout: 15 * time.Second}) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetFetchOptionsBuilder(fb). diff --git a/src/query/api/v1/handler/prometheus/native/read.go b/src/query/api/v1/handler/prometheus/native/read.go index be153d0d4c..255f6dd69c 100644 --- a/src/query/api/v1/handler/prometheus/native/read.go +++ b/src/query/api/v1/handler/prometheus/native/read.go @@ -150,7 +150,7 @@ func (h *promReadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } w.Header().Set(xhttp.HeaderContentType, xhttp.ContentTypeJSON) - handleroptions.AddWarningHeaders(w, result.Meta) + handleroptions.AddResponseHeaders(w, result.Meta, parsedOptions.FetchOpts) h.promReadMetrics.fetchSuccess.Inc(1) keepNaNs := h.opts.Config().ResultOptions.KeepNaNs diff --git a/src/query/api/v1/handler/prometheus/native/read_common.go b/src/query/api/v1/handler/prometheus/native/read_common.go index b2c1760125..5420693b7b 100644 --- a/src/query/api/v1/handler/prometheus/native/read_common.go +++ b/src/query/api/v1/handler/prometheus/native/read_common.go @@ -102,7 +102,8 @@ func parseRequest( LimitMaxTimeseries: fetchOpts.SeriesLimit, LimitMaxDocs: fetchOpts.DocsLimit, Instantaneous: instantaneous, - }} + }, + } restrictOpts := fetchOpts.RestrictQueryOptions.GetRestrictByType() if restrictOpts != nil { @@ -119,11 +120,9 @@ func parseRequest( params models.RequestParams ) if instantaneous { - params, err = parseInstantaneousParams(r, engine.Options(), - opts.TimeoutOpts(), fetchOpts) + params, err = parseInstantaneousParams(r, engine.Options(), fetchOpts) } else { - params, err = parseParams(r, engine.Options(), - opts.TimeoutOpts(), fetchOpts) + params, err = parseParams(r, engine.Options(), fetchOpts) } if err != nil { return ParsedOptions{}, err diff --git a/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go b/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go index a1352be788..a4c65f0e51 100644 --- a/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go +++ b/src/query/api/v1/handler/prometheus/native/read_instantaneous_test.go @@ -95,7 +95,7 @@ func testPromReadInstantHandler( ) { values, bounds := test.GenerateValuesAndBounds(nil, nil) - setup := newTestSetup(timeoutOpts, nil) + setup := newTestSetup(t, nil) promReadInstant := setup.Handlers.instantRead seriesMeta := test.NewSeriesMeta("dummy", len(values)) @@ -172,7 +172,7 @@ func testPromReadInstantHandler( } func TestPromReadInstantHandlerStorageError(t *testing.T) { - setup := newTestSetup(timeoutOpts, nil) + setup := newTestSetup(t, nil) promReadInstant := setup.Handlers.instantRead storageErr := fmt.Errorf("storage err") diff --git a/src/query/api/v1/handler/prometheus/native/read_test.go b/src/query/api/v1/handler/prometheus/native/read_test.go index c1b3bffca9..7808dd914c 100644 --- a/src/query/api/v1/handler/prometheus/native/read_test.go +++ b/src/query/api/v1/handler/prometheus/native/read_test.go @@ -30,7 +30,6 @@ import ( "github.com/golang/mock/gomock" "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/query/api/v1/handler" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/options" "github.com/m3db/m3/src/query/block" @@ -49,16 +48,14 @@ import ( ) func TestParseRequest(t *testing.T) { - setup := newTestSetup(&prometheus.TimeoutOpts{ - FetchTimeout: 10 * time.Second, - }, nil) + setup := newTestSetup(t, nil) req, _ := http.NewRequest("GET", PromReadURL, nil) req.URL.RawQuery = defaultParams().Encode() parsed, err := ParseRequest(req.Context(), req, false, setup.options) require.NoError(t, err) - require.Equal(t, time.Second*10, parsed.Params.Timeout) - require.Equal(t, time.Second*0, parsed.FetchOpts.Timeout) + require.Equal(t, 15*time.Second, parsed.Params.Timeout) + require.Equal(t, 15*time.Second, parsed.FetchOpts.Timeout) require.Equal(t, 0, parsed.FetchOpts.DocsLimit) require.Equal(t, 0, parsed.FetchOpts.SeriesLimit) require.Equal(t, false, parsed.FetchOpts.RequireExhaustive) @@ -95,7 +92,7 @@ func TestPromReadHandlerWithTimeout(t *testing.T) { return nil, nil }) - setup := newTestSetup(nil, engine) + setup := newTestSetup(t, engine) promRead := setup.Handlers.read req, _ := http.NewRequest("GET", PromReadURL, nil) @@ -129,7 +126,7 @@ func testPromReadHandlerRead( ) { values, bounds := test.GenerateValuesAndBounds(nil, nil) - setup := newTestSetup(timeoutOpts, nil) + setup := newTestSetup(t, nil) promRead := setup.Handlers.read seriesMeta := test.NewSeriesMeta("dummy", len(values)) @@ -176,12 +173,11 @@ func newReadRequest(t *testing.T, params url.Values) *http.Request { } type testSetup struct { - Storage mock.Storage - Handlers testSetupHandlers - QueryOpts *executor.QueryOptions - FetchOpts *storage.FetchOptions - TimeoutOpts *prometheus.TimeoutOpts - options options.HandlerOptions + Storage mock.Storage + Handlers testSetupHandlers + QueryOpts *executor.QueryOptions + FetchOpts *storage.FetchOptions + options options.HandlerOptions } type testSetupHandlers struct { @@ -190,7 +186,7 @@ type testSetupHandlers struct { } func newTestSetup( - timeout *prometheus.TimeoutOpts, + t *testing.T, mockEngine *executor.MockEngine, ) *testSetup { mockStorage := mock.NewMockStorage() @@ -204,8 +200,11 @@ func newTestSetup( if mockEngine != nil { engine = mockEngine } - fetchOptsBuilderCfg := handleroptions.FetchOptionsBuilderOptions{} - fetchOptsBuilder := handleroptions.NewFetchOptionsBuilder(fetchOptsBuilderCfg) + fetchOptsBuilderCfg := handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + } + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder(fetchOptsBuilderCfg) + require.NoError(t, err) tagOpts := models.NewTagOptions() limitsConfig := config.LimitsConfiguration{} keepNaNs := false @@ -214,7 +213,6 @@ func newTestSetup( SetEngine(engine). SetFetchOptionsBuilder(fetchOptsBuilder). SetTagOptions(tagOpts). - SetTimeoutOpts(timeout). SetInstrumentOpts(instrumentOpts). SetConfig(config.Configuration{ Limits: limitsConfig, @@ -232,10 +230,9 @@ func newTestSetup( read: read, instantRead: instantRead, }, - QueryOpts: &executor.QueryOptions{}, - FetchOpts: storage.NewFetchOptions(), - TimeoutOpts: timeoutOpts, - options: opts, + QueryOpts: &executor.QueryOptions{}, + FetchOpts: storage.NewFetchOptions(), + options: opts, } } diff --git a/src/query/api/v1/handler/prometheus/remote/match.go b/src/query/api/v1/handler/prometheus/remote/match.go index 7244b0e552..7359bc743e 100644 --- a/src/query/api/v1/handler/prometheus/remote/match.go +++ b/src/query/api/v1/handler/prometheus/remote/match.go @@ -103,7 +103,7 @@ func (h *PromSeriesMatchHandler) ServeHTTP(w http.ResponseWriter, r *http.Reques meta = meta.CombineMetadata(result.Metadata) } - handleroptions.AddWarningHeaders(w, meta) + handleroptions.AddResponseHeaders(w, meta, opts) // TODO: Support multiple result types if err := prometheus.RenderSeriesMatchResultsJSON(w, results, false); err != nil { logger.Error("unable to write matched series", zap.Error(err)) diff --git a/src/query/api/v1/handler/prometheus/remote/read.go b/src/query/api/v1/handler/prometheus/remote/read.go index 8c7f050beb..83f32b8f27 100644 --- a/src/query/api/v1/handler/prometheus/remote/read.go +++ b/src/query/api/v1/handler/prometheus/remote/read.go @@ -130,6 +130,9 @@ func (h *promReadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } + // Write headers before response. + handleroptions.AddResponseHeaders(w, readResult.Meta, fetchOpts) + // NB: if this errors, all relevant headers and information should already // be sent to the writer; so it is not necessary to do anything here other // than increment success/failure metrics. @@ -174,8 +177,6 @@ func (h *promReadHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { } w.Header().Set(xhttp.HeaderContentType, xhttp.ContentTypeJSON) - handleroptions.AddWarningHeaders(w, readResult.Meta) - err = json.NewEncoder(w).Encode(result) default: err = WriteSnappyCompressed(w, readResult, logger) @@ -228,7 +229,6 @@ func WriteSnappyCompressed( w.Header().Set(xhttp.HeaderContentType, xhttp.ContentTypeProtobuf) w.Header().Set("Content-Encoding", "snappy") - handleroptions.AddWarningHeaders(w, readResult.Meta) compressed := snappy.Encode(nil, data) if _, err := w.Write(compressed); err != nil { @@ -404,18 +404,11 @@ func parseRequest( return nil, nil, err } - timeout := opts.TimeoutOpts().FetchTimeout - timeout, err = prometheus.ParseRequestTimeout(r, timeout) - if err != nil { - return nil, nil, err - } - fetchOpts, rErr := opts.FetchOptionsBuilder().NewFetchOptions(r) if rErr != nil { return nil, nil, rErr } - fetchOpts.Timeout = timeout return req, fetchOpts, nil } diff --git a/src/query/api/v1/handler/prometheus/remote/read_test.go b/src/query/api/v1/handler/prometheus/remote/read_test.go index 9ded0bdc53..7d723518cb 100644 --- a/src/query/api/v1/handler/prometheus/remote/read_test.go +++ b/src/query/api/v1/handler/prometheus/remote/read_test.go @@ -37,7 +37,6 @@ import ( "github.com/m3db/m3/src/dbnode/client" xmetrics "github.com/m3db/m3/src/dbnode/x/metrics" "github.com/m3db/m3/src/query/api/v1/handler" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/options" "github.com/m3db/m3/src/query/block" @@ -62,10 +61,6 @@ import ( var ( promReadTestMetrics = newPromReadMetrics(tally.NewTestScope("", nil)) defaultLookbackDuration = time.Minute - - timeoutOpts = &prometheus.TimeoutOpts{ - FetchTimeout: 15 * time.Second, - } ) type testVals struct { @@ -151,25 +146,26 @@ func setupServer(t *testing.T) *httptest.Server { Return(nil, client.FetchResponseMetadata{Exhaustive: false}, fmt.Errorf("not initialized")).MaxTimes(1) storage := test.NewSlowStorage(lstore, 10*time.Millisecond) - promRead := readHandler(storage, timeoutOpts) + promRead := readHandler(t, storage) server := httptest.NewServer(test.NewSlowHandler(promRead, 10*time.Millisecond)) return server } -func readHandler(store storage.Storage, - timeoutOpts *prometheus.TimeoutOpts) http.Handler { +func readHandler(t *testing.T, store storage.Storage) http.Handler { fetchOpts := handleroptions.FetchOptionsBuilderOptions{ Limits: handleroptions.FetchOptionsBuilderLimitsOptions{ SeriesLimit: 100, }, + Timeout: 15 * time.Second, } + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder(fetchOpts) + require.NoError(t, err) iOpts := instrument.NewOptions() engine := newEngine(store, defaultLookbackDuration, iOpts) opts := options.EmptyHandlerOptions(). SetEngine(engine). SetInstrumentOpts(iOpts). - SetFetchOptionsBuilder(handleroptions.NewFetchOptionsBuilder(fetchOpts)). - SetTimeoutOpts(timeoutOpts) + SetFetchOptionsBuilder(fetchOptsBuilder) return NewPromReadHandler(opts) } @@ -181,14 +177,16 @@ func TestPromReadParsing(t *testing.T) { Limits: handleroptions.FetchOptionsBuilderLimitsOptions{ SeriesLimit: 100, }, + Timeout: 15 * time.Second, } + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder(builderOpts) + require.NoError(t, err) engine := newEngine(storage, defaultLookbackDuration, instrument.NewOptions()) opts := options.EmptyHandlerOptions(). SetEngine(engine). - SetFetchOptionsBuilder(handleroptions.NewFetchOptionsBuilder(builderOpts)). - SetTimeoutOpts(timeoutOpts) + SetFetchOptionsBuilder(fetchOptsBuilder) req := httptest.NewRequest("POST", PromReadURL, test.GeneratePromReadBody(t)) r, fetchOpts, err := ParseRequest(context.TODO(), req, opts) @@ -197,14 +195,6 @@ func TestPromReadParsing(t *testing.T) { fmt.Println(fetchOpts) } -func TestPromFetchTimeoutParsing(t *testing.T) { - url := fmt.Sprintf("%s?timeout=2m", PromReadURL) - req := httptest.NewRequest("POST", url, test.GeneratePromReadBody(t)) - dur, err := prometheus.ParseRequestTimeout(req, time.Second) - require.NoError(t, err) - assert.Equal(t, 2*time.Minute, dur) -} - func TestPromReadParsingBad(t *testing.T) { req := httptest.NewRequest("POST", PromReadURL, strings.NewReader("bad body")) _, _, err := ParseRequest(context.TODO(), req, options.EmptyHandlerOptions()) @@ -293,13 +283,15 @@ func TestReadErrorMetricsCount(t *testing.T) { Limits: handleroptions.FetchOptionsBuilderLimitsOptions{ SeriesLimit: 100, }, + Timeout: 15 * time.Second, } + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder(buildOpts) + require.NoError(t, err) engine := newEngine(storage, defaultLookbackDuration, instrument.NewOptions()) opts := options.EmptyHandlerOptions(). SetEngine(engine). - SetTimeoutOpts(&prometheus.TimeoutOpts{FetchTimeout: time.Minute}). - SetFetchOptionsBuilder(handleroptions.NewFetchOptionsBuilder(buildOpts)) + SetFetchOptionsBuilder(fetchOptsBuilder) promRead := &promReadHandler{ promReadMetrics: readMetrics, opts: opts, diff --git a/src/query/api/v1/handler/prometheus/remote/tag_values.go b/src/query/api/v1/handler/prometheus/remote/tag_values.go index a8aa873a35..f146ffb649 100644 --- a/src/query/api/v1/handler/prometheus/remote/tag_values.go +++ b/src/query/api/v1/handler/prometheus/remote/tag_values.go @@ -102,7 +102,7 @@ func (h *TagValuesHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { return } - handleroptions.AddWarningHeaders(w, result.Metadata) + handleroptions.AddResponseHeaders(w, result.Metadata, opts) // TODO: Support multiple result types err = prometheus.RenderTagValuesResultsJSON(w, result) if err != nil { diff --git a/src/query/api/v1/handler/prometheus/remote/tag_values_test.go b/src/query/api/v1/handler/prometheus/remote/tag_values_test.go index 23309b5f89..89a799f39c 100644 --- a/src/query/api/v1/handler/prometheus/remote/tag_values_test.go +++ b/src/query/api/v1/handler/prometheus/remote/tag_values_test.go @@ -103,8 +103,11 @@ func TestTagValues(t *testing.T) { return now } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetNowFn(nowFn). @@ -176,8 +179,11 @@ func TestTagValueErrors(t *testing.T) { return now } - fb := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + fb, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(store). SetNowFn(nowFn). diff --git a/src/query/api/v1/handler/search_test.go b/src/query/api/v1/handler/search_test.go index f6d38cb080..87201665f9 100644 --- a/src/query/api/v1/handler/search_test.go +++ b/src/query/api/v1/handler/search_test.go @@ -101,8 +101,11 @@ func searchServer(t *testing.T) *SearchHandler { session.EXPECT().FetchTaggedIDs(gomock.Any(), gomock.Any(), gomock.Any()). Return(mockTaggedIDsIter, client.FetchResponseMetadata{Exhaustive: false}, nil).AnyTimes() - builder := handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{}) + builder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) opts := options.EmptyHandlerOptions(). SetStorage(storage).SetFetchOptionsBuilder(builder) search := NewSearchHandler(opts) diff --git a/src/query/api/v1/handler/topic/common.go b/src/query/api/v1/handler/topic/common.go index dc8f461e36..9e2b369149 100644 --- a/src/query/api/v1/handler/topic/common.go +++ b/src/query/api/v1/handler/topic/common.go @@ -28,8 +28,8 @@ import ( "github.com/m3db/m3/src/cluster/kv" "github.com/m3db/m3/src/cmd/services/m3query/config" "github.com/m3db/m3/src/msg/topic" - "github.com/m3db/m3/src/query/api/v1/handler" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" + "github.com/m3db/m3/src/query/util/queryhttp" xerrors "github.com/m3db/m3/src/x/errors" "github.com/m3db/m3/src/x/instrument" @@ -70,32 +70,46 @@ func Service(clusterClient clusterclient.Client, opts handleroptions.ServiceOpti // RegisterRoutes registers the topic routes func RegisterRoutes( - addRoute handler.AddRouteFn, + r *queryhttp.EndpointRegistry, client clusterclient.Client, cfg config.Configuration, instrumentOpts instrument.Options, ) error { - if err := addRoute(InitURL, newInitHandler(client, cfg, instrumentOpts), - InitHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: InitURL, + Handler: newInitHandler(client, cfg, instrumentOpts), + Methods: []string{InitHTTPMethod}, + }); err != nil { return err } - if err := addRoute(GetURL, newGetHandler(client, cfg, instrumentOpts), - GetHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: GetURL, + Handler: newGetHandler(client, cfg, instrumentOpts), + Methods: []string{GetHTTPMethod}, + }); err != nil { return err } - if err := addRoute(AddURL, newAddHandler(client, cfg, instrumentOpts), - AddHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: AddURL, + Handler: newAddHandler(client, cfg, instrumentOpts), + Methods: []string{AddHTTPMethod}, + }); err != nil { return err } - if err := addRoute(UpdateURL, newUpdateHandler(client, cfg, instrumentOpts), - UpdateHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: UpdateURL, + Handler: newUpdateHandler(client, cfg, instrumentOpts), + Methods: []string{UpdateHTTPMethod}, + }); err != nil { return err } - if err := addRoute(DeleteURL, newDeleteHandler(client, cfg, instrumentOpts), - DeleteHTTPMethod); err != nil { + if err := r.Register(queryhttp.RegisterOptions{ + Path: DeleteURL, + Handler: newDeleteHandler(client, cfg, instrumentOpts), + Methods: []string{DeleteHTTPMethod}, + }); err != nil { return err } - return nil } diff --git a/src/query/api/v1/handler/types.go b/src/query/api/v1/handler/types.go index 5b0f523987..54a6a605ae 100644 --- a/src/query/api/v1/handler/types.go +++ b/src/query/api/v1/handler/types.go @@ -20,14 +20,9 @@ package handler -import "net/http" - // HeaderKeyType is the type for the header key. type HeaderKeyType int -// AddRouteFn is the function type for adding new HTTP route. -type AddRouteFn func(path string, handler http.Handler, methods ...string) error - const ( // HeaderKey is the key which headers will be added to in the request context. HeaderKey HeaderKeyType = iota diff --git a/src/query/api/v1/httpd/handler.go b/src/query/api/v1/httpd/handler.go index f434464fcb..83b34ec6bf 100644 --- a/src/query/api/v1/httpd/handler.go +++ b/src/query/api/v1/httpd/handler.go @@ -43,6 +43,7 @@ import ( "github.com/m3db/m3/src/query/api/v1/handler/topic" "github.com/m3db/m3/src/query/api/v1/options" "github.com/m3db/m3/src/query/util/logging" + "github.com/m3db/m3/src/query/util/queryhttp" xdebug "github.com/m3db/m3/src/x/debug" "github.com/m3db/m3/src/x/headers" xhttp "github.com/m3db/m3/src/x/net/http" @@ -76,7 +77,7 @@ var ( // Handler represents the top-level HTTP handler. type Handler struct { - router *mux.Router + registry *queryhttp.EndpointRegistry handler http.Handler options options.HandlerOptions customHandlers []options.CustomHandler @@ -97,8 +98,10 @@ func NewHandler( handlerWithMiddleware := applyMiddleware(r, opentracing.GlobalTracer()) logger := handlerOptions.InstrumentOpts().Logger() + instrumentOpts := handlerOptions.InstrumentOpts().SetMetricsScope( + handlerOptions.InstrumentOpts().MetricsScope().SubScope("http_handler")) return &Handler{ - router: r, + registry: queryhttp.NewEndpointRegistry(r, instrumentOpts), handler: handlerWithMiddleware, options: handlerOptions, customHandlers: customHandlers, @@ -131,36 +134,22 @@ func applyMiddleware(base *mux.Router, tracer opentracing.Tracer) http.Handler { // RegisterRoutes registers all http routes. func (h *Handler) RegisterRoutes() error { - var ( - instrumentOpts = h.options.InstrumentOpts() - - // Wrap requests with response time logging as well as panic recovery. - wrapped = func(n http.Handler) http.Handler { - return logging.WithResponseTimeAndPanicErrorLogging(n, instrumentOpts) - } - - panicOnly = func(n http.Handler) http.Handler { - return logging.WithPanicErrorResponder(n, instrumentOpts) - } - - wrappedRouteFn = func(path string, handler http.Handler, methods ...string) error { - return h.addRouteHandlerFn(h.router, path, wrapped(handler).ServeHTTP, methods...) - } - - routeFn = func(path string, handler http.Handler, methods ...string) error { - return h.addRouteHandlerFn(h.router, path, handler.ServeHTTP, methods...) - } - ) - - if err := wrappedRouteFn(openapi.URL, openapi.NewDocHandler(instrumentOpts), - openapi.HTTPMethod); err != nil { + instrumentOpts := h.options.InstrumentOpts() + + // OpenAPI. + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: openapi.URL, + Handler: openapi.NewDocHandler(instrumentOpts), + Methods: methods(openapi.HTTPMethod), + }); err != nil { + return err + } + if err := h.registry.Register(queryhttp.RegisterOptions{ + PathPrefix: openapi.StaticURLPrefix, + Handler: openapi.StaticHandler(), + }); err != nil { return err } - - h.router.PathPrefix(openapi.StaticURLPrefix). - Handler(wrapped(openapi.StaticHandler())). - Name(openapi.StaticURLPrefix) - // Prometheus remote read/write endpoints. remoteSourceOpts := h.options.SetInstrumentOpts(instrumentOpts. @@ -206,99 +195,160 @@ func (h *Handler) RegisterRoutes() error { M3QueryHandler: nativePromReadInstantHandler.ServeHTTP, }) - if err := wrappedRouteFn(native.PromReadURL, h.options.QueryRouter(), - native.PromReadHTTPMethods...); err != nil { + // Query routable endpoints. + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.PromReadURL, + Handler: h.options.QueryRouter(), + Methods: native.PromReadHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(native.PromReadInstantURL, h.options.InstantQueryRouter(), - native.PromReadInstantHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.PromReadInstantURL, + Handler: h.options.InstantQueryRouter(), + Methods: native.PromReadInstantHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(native.PrometheusReadURL, promqlQueryHandler, - native.PromReadHTTPMethods...); err != nil { + // Prometheus endpoints. + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: "/prometheus" + native.PromReadURL, + Handler: promqlQueryHandler, + Methods: native.PromReadHTTPMethods, + }); err != nil { return err } - - if err := wrappedRouteFn(native.PrometheusReadInstantURL, promqlInstantQueryHandler, - native.PromReadInstantHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: "/prometheus" + native.PromReadInstantURL, + Handler: promqlInstantQueryHandler, + Methods: native.PromReadInstantHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(remote.PromReadURL, promRemoteReadHandler, - remote.PromReadHTTPMethods...); err != nil { + // M3Query endpoints. + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: "/m3query" + native.PromReadURL, + Handler: nativePromReadHandler, + Methods: native.PromReadHTTPMethods, + }); err != nil { return err } - if err := routeFn(remote.PromWriteURL, panicOnly(promRemoteWriteHandler), - remote.PromWriteHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: "/m3query" + native.PromReadInstantURL, + Handler: nativePromReadInstantHandler, + Methods: native.PromReadInstantHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(native.M3QueryReadURL, nativePromReadHandler, - native.PromReadHTTPMethods...); err != nil { + // Prometheus remote read and write endpoints. + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: remote.PromReadURL, + Handler: promRemoteReadHandler, + Methods: remote.PromReadHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(native.M3QueryReadInstantURL, nativePromReadInstantHandler, - native.PromReadInstantHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: remote.PromWriteURL, + Handler: promRemoteWriteHandler, + Methods: methods(remote.PromWriteHTTPMethod), + // Register with no response logging for write calls since so frequent. + }, logging.WithNoResponseLog()); err != nil { return err } // InfluxDB write endpoint. - if err := wrappedRouteFn(influxdb.InfluxWriteURL, influxdb.NewInfluxWriterHandler(h.options), - influxdb.InfluxWriteHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: influxdb.InfluxWriteURL, + Handler: influxdb.NewInfluxWriterHandler(h.options), + Methods: methods(influxdb.InfluxWriteHTTPMethod), + // Register with no response logging for write calls since so frequent. + }, logging.WithNoResponseLog()); err != nil { return err } // Native M3 search and write endpoints. - if err := wrappedRouteFn(handler.SearchURL, handler.NewSearchHandler(h.options), - handler.SearchHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: handler.SearchURL, + Handler: handler.NewSearchHandler(h.options), + Methods: methods(handler.SearchHTTPMethod), + }); err != nil { return err } - if err := wrappedRouteFn(m3json.WriteJSONURL, m3json.NewWriteJSONHandler(h.options), - m3json.JSONWriteHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: m3json.WriteJSONURL, + Handler: m3json.NewWriteJSONHandler(h.options), + Methods: methods(m3json.JSONWriteHTTPMethod), + }); err != nil { return err } // Tag completion endpoints. - if err := wrappedRouteFn(native.CompleteTagsURL, native.NewCompleteTagsHandler(h.options), - native.CompleteTagsHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.CompleteTagsURL, + Handler: native.NewCompleteTagsHandler(h.options), + Methods: methods(native.CompleteTagsHTTPMethod), + }); err != nil { return err } - if err := wrappedRouteFn(remote.TagValuesURL, remote.NewTagValuesHandler(h.options), - remote.TagValuesHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: remote.TagValuesURL, + Handler: remote.NewTagValuesHandler(h.options), + Methods: methods(remote.TagValuesHTTPMethod), + }); err != nil { return err } // List tag endpoints. - if err := wrappedRouteFn(native.ListTagsURL, native.NewListTagsHandler(h.options), - native.ListTagsHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.ListTagsURL, + Handler: native.NewListTagsHandler(h.options), + Methods: native.ListTagsHTTPMethods, + }); err != nil { return err } // Query parse endpoints. - if err := wrappedRouteFn(native.PromParseURL, native.NewPromParseHandler(h.options), - native.PromParseHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.PromParseURL, + Handler: native.NewPromParseHandler(h.options), + Methods: methods(native.PromParseHTTPMethod), + }); err != nil { return err } - if err := wrappedRouteFn(native.PromThresholdURL, native.NewPromThresholdHandler(h.options), - native.PromThresholdHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: native.PromThresholdURL, + Handler: native.NewPromThresholdHandler(h.options), + Methods: methods(native.PromThresholdHTTPMethod), + }); err != nil { return err } // Series match endpoints. - if err := wrappedRouteFn(remote.PromSeriesMatchURL, - remote.NewPromSeriesMatchHandler(h.options), - remote.PromSeriesMatchHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: remote.PromSeriesMatchURL, + Handler: remote.NewPromSeriesMatchHandler(h.options), + Methods: remote.PromSeriesMatchHTTPMethods, + }); err != nil { return err } // Graphite endpoints. - if err := wrappedRouteFn(graphite.ReadURL, graphite.NewRenderHandler(h.options), - graphite.ReadHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: graphite.ReadURL, + Handler: graphite.NewRenderHandler(h.options), + Methods: graphite.ReadHTTPMethods, + }); err != nil { return err } - if err := wrappedRouteFn(graphite.FindURL, graphite.NewFindHandler(h.options), - graphite.FindHTTPMethods...); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: graphite.FindURL, + Handler: graphite.NewFindHandler(h.options), + Methods: graphite.FindHTTPMethods, + }); err != nil { return err } @@ -334,26 +384,36 @@ func (h *Handler) RegisterRoutes() error { } // Register debug dump handler. - if err := wrappedRouteFn(xdebug.DebugURL, debugWriter.HTTPHandler()); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: xdebug.DebugURL, + Handler: debugWriter.HTTPHandler(), + Methods: methods(xdebug.DebugMethod), + }); err != nil { return err } if clusterClient != nil { - if err := database.RegisterRoutes(wrappedRouteFn, clusterClient, + err = database.RegisterRoutes(h.registry, clusterClient, h.options.Config(), h.options.EmbeddedDbCfg(), - serviceOptionDefaults, instrumentOpts); err != nil { + serviceOptionDefaults, instrumentOpts) + if err != nil { return err } - if err := placement.RegisterRoutes(routeFn, serviceOptionDefaults, - placementOpts); err != nil { + + err = placement.RegisterRoutes(h.registry, + serviceOptionDefaults, placementOpts) + if err != nil { return err } - if err := namespace.RegisterRoutes(wrappedRouteFn, clusterClient, - h.options.Clusters(), serviceOptionDefaults, instrumentOpts); err != nil { + + err = namespace.RegisterRoutes(h.registry, clusterClient, + h.options.Clusters(), serviceOptionDefaults, instrumentOpts) + if err != nil { return err } - if err := topic.RegisterRoutes(wrappedRouteFn, clusterClient, config, - instrumentOpts); err != nil { + + err = topic.RegisterRoutes(h.registry, clusterClient, config, instrumentOpts) + if err != nil { return err } @@ -366,8 +426,11 @@ func (h *Handler) RegisterRoutes() error { Tagged(remoteSource). Tagged(experimentalAPIGroup), ) - if err := wrappedRouteFn(annotated.WriteURL, experimentalAnnotatedWriteHandler, - annotated.WriteHTTPMethod); err != nil { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: annotated.WriteURL, + Handler: experimentalAnnotatedWriteHandler, + Methods: methods(annotated.WriteHTTPMethod), + }); err != nil { return err } } @@ -376,32 +439,40 @@ func (h *Handler) RegisterRoutes() error { if err := h.registerHealthEndpoints(); err != nil { return err } - h.registerProfileEndpoints() + if err := h.registerProfileEndpoints(); err != nil { + return err + } if err := h.registerRoutesEndpoint(); err != nil { return err } - // Register custom endpoints. + // Register custom endpoints last to have these conflict with + // any existing routes. for _, custom := range h.customHandlers { for _, method := range custom.Methods() { - routeName := routeName(custom.Route(), method) - route := h.router.Get(routeName) var prevHandler http.Handler - if route != nil { + route, prevRoute := h.registry.PathRoute(custom.Route(), method) + if prevRoute { prevHandler = route.GetHandler() } - customHandler, err := custom.Handler(nativeSourceOpts, prevHandler) + + handler, err := custom.Handler(nativeSourceOpts, prevHandler) if err != nil { - return fmt.Errorf("failed to register custom handler with path %s: %w", - routeName, err) + return err } - if route == nil { - if err := wrappedRouteFn(custom.Route(), customHandler, method); err != nil { + if !prevRoute { + if err := h.registry.Register(queryhttp.RegisterOptions{ + Path: custom.Route(), + Handler: handler, + Methods: methods(method), + }); err != nil { return err } } else { - route.Handler(wrapped(customHandler)) + // Do not re-instrument this route since the prev handler + // is already instrumented. + route.Handler(handler) } } } @@ -409,34 +480,6 @@ func (h *Handler) RegisterRoutes() error { return nil } -func (h *Handler) addRouteHandlerFn( - router *mux.Router, - path string, - handlerFn http.HandlerFunc, - methods ...string, -) error { - for _, method := range methods { - routeName := routeName(path, method) - if previousRoute := router.Get(routeName); previousRoute != nil { - return fmt.Errorf("route already exists: %s", routeName) - } - - router. - HandleFunc(path, handlerFn). - Name(routeName). - Methods(method) - } - - return nil -} - -func routeName(p string, method string) string { - if method == "" { - return p - } - return fmt.Sprintf("%s %s", p, method) -} - func (h *Handler) placementOpts() (placement.HandlerOptions, error) { return placement.NewHandlerOptions( h.options.ClusterClient(), @@ -471,44 +514,56 @@ func (h *Handler) m3AggServiceOptions() *handleroptions.M3AggServiceOptions { // Endpoints useful for profiling the service. func (h *Handler) registerHealthEndpoints() error { - return h.addRouteHandlerFn(h.router, healthURL, func(w http.ResponseWriter, r *http.Request) { - json.NewEncoder(w).Encode(struct { - Uptime string `json:"uptime"` - }{ - Uptime: time.Since(h.options.CreatedAt()).String(), - }) - }, http.MethodGet) + return h.registry.Register(queryhttp.RegisterOptions{ + Path: healthURL, + Handler: http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + json.NewEncoder(w).Encode(struct { + Uptime string `json:"uptime"` + }{ + Uptime: time.Since(h.options.CreatedAt()).String(), + }) + }), + Methods: methods(http.MethodGet), + }) } // Endpoints useful for profiling the service. -func (h *Handler) registerProfileEndpoints() { - h.router. - PathPrefix("/debug/pprof/"). - Handler(http.DefaultServeMux). - Name("/debug/pprof/") +func (h *Handler) registerProfileEndpoints() error { + return h.registry.Register(queryhttp.RegisterOptions{ + PathPrefix: "/debug/pprof", + Handler: http.DefaultServeMux, + }) } // Endpoints useful for viewing routes directory. func (h *Handler) registerRoutesEndpoint() error { - return h.addRouteHandlerFn(h.router, routesURL, func(w http.ResponseWriter, r *http.Request) { - var routes []string - err := h.router.Walk( - func(route *mux.Route, router *mux.Router, ancestors []*mux.Route) error { - str, err := route.GetPathTemplate() - if err != nil { - return err - } - routes = append(routes, str) - return nil + return h.registry.Register(queryhttp.RegisterOptions{ + Path: routesURL, + Handler: http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { + var routes []string + err := h.registry.Walk( + func(route *mux.Route, router *mux.Router, ancestors []*mux.Route) error { + str, err := route.GetPathTemplate() + if err != nil { + return err + } + routes = append(routes, str) + return nil + }) + if err != nil { + xhttp.WriteError(w, err) + return + } + json.NewEncoder(w).Encode(struct { + Routes []string `json:"routes"` + }{ + Routes: routes, }) - if err != nil { - xhttp.WriteError(w, err) - return - } - json.NewEncoder(w).Encode(struct { - Routes []string `json:"routes"` - }{ - Routes: routes, - }) - }, http.MethodGet) + }), + Methods: methods(http.MethodGet), + }) +} + +func methods(str ...string) []string { + return str } diff --git a/src/query/api/v1/httpd/handler_test.go b/src/query/api/v1/httpd/handler_test.go index 777cf361c8..53558d7192 100644 --- a/src/query/api/v1/httpd/handler_test.go +++ b/src/query/api/v1/httpd/handler_test.go @@ -28,12 +28,8 @@ import ( "testing" "time" - "github.com/prometheus/prometheus/promql" - "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest" - dbconfig "github.com/m3db/m3/src/cmd/services/m3dbnode/config" "github.com/m3db/m3/src/cmd/services/m3query/config" - "github.com/m3db/m3/src/dbnode/client" m3json "github.com/m3db/m3/src/query/api/v1/handler/json" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/native" @@ -45,12 +41,14 @@ import ( "github.com/m3db/m3/src/query/storage" "github.com/m3db/m3/src/query/test/m3" "github.com/m3db/m3/src/query/ts/m3db" + "github.com/m3db/m3/src/query/util/queryhttp" "github.com/m3db/m3/src/x/instrument" xsync "github.com/m3db/m3/src/x/sync" "github.com/golang/mock/gomock" "github.com/gorilla/mux" "github.com/opentracing/opentracing-go/mocktracer" + "github.com/prometheus/prometheus/promql" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -93,6 +91,13 @@ func setupHandler( instrumentOpts := instrument.NewOptions() downsamplerAndWriter := ingest.NewDownsamplerAndWriter(store, nil, testWorkerPool, instrument.NewOptions()) engine := newEngine(store, time.Minute, instrumentOpts) + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + if err != nil { + return nil, err + } opts, err := options.NewHandlerOptions( downsamplerAndWriter, makeTagOptions(), @@ -102,7 +107,7 @@ func setupHandler( nil, config.Configuration{LookbackDuration: &defaultLookbackDuration}, nil, - handleroptions.NewFetchOptionsBuilder(handleroptions.FetchOptionsBuilderOptions{}), + fetchOptsBuilder, models.QueryContextOptions{}, instrumentOpts, defaultCPUProfileduration, @@ -113,7 +118,6 @@ func setupHandler( graphite.M3WrappedStorageOptions{}, testM3DBOpts, ) - if err != nil { return nil, err } @@ -128,41 +132,6 @@ func newPromEngine() *promql.Engine { }) } -func TestHandlerFetchTimeout(t *testing.T) { - ctrl := gomock.NewController(t) - storage, _ := m3.NewStorageAndSession(t, ctrl) - downsamplerAndWriter := ingest.NewDownsamplerAndWriter(storage, nil, testWorkerPool, instrument.NewOptions()) - - fourMin := 4 * time.Minute - dbconfig := &dbconfig.DBConfiguration{Client: client.Configuration{FetchTimeout: &fourMin}} - engine := newEngine(storage, time.Minute, instrument.NewOptions()) - cfg := config.Configuration{LookbackDuration: &defaultLookbackDuration} - opts, err := options.NewHandlerOptions( - downsamplerAndWriter, - makeTagOptions(), - engine, - nil, - nil, - nil, - cfg, - dbconfig, - handleroptions.NewFetchOptionsBuilder(handleroptions.FetchOptionsBuilderOptions{}), - models.QueryContextOptions{}, - instrument.NewOptions(), - defaultCPUProfileduration, - defaultPlacementServices, - svcDefaultOptions, - nil, - nil, - graphite.M3WrappedStorageOptions{}, - testM3DBOpts, - ) - require.NoError(t, err) - - h := NewHandler(opts) - assert.Equal(t, 4*time.Minute, h.options.TimeoutOpts().FetchTimeout) -} - func TestPromRemoteReadGet(t *testing.T) { req := httptest.NewRequest("GET", remote.PromReadURL, nil) res := httptest.NewRecorder() @@ -171,7 +140,6 @@ func TestPromRemoteReadGet(t *testing.T) { h, err := setupHandler(storage) require.NoError(t, err, "unable to setup handler") - assert.Equal(t, 30*time.Second, h.options.TimeoutOpts().FetchTimeout) err = h.RegisterRoutes() require.NoError(t, err, "unable to register routes") h.Router().ServeHTTP(res, req) @@ -320,7 +288,7 @@ func TestCORSMiddleware(t *testing.T) { h, err := setupHandler(s) require.NoError(t, err, "unable to setup handler") - setupTestRoute(h.router) + setupTestRouteRegistry(h.registry) res := doTestRequest(h.Router()) assert.Equal(t, "hello!", res.Body.String()) @@ -337,7 +305,7 @@ func doTestRequest(handler http.Handler) *httptest.ResponseRecorder { func TestTracingMiddleware(t *testing.T) { mtr := mocktracer.New() router := mux.NewRouter() - setupTestRoute(router) + setupTestRouteRouter(router) handler := applyMiddleware(router, mtr) doTestRequest(handler) @@ -348,7 +316,7 @@ func TestTracingMiddleware(t *testing.T) { func TestCompressionMiddleware(t *testing.T) { mtr := mocktracer.New() router := mux.NewRouter() - setupTestRoute(router) + setupTestRouteRouter(router) handler := applyMiddleware(router, mtr) req := httptest.NewRequest("GET", testRoute, nil) @@ -364,7 +332,18 @@ func TestCompressionMiddleware(t *testing.T) { const testRoute = "/foobar" -func setupTestRoute(r *mux.Router) { +func setupTestRouteRegistry(r *queryhttp.EndpointRegistry) { + r.Register(queryhttp.RegisterOptions{ + Path: testRoute, + Handler: http.HandlerFunc(func(writer http.ResponseWriter, r *http.Request) { + writer.WriteHeader(http.StatusOK) + writer.Write([]byte("hello!")) + }), + Methods: methods(http.MethodGet), + }) +} + +func setupTestRouteRouter(r *mux.Router) { r.HandleFunc(testRoute, func(writer http.ResponseWriter, r *http.Request) { writer.WriteHeader(http.StatusOK) writer.Write([]byte("hello!")) @@ -389,10 +368,10 @@ func init() { type assertFn func(t *testing.T, prev http.Handler, r *http.Request) type customHandler struct { - t *testing.T + t *testing.T routeName string - methods []string - assertFn assertFn + methods []string + assertFn assertFn } func (h *customHandler) Route() string { return h.routeName } @@ -417,44 +396,47 @@ func TestCustomRoutes(t *testing.T) { instrumentOpts := instrument.NewOptions() downsamplerAndWriter := ingest.NewDownsamplerAndWriter(store, nil, testWorkerPool, instrument.NewOptions()) engine := newEngine(store, time.Minute, instrumentOpts) + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Timeout: 15 * time.Second, + }) + require.NoError(t, err) opts, err := options.NewHandlerOptions( downsamplerAndWriter, makeTagOptions().SetMetricName([]byte("z")), engine, newPromEngine(), nil, nil, config.Configuration{LookbackDuration: &defaultLookbackDuration}, nil, - handleroptions.NewFetchOptionsBuilder(handleroptions.FetchOptionsBuilderOptions{}), - models.QueryContextOptions{}, instrumentOpts, defaultCPUProfileduration, + fetchOptsBuilder, models.QueryContextOptions{}, instrumentOpts, defaultCPUProfileduration, defaultPlacementServices, svcDefaultOptions, NewQueryRouter(), NewQueryRouter(), graphite.M3WrappedStorageOptions{}, testM3DBOpts) - require.NoError(t, err) custom := &customHandler{ - t: t, + t: t, routeName: "/custom", - methods: []string{http.MethodGet, http.MethodHead}, + methods: []string{http.MethodGet, http.MethodHead}, assertFn: func(t *testing.T, prev http.Handler, r *http.Request) { assert.Nil(t, prev, "Should not shadow already existing handler") }, } customShadowGet := &customHandler{ - t: t, + t: t, routeName: "/custom", - methods: []string{http.MethodGet}, + methods: []string{http.MethodGet}, assertFn: func(t *testing.T, prev http.Handler, r *http.Request) { assert.NotNil(t, prev, "Should shadow already existing handler") }, } customShadowHead := &customHandler{ - t: t, + t: t, routeName: "/custom", - methods: []string{http.MethodHead}, + methods: []string{http.MethodHead}, assertFn: func(t *testing.T, prev http.Handler, r *http.Request) { assert.NotNil(t, prev, "Should shadow already existing handler") }, } customNew := &customHandler{ - t: t, + t: t, routeName: "/custom/new", - methods: []string{http.MethodGet, http.MethodHead}, + methods: []string{http.MethodGet, http.MethodHead}, assertFn: func(t *testing.T, prev http.Handler, r *http.Request) { assert.Nil(t, prev, "Should not shadow already existing handler") }, @@ -482,23 +464,3 @@ func assertRoute(t *testing.T, routeName string, method string, handler *Handler handler.Router().ServeHTTP(res, req) require.Equal(t, expectedStatusCode, res.Code) } - -func TestRouteName(t *testing.T) { - assert.Equal( - t, - "/api/v1/test GET", - routeName("/api/v1/test", "GET"), - ) - - assert.Equal( - t, - "/api/v1/test", - routeName("/api/v1/test", ""), - ) - - assert.Equal( - t, - "/api/v1/test POST", - routeName("/api/v1/test", "POST"), - ) -} diff --git a/src/query/api/v1/options/handler.go b/src/query/api/v1/options/handler.go index a178b1bd33..db69f58611 100644 --- a/src/query/api/v1/options/handler.go +++ b/src/query/api/v1/options/handler.go @@ -30,7 +30,6 @@ import ( "github.com/m3db/m3/src/cmd/services/m3coordinator/ingest" dbconfig "github.com/m3db/m3/src/cmd/services/m3dbnode/config" "github.com/m3db/m3/src/cmd/services/m3query/config" - "github.com/m3db/m3/src/query/api/v1/handler/prometheus" "github.com/m3db/m3/src/query/api/v1/handler/prometheus/handleroptions" "github.com/m3db/m3/src/query/executor" graphite "github.com/m3db/m3/src/query/graphite/storage" @@ -143,11 +142,6 @@ type HandlerOptions interface { // SetTagOptions sets the tag options. SetTagOptions(opts models.TagOptions) HandlerOptions - // TimeoutOpts returns the timeout options. - TimeoutOpts() *prometheus.TimeoutOpts - // SetTimeoutOpts sets the timeout options. - SetTimeoutOpts(t *prometheus.TimeoutOpts) HandlerOptions - // FetchOptionsBuilder returns the fetch options builder. FetchOptionsBuilder() handleroptions.FetchOptionsBuilder // SetFetchOptionsBuilder sets the fetch options builder. @@ -230,7 +224,6 @@ type handlerOptions struct { embeddedDbCfg *dbconfig.DBConfiguration createdAt time.Time tagOptions models.TagOptions - timeoutOpts *prometheus.TimeoutOpts fetchOptionsBuilder handleroptions.FetchOptionsBuilder queryContextOptions models.QueryContextOptions instrumentOpts instrument.Options @@ -275,13 +268,6 @@ func NewHandlerOptions( graphiteStorageOpts graphite.M3WrappedStorageOptions, m3dbOpts m3db.Options, ) (HandlerOptions, error) { - timeout := cfg.Query.TimeoutOrDefault() - if embeddedDbCfg != nil && - embeddedDbCfg.Client.FetchTimeout != nil && - *embeddedDbCfg.Client.FetchTimeout > timeout { - timeout = *embeddedDbCfg.Client.FetchTimeout - } - storeMetricsType := false if cfg.StoreMetricsType != nil { storeMetricsType = *cfg.StoreMetricsType @@ -306,14 +292,11 @@ func NewHandlerOptions( placementServiceNames: placementServiceNames, serviceOptionDefaults: serviceOptionDefaults, nowFn: time.Now, - timeoutOpts: &prometheus.TimeoutOpts{ - FetchTimeout: timeout, - }, - queryRouter: queryRouter, - instantQueryRouter: instantQueryRouter, - graphiteStorageOpts: graphiteStorageOpts, - m3dbOpts: m3dbOpts, - storeMetricsType: storeMetricsType, + queryRouter: queryRouter, + instantQueryRouter: instantQueryRouter, + graphiteStorageOpts: graphiteStorageOpts, + m3dbOpts: m3dbOpts, + storeMetricsType: storeMetricsType, }, nil } @@ -414,16 +397,6 @@ func (o *handlerOptions) SetTagOptions(tags models.TagOptions) HandlerOptions { return &opts } -func (o *handlerOptions) TimeoutOpts() *prometheus.TimeoutOpts { - return o.timeoutOpts -} - -func (o *handlerOptions) SetTimeoutOpts(t *prometheus.TimeoutOpts) HandlerOptions { - opts := *o - opts.timeoutOpts = t - return &opts -} - func (o *handlerOptions) FetchOptionsBuilder() handleroptions.FetchOptionsBuilder { return o.fetchOptionsBuilder } diff --git a/src/query/graphite/native/builtin_functions.go b/src/query/graphite/native/builtin_functions.go index f757ddb528..bc65fc74d6 100644 --- a/src/query/graphite/native/builtin_functions.go +++ b/src/query/graphite/native/builtin_functions.go @@ -2375,7 +2375,6 @@ func init() { }) MustRegisterFunction(asPercent).WithDefaultParams(map[uint8]interface{}{ 2: []*ts.Series(nil), // total - 3: nil, // nodes }) MustRegisterFunction(averageAbove) MustRegisterFunction(averageSeries) @@ -2406,9 +2405,7 @@ func init() { MustRegisterFunction(groupByNode).WithDefaultParams(map[uint8]interface{}{ 3: "average", // fname }) - MustRegisterFunction(groupByNodes).WithDefaultParams(map[uint8]interface{}{ - 3: nil, // nodes - }) + MustRegisterFunction(groupByNodes) MustRegisterFunction(highest).WithDefaultParams(map[uint8]interface{}{ 2: 1, // n, 3: "average", // f diff --git a/src/query/graphite/native/compiler.go b/src/query/graphite/native/compiler.go index bbca546914..fec31a438b 100644 --- a/src/query/graphite/native/compiler.go +++ b/src/query/graphite/native/compiler.go @@ -170,6 +170,11 @@ func (c *compiler) compileFunctionCall(fname string, nextToken *lexer.Token) (*f } argTypes := fn.in + argTypesRequired := len(fn.in) + if fn.variadic { + // Variadic can avoid specifying the last arg. + argTypesRequired-- + } var args []funcArg // build up arguments for function call @@ -206,7 +211,7 @@ func (c *compiler) compileFunctionCall(fname string, nextToken *lexer.Token) (*f } // all required argument types should be filled with values now - if len(args) < len(argTypes) { + if len(args) < argTypesRequired { variadicComment := "" if fn.variadic { variadicComment = "at least " diff --git a/src/query/graphite/native/compiler_test.go b/src/query/graphite/native/compiler_test.go index 40f917082f..589bdf1ee3 100644 --- a/src/query/graphite/native/compiler_test.go +++ b/src/query/graphite/native/compiler_test.go @@ -309,9 +309,6 @@ func TestCompileErrors(t *testing.T) { {"aliasByNode()", "invalid expression 'aliasByNode()': invalid number of arguments for aliasByNode;" + " expected at least 2, received 0"}, - {"aliasByNode(foo.*.zed)", // check that at least 1 param is provided for variadic args - "invalid expression 'aliasByNode(foo.*.zed)': invalid number of arguments for " + - "aliasByNode; expected at least 2, received 1"}, {"aliasByNode(foo.*.zed, 2, false)", "invalid expression 'aliasByNode(foo.*.zed, 2, false)': invalid function call " + "aliasByNode, arg 2: expected a int, received 'false'"}, @@ -327,9 +324,6 @@ func TestCompileErrors(t *testing.T) { {"summarize(foo.bar.baz.quux)", "invalid expression 'summarize(foo.bar.baz.quux)':" + " invalid number of arguments for summarize; expected 4, received 1"}, - {"sumSeries()", // check that at least 1 series is provided for variadic timeSeriesList - "invalid expression 'sumSeries()': invalid number of arguments for sumSeries;" + - " expected at least 1, received 0"}, {"sumSeries(foo.bar.baz.quux,)", "invalid expression 'sumSeries(foo.bar.baz.quux,)': invalid function call sumSeries, " + "arg 1: invalid expression 'sumSeries(foo.bar.baz.quux,)': ) not valid"}, diff --git a/src/query/graphite/native/engine_test.go b/src/query/graphite/native/engine_test.go index cc4f9d229a..55d2c8b5c2 100644 --- a/src/query/graphite/native/engine_test.go +++ b/src/query/graphite/native/engine_test.go @@ -125,6 +125,14 @@ func TestExecute(t *testing.T) { {"foo.bar.q.zed", "foo.q", 0}, {"foo.bar.x.zed", "foo.x", 2}, }}, + {"groupByNodes(foo.bar.*.zed, \"sum\")", false, []queryTestResult{ + {"foo.bar.*.zed", "foo.bar.*.zed", 3}, + }}, + {"groupByNodes(foo.bar.*.zed, \"sum\", 2)", false, []queryTestResult{ + {"foo.bar.q.zed", "foo.bar.q.zed", 0}, + {"foo.bar.g.zed", "foo.bar.g.zed", 1}, + {"foo.bar.x.zed", "foo.bar.x.zed", 2}, + }}, } ctx := common.NewContext(common.ContextOptions{Start: time.Now().Add(-1 * time.Hour), End: time.Now(), Engine: engine}) @@ -140,7 +148,7 @@ func TestExecute(t *testing.T) { buildTestSeriesFn(stepSize, queries...)) expr, err := engine.Compile(test.query) - require.Nil(t, err) + require.NoError(t, err) results, err := expr.Execute(ctx) require.Nil(t, err, "failed to execute %s", test.query) diff --git a/src/query/graphite/native/functions.go b/src/query/graphite/native/functions.go index b68fa5bf10..b0109922ac 100644 --- a/src/query/graphite/native/functions.go +++ b/src/query/graphite/native/functions.go @@ -422,7 +422,12 @@ func (f *Function) reflectCall(ctx *common.Context, args []reflect.Value) (refle } numTypes := len(f.in) - if len(in) < numTypes { + numRequiredTypes := numTypes + if f.variadic { + // Variadic can avoid specifying the last arg. + numRequiredTypes-- + } + if len(in) < numRequiredTypes { err := fmt.Errorf("call args mismatch: expected at least %d, actual %d", len(f.in), len(in)) return reflect.Value{}, err diff --git a/src/query/server/query.go b/src/query/server/query.go index 8dfdfa407b..303ea23fd4 100644 --- a/src/query/server/query.go +++ b/src/query/server/query.go @@ -273,18 +273,30 @@ func Run(runOpts RunOptions) { logger.Fatal("could not parse query restrict tags config", zap.Error(err)) } + timeout := cfg.Query.TimeoutOrDefault() + if runOpts.DBConfig != nil && + runOpts.DBConfig.Client.FetchTimeout != nil && + *runOpts.DBConfig.Client.FetchTimeout > timeout { + timeout = *runOpts.DBConfig.Client.FetchTimeout + } + + fetchOptsBuilderLimitsOpts := cfg.Limits.PerQuery.AsFetchOptionsBuilderLimitsOptions() + fetchOptsBuilder, err := handleroptions.NewFetchOptionsBuilder( + handleroptions.FetchOptionsBuilderOptions{ + Limits: fetchOptsBuilderLimitsOpts, + RestrictByTag: storageRestrictByTags, + Timeout: timeout, + }) + if err != nil { + logger.Fatal("could not set fetch options parser", zap.Error(err)) + } + var ( - clusterNamespacesWatcher m3.ClusterNamespacesWatcher - backendStorage storage.Storage - clusterClient clusterclient.Client - downsampler downsample.Downsampler - fetchOptsBuilderLimitsOpts = cfg.Limits.PerQuery.AsFetchOptionsBuilderLimitsOptions() - fetchOptsBuilder = handleroptions.NewFetchOptionsBuilder( - handleroptions.FetchOptionsBuilderOptions{ - Limits: fetchOptsBuilderLimitsOpts, - RestrictByTag: storageRestrictByTags, - }) - queryCtxOpts = models.QueryContextOptions{ + clusterNamespacesWatcher m3.ClusterNamespacesWatcher + backendStorage storage.Storage + clusterClient clusterclient.Client + downsampler downsample.Downsampler + queryCtxOpts = models.QueryContextOptions{ LimitMaxTimeseries: fetchOptsBuilderLimitsOpts.SeriesLimit, LimitMaxDocs: fetchOptsBuilderLimitsOpts.DocsLimit, RequireExhaustive: fetchOptsBuilderLimitsOpts.RequireExhaustive, diff --git a/src/query/util/logging/log.go b/src/query/util/logging/log.go index 07b31c7ce4..683eff8bfc 100644 --- a/src/query/util/logging/log.go +++ b/src/query/util/logging/log.go @@ -23,6 +23,7 @@ package logging import ( "context" "fmt" + "io" "net/http" "sync" "time" @@ -121,9 +122,24 @@ func withResponseTimeLogging( func withResponseTimeLoggingFunc( next func(w http.ResponseWriter, r *http.Request), instrumentOpts instrument.Options, + opts ...MiddlewareOption, ) http.HandlerFunc { + middlewareOpts := defaultMiddlewareOptions + for _, opt := range opts { + opt(&middlewareOpts) + } + + threshold := middlewareOpts.responseLogThreshold return func(w http.ResponseWriter, r *http.Request) { startTime := time.Now() + for _, fn := range middlewareOpts.preHooks { + fn(r) + } + + // Track status code. + statusCodeTracking := &statusCodeTracker{ResponseWriter: w} + w = statusCodeTracking.wrappedResponseWriter() + rqCtx := NewContextWithGeneratedID(r.Context(), instrumentOpts) logger := WithContext(rqCtx, instrumentOpts) @@ -137,10 +153,20 @@ func withResponseTimeLoggingFunc( next(w, r.WithContext(rqCtx)) endTime := time.Now() d := endTime.Sub(startTime) - if d > time.Second { + if threshold > 0 && d >= threshold { logger.Info("finished handling request", zap.Time("time", endTime), zap.Duration("response", d), zap.String("url", r.URL.RequestURI())) } + + for _, fn := range middlewareOpts.postHooks { + fn(r, RequestMiddlewareMetadata{ + Start: startTime, + End: endTime, + Duration: d, + WroteHeader: statusCodeTracking.wroteHeader, + StatusCode: statusCodeTracking.status, + }) + } } } @@ -225,8 +251,10 @@ func (w *responseWrittenResponseWriter) WriteHeader(statusCode int) { func WithResponseTimeAndPanicErrorLogging( next http.Handler, instrumentOpts instrument.Options, + opts ...MiddlewareOption, ) http.Handler { - return WithResponseTimeAndPanicErrorLoggingFunc(next.ServeHTTP, instrumentOpts) + return WithResponseTimeAndPanicErrorLoggingFunc(next.ServeHTTP, + instrumentOpts, opts...) } // WithResponseTimeAndPanicErrorLoggingFunc wraps around the http request @@ -234,10 +262,318 @@ func WithResponseTimeAndPanicErrorLogging( func WithResponseTimeAndPanicErrorLoggingFunc( next func(w http.ResponseWriter, r *http.Request), instrumentOpts instrument.Options, + opts ...MiddlewareOption, ) http.Handler { // Wrap panic first, to be able to capture slow requests that panic in the // logs. return withResponseTimeLoggingFunc( withPanicErrorResponderFunc(next, instrumentOpts), - instrumentOpts) + instrumentOpts, opts...) +} + +type middlewareOptions struct { + responseLogThreshold time.Duration + preHooks []PreRequestMiddleware + postHooks []PostRequestMiddleware +} + +var ( + defaultMiddlewareOptions = middlewareOptions{ + responseLogThreshold: time.Second, + } +) + +// PreRequestMiddleware is middleware that runs before a request. +type PreRequestMiddleware func(req *http.Request) + +// RequestMiddlewareMetadata is metadata available to middleware about a request. +type RequestMiddlewareMetadata struct { + Start time.Time + End time.Time + Duration time.Duration + WroteHeader bool + StatusCode int +} + +// PostRequestMiddleware is middleware that runs before a request. +type PostRequestMiddleware func(req *http.Request, meta RequestMiddlewareMetadata) + +// MiddlewareOption is an option to pass to a middleware. +type MiddlewareOption func(*middlewareOptions) + +// WithResponseLogThreshold is a middleware option to set response log threshold. +func WithResponseLogThreshold(threshold time.Duration) MiddlewareOption { + return func(opts *middlewareOptions) { + opts.responseLogThreshold = threshold + } +} + +// WithNoResponseLog is a middleware option to disable response logging. +func WithNoResponseLog() MiddlewareOption { + return func(opts *middlewareOptions) { + opts.responseLogThreshold = 0 + } +} + +// WithPreRequestMiddleware is a middleware option to set pre-request middleware. +func WithPreRequestMiddleware(m PreRequestMiddleware) MiddlewareOption { + return func(opts *middlewareOptions) { + opts.preHooks = append(opts.preHooks, m) + } +} + +// WithPostRequestMiddleware is a middleware option to set post-request middleware. +func WithPostRequestMiddleware(m PostRequestMiddleware) MiddlewareOption { + return func(opts *middlewareOptions) { + opts.postHooks = append(opts.postHooks, m) + } +} + +type statusCodeTracker struct { + http.ResponseWriter + status int + wroteHeader bool +} + +func (w *statusCodeTracker) WriteHeader(status int) { + w.status = status + w.wroteHeader = true + w.ResponseWriter.WriteHeader(status) +} + +func (w *statusCodeTracker) Write(b []byte) (int, error) { + if !w.wroteHeader { + w.wroteHeader = true + w.status = 200 + } + return w.ResponseWriter.Write(b) +} + +// wrappedResponseWriter returns a wrapped version of the original +// ResponseWriter and only implements the same combination of additional +// interfaces as the original. This implementation is based on +// https://github.com/felixge/httpsnoop. +func (w *statusCodeTracker) wrappedResponseWriter() http.ResponseWriter { + var ( + hj, i0 = w.ResponseWriter.(http.Hijacker) + cn, i1 = w.ResponseWriter.(http.CloseNotifier) + pu, i2 = w.ResponseWriter.(http.Pusher) + fl, i3 = w.ResponseWriter.(http.Flusher) + rf, i4 = w.ResponseWriter.(io.ReaderFrom) + ) + + switch { + case !i0 && !i1 && !i2 && !i3 && !i4: + return struct { + http.ResponseWriter + }{w} + case !i0 && !i1 && !i2 && !i3 && i4: + return struct { + http.ResponseWriter + io.ReaderFrom + }{w, rf} + case !i0 && !i1 && !i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Flusher + }{w, fl} + case !i0 && !i1 && !i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Flusher + io.ReaderFrom + }{w, fl, rf} + case !i0 && !i1 && i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.Pusher + }{w, pu} + case !i0 && !i1 && i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.Pusher + io.ReaderFrom + }{w, pu, rf} + case !i0 && !i1 && i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Pusher + http.Flusher + }{w, pu, fl} + case !i0 && !i1 && i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Pusher + http.Flusher + io.ReaderFrom + }{w, pu, fl, rf} + case !i0 && i1 && !i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.CloseNotifier + }{w, cn} + case !i0 && i1 && !i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.CloseNotifier + io.ReaderFrom + }{w, cn, rf} + case !i0 && i1 && !i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Flusher + }{w, cn, fl} + case !i0 && i1 && !i2 && i3 && i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Flusher + io.ReaderFrom + }{w, cn, fl, rf} + case !i0 && i1 && i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Pusher + }{w, cn, pu} + case !i0 && i1 && i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Pusher + io.ReaderFrom + }{w, cn, pu, rf} + case !i0 && i1 && i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Pusher + http.Flusher + }{w, cn, pu, fl} + case !i0 && i1 && i2 && i3 && i4: + return struct { + http.ResponseWriter + http.CloseNotifier + http.Pusher + http.Flusher + io.ReaderFrom + }{w, cn, pu, fl, rf} + case i0 && !i1 && !i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + }{w, hj} + case i0 && !i1 && !i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + io.ReaderFrom + }{w, hj, rf} + case i0 && !i1 && !i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Flusher + }{w, hj, fl} + case i0 && !i1 && !i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Flusher + io.ReaderFrom + }{w, hj, fl, rf} + case i0 && !i1 && i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Pusher + }{w, hj, pu} + case i0 && !i1 && i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Pusher + io.ReaderFrom + }{w, hj, pu, rf} + case i0 && !i1 && i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Pusher + http.Flusher + }{w, hj, pu, fl} + case i0 && !i1 && i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.Pusher + http.Flusher + io.ReaderFrom + }{w, hj, pu, fl, rf} + case i0 && i1 && !i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + }{w, hj, cn} + case i0 && i1 && !i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + io.ReaderFrom + }{w, hj, cn, rf} + case i0 && i1 && !i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Flusher + }{w, hj, cn, fl} + case i0 && i1 && !i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Flusher + io.ReaderFrom + }{w, hj, cn, fl, rf} + case i0 && i1 && i2 && !i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Pusher + }{w, hj, cn, pu} + case i0 && i1 && i2 && !i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Pusher + io.ReaderFrom + }{w, hj, cn, pu, rf} + case i0 && i1 && i2 && i3 && !i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Pusher + http.Flusher + }{w, hj, cn, pu, fl} + case i0 && i1 && i2 && i3 && i4: + return struct { + http.ResponseWriter + http.Hijacker + http.CloseNotifier + http.Pusher + http.Flusher + io.ReaderFrom + }{w, hj, cn, pu, fl, rf} + default: + return struct { + http.ResponseWriter + }{w} + } } diff --git a/src/query/util/queryhttp/queryhttp.go b/src/query/util/queryhttp/queryhttp.go new file mode 100644 index 0000000000..9ad8fbcf47 --- /dev/null +++ b/src/query/util/queryhttp/queryhttp.go @@ -0,0 +1,273 @@ +// Copyright (c) 2020 Uber Technologies, Inc. +// +// Permission is hereby granted, free of charge, to any person obtaining a copy +// of this software and associated documentation files (the "Software"), to deal +// in the Software without restriction, including without limitation the rights +// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +// copies of the Software, and to permit persons to whom the Software is +// furnished to do so, subject to the following conditions: +// +// The above copyright notice and this permission notice shall be included in +// all copies or substantial portions of the Software. +// +// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +// THE SOFTWARE. + +package queryhttp + +import ( + "fmt" + "net/http" + "strconv" + "sync" + + "github.com/m3db/m3/src/query/util/logging" + "github.com/m3db/m3/src/x/instrument" + "github.com/uber-go/tally" + + "github.com/gorilla/mux" +) + +var ( + histogramTimerOptions = instrument.NewHistogramTimerOptions( + instrument.HistogramTimerOptions{ + // Use sparse histogram timer buckets to not overload with latency metrics. + HistogramBuckets: instrument.SparseHistogramTimerHistogramBuckets(), + }) +) + +// NewEndpointRegistry returns a new endpoint registry. +func NewEndpointRegistry( + router *mux.Router, + instrumentOpts instrument.Options, +) *EndpointRegistry { + return &EndpointRegistry{ + router: router, + instrumentOpts: instrumentOpts.SetMetricsScope( + instrumentOpts.MetricsScope().SubScope("http_handler")), + registered: make(map[routeKey]*mux.Route), + } +} + +// EndpointRegistry is an endpoint registry that can register routes +// and instrument them. +type EndpointRegistry struct { + router *mux.Router + instrumentOpts instrument.Options + registered map[routeKey]*mux.Route +} + +type routeKey struct { + path string + pathPrefix string + method string +} + +// RegisterOptions are options for registering a handler. +type RegisterOptions struct { + Path string + PathPrefix string + Handler http.Handler + Methods []string +} + +// Register registers an endpoint. +func (r *EndpointRegistry) Register( + opts RegisterOptions, + middlewareOpts ...logging.MiddlewareOption, +) error { + // Wrap requests with response time logging as well as panic recovery. + var ( + route *mux.Route + metrics = newRouteMetrics(r.instrumentOpts) + middlewareOptions []logging.MiddlewareOption + ) + postRequestOption := logging.WithPostRequestMiddleware( + logging.PostRequestMiddleware(func( + r *http.Request, + meta logging.RequestMiddlewareMetadata, + ) { + if !meta.WroteHeader { + return + } + + p, err := route.GetPathTemplate() + if err != nil { + p = "unknown" + } + + counter, timer := metrics.metric(p, meta.StatusCode) + counter.Inc(1) + timer.Record(meta.Duration) + })) + middlewareOptions = append(middlewareOptions, postRequestOption) + middlewareOptions = append(middlewareOptions, middlewareOpts...) + + wrapped := func(n http.Handler) http.Handler { + return logging.WithResponseTimeAndPanicErrorLogging(n, r.instrumentOpts, + middlewareOptions...) + } + + handler := wrapped(opts.Handler) + if p := opts.Path; p != "" && len(opts.Methods) > 0 { + for _, method := range opts.Methods { + key := routeKey{ + path: p, + method: method, + } + if _, ok := r.registered[key]; ok { + return fmt.Errorf("route already exists: path=%s, method=%s", + p, method) + } + + route = r.router.HandleFunc(p, handler.ServeHTTP).Methods(method) + r.registered[key] = route + } + } else if p := opts.PathPrefix; p != "" { + key := routeKey{ + pathPrefix: p, + } + if _, ok := r.registered[key]; ok { + return fmt.Errorf("route already exists: pathPrefix=%s", p) + } + route = r.router.PathPrefix(p).Handler(handler) + r.registered[key] = route + } else { + return fmt.Errorf("no path and methods or path prefix set: +%v", opts) + } + + return nil +} + +// RegisterPathsOptions is options for registering multiple paths +// with the same handler. +type RegisterPathsOptions struct { + Handler http.Handler + Methods []string +} + +// RegisterPaths registers multiple paths for the same handler. +func (r *EndpointRegistry) RegisterPaths( + paths []string, + opts RegisterPathsOptions, + middlewareOpts ...logging.MiddlewareOption, +) error { + for _, p := range paths { + if err := r.Register(RegisterOptions{ + Path: p, + Handler: opts.Handler, + Methods: opts.Methods, + }); err != nil { + return err + } + } + return nil +} + +// PathRoute resolves a registered route that was registered by path and method, +// not by path prefix. +func (r *EndpointRegistry) PathRoute(path, method string) (*mux.Route, bool) { + key := routeKey{ + path: path, + method: method, + } + h, ok := r.registered[key] + return h, ok +} + +// PathPrefixRoute resolves a registered route that was registered by path +// prefix, not by path and method. +func (r *EndpointRegistry) PathPrefixRoute(pathPrefix string) (*mux.Route, bool) { + key := routeKey{ + pathPrefix: pathPrefix, + } + h, ok := r.registered[key] + return h, ok +} + +// Walk walks the router and all its sub-routers, calling walkFn for each route +// in the tree. The routes are walked in the order they were added. Sub-routers +// are explored depth-first. +func (r *EndpointRegistry) Walk(walkFn mux.WalkFunc) error { + return r.router.Walk(walkFn) +} + +func routeName(p string, method string) string { + if method == "" { + return p + } + return fmt.Sprintf("%s %s", p, method) +} + +type routeMetrics struct { + sync.RWMutex + instrumentOpts instrument.Options + metrics map[routeMetricKey]routeMetric + timers map[string]tally.Timer +} + +type routeMetricKey struct { + path string + status int +} + +type routeMetric struct { + status tally.Counter +} + +func newRouteMetrics(instrumentOpts instrument.Options) *routeMetrics { + return &routeMetrics{ + instrumentOpts: instrumentOpts, + metrics: make(map[routeMetricKey]routeMetric), + timers: make(map[string]tally.Timer), + } +} + +func (m *routeMetrics) metric(path string, status int) (tally.Counter, tally.Timer) { + key := routeMetricKey{ + path: path, + status: status, + } + m.RLock() + metric, ok1 := m.metrics[key] + timer, ok2 := m.timers[path] + m.RUnlock() + if ok1 && ok2 { + return metric.status, timer + } + + m.Lock() + defer m.Unlock() + + metric, ok1 = m.metrics[key] + timer, ok2 = m.timers[path] + if ok1 && ok2 { + return metric.status, timer + } + + scopePath := m.instrumentOpts.MetricsScope().Tagged(map[string]string{ + "path": path, + }) + + scopePathAndStatus := scopePath.Tagged(map[string]string{ + "status": strconv.Itoa(status), + }) + + if !ok1 { + metric = routeMetric{ + status: scopePathAndStatus.Counter("request"), + } + m.metrics[key] = metric + } + if !ok2 { + timer = instrument.NewTimer(scopePath, "latency", histogramTimerOptions) + m.timers[path] = timer + } + + return metric.status, timer +} diff --git a/src/x/debug/debug.go b/src/x/debug/debug.go index 47d109c5b5..341c562c54 100644 --- a/src/x/debug/debug.go +++ b/src/x/debug/debug.go @@ -40,6 +40,8 @@ import ( const ( // DebugURL is the url for the debug dump endpoint. DebugURL = "/debug/dump" + // DebugMethod is the HTTP method. + DebugMethod = http.MethodGet ) // Source is the interface that must be implemented to provide a new debug diff --git a/src/x/headers/headers.go b/src/x/headers/headers.go index 230521b744..65a4a74fb8 100644 --- a/src/x/headers/headers.go +++ b/src/x/headers/headers.go @@ -110,6 +110,9 @@ const ( // LimitHeader is the header added when returned series are limited. LimitHeader = M3HeaderPrefix + "Results-Limited" + // TimeoutHeader is the header added with the effective timeout. + TimeoutHeader = M3HeaderPrefix + "Timeout" + // LimitHeaderSeriesLimitApplied is the header applied when fetch results // are maxed. LimitHeaderSeriesLimitApplied = "max_fetch_series_limit_applied" diff --git a/src/x/instrument/methods.go b/src/x/instrument/methods.go index fd61e976ab..e3423d4010 100644 --- a/src/x/instrument/methods.go +++ b/src/x/instrument/methods.go @@ -54,6 +54,34 @@ func (o TimerOptions) NewTimer(scope tally.Scope, name string) tally.Timer { return NewTimer(scope, name, o) } +// SparseHistogramTimerHistogramBuckets returns a small spare set of +// histogram timer histogram buckets, from 1ms up to 8m. +func SparseHistogramTimerHistogramBuckets() tally.Buckets { + return tally.ValueBuckets{ + 0.001, + 0.005, + 0.01, + 0.025, + 0.05, + 0.075, + 0.1, + 0.25, + 0.5, + 0.75, + 1, + 2.5, + 5, + 7.5, + 10, + 25, + 50, + 75, + 100, + 250, + 500, + } +} + // DefaultHistogramTimerHistogramBuckets returns a set of default // histogram timer histogram buckets, from 2ms up to 1hr. func DefaultHistogramTimerHistogramBuckets() tally.Buckets { diff --git a/src/x/net/http/errors.go b/src/x/net/http/errors.go index fb90d73874..a0fbcde08d 100644 --- a/src/x/net/http/errors.go +++ b/src/x/net/http/errors.go @@ -92,25 +92,27 @@ func WriteError(w http.ResponseWriter, err error, opts ...WriteErrorOption) { fn(&o) } + statusCode := getStatusCode(err) + if o.response == nil { + w.Header().Set(HeaderContentType, ContentTypeJSON) + w.WriteHeader(statusCode) + json.NewEncoder(w).Encode(ErrorResponse{Error: err.Error()}) + } else { + w.WriteHeader(statusCode) + w.Write(o.response) + } +} + +func getStatusCode(err error) int { switch v := err.(type) { case Error: - w.WriteHeader(v.Code()) + return v.Code() case error: if xerrors.IsInvalidParams(v) { - w.WriteHeader(http.StatusBadRequest) + return http.StatusBadRequest } else if errors.Is(err, context.DeadlineExceeded) { - w.WriteHeader(http.StatusGatewayTimeout) - } else { - w.WriteHeader(http.StatusInternalServerError) + return http.StatusGatewayTimeout } - default: - w.WriteHeader(http.StatusInternalServerError) } - - if o.response != nil { - w.Write(o.response) - return - } - - json.NewEncoder(w).Encode(ErrorResponse{Error: err.Error()}) + return http.StatusInternalServerError }