Skip to content

Commit

Permalink
[query] Fix time() function in binary comparisons (#1888)
Browse files Browse the repository at this point in the history
  • Loading branch information
arnikola authored Aug 22, 2019
1 parent 9583d1e commit f04a2e1
Show file tree
Hide file tree
Showing 40 changed files with 768 additions and 309 deletions.
11 changes: 8 additions & 3 deletions src/query/block/accounted.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,22 +22,27 @@ package block

import "github.com/m3db/m3/src/query/cost"

// AccountedBlock is a wrapper for a block which enforces limits on the number of datapoints used by the block.
// AccountedBlock is a wrapper for a block which enforces limits on the number
// of datapoints used by the block.
type AccountedBlock struct {
Block

enforcer cost.ChainedEnforcer
}

// NewAccountedBlock wraps the given block and enforces datapoint limits.
func NewAccountedBlock(wrapped Block, enforcer cost.ChainedEnforcer) *AccountedBlock {
func NewAccountedBlock(
wrapped Block,
enforcer cost.ChainedEnforcer,
) *AccountedBlock {
return &AccountedBlock{
Block: wrapped,
enforcer: enforcer,
}
}

// Close closes the block, and marks the number of datapoints used by this block as finished.
// Close closes the block, and marks the number of datapoints used
// by this block as finished.
func (ab *AccountedBlock) Close() error {
ab.enforcer.Close()
return ab.Block.Close()
Expand Down
7 changes: 6 additions & 1 deletion src/query/block/accounted_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/m3db/m3/src/query/cost"

"github.com/golang/mock/gomock"
"github.com/stretchr/testify/assert"
)

func TestAccountedBlock_Close(t *testing.T) {
Expand All @@ -37,5 +38,9 @@ func TestAccountedBlock_Close(t *testing.T) {
mockEnforcer := cost.NewMockChainedEnforcer(ctrl)
mockEnforcer.EXPECT().Close()

NewAccountedBlock(wrapped, mockEnforcer).Close()
block := NewAccountedBlock(wrapped, mockEnforcer)

wrapped.EXPECT().Info().Return(NewBlockInfo(BlockM3TSZCompressed))
assert.Equal(t, BlockM3TSZCompressed, block.Info().Type())
assert.NotPanics(t, func() { block.Close() })
}
28 changes: 28 additions & 0 deletions src/query/block/block_mock.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

15 changes: 12 additions & 3 deletions src/query/block/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@ type ColumnBlockBuilder struct {
}

type columnBlock struct {
blockType BlockType
columns []column
meta Metadata
seriesMeta []SeriesMeta
Expand Down Expand Up @@ -78,6 +79,7 @@ func (c *columnBlock) WithMetadata(
columns: c.columns,
meta: meta,
seriesMeta: seriesMetas,
blockType: BlockDecompressed,
}, nil
}

Expand All @@ -90,6 +92,10 @@ func (c *columnBlock) StepCount() int {
return len(c.columns)
}

func (c *columnBlock) Info() BlockInfo {
return NewBlockInfo(c.blockType)
}

// Close frees up any resources
// TODO: actually free up the resources
func (c *columnBlock) Close() error {
Expand Down Expand Up @@ -182,6 +188,7 @@ func NewColumnBlockBuilder(
block: &columnBlock{
meta: meta,
seriesMeta: seriesMeta,
blockType: BlockDecompressed,
},
}
}
Expand Down Expand Up @@ -222,19 +229,21 @@ func (cb ColumnBlockBuilder) AppendValues(idx int, values []float64) error {
return nil
}

// AddCols adds new columns
func (cb ColumnBlockBuilder) AddCols(num int) error {
newCols := make([]column, num)
cb.block.columns = append(cb.block.columns, newCols...)
return nil
}

// Build extracts the block
// TODO: Return an immutable copy
func (cb ColumnBlockBuilder) Build() Block {
return NewAccountedBlock(cb.block, cb.enforcer)
}

func (cb ColumnBlockBuilder) BuildAsType(blockType BlockType) Block {
cb.block.blockType = blockType
return NewAccountedBlock(cb.block, cb.enforcer)
}

type column struct {
Values []float64
}
Expand Down
45 changes: 45 additions & 0 deletions src/query/block/column_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
// Copyright (c) 2019 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.

package block

import (
"context"
"testing"

"github.com/m3db/m3/src/query/cost"
"github.com/m3db/m3/src/query/models"

"github.com/stretchr/testify/assert"
"github.com/uber-go/tally"
)

func TestColumnBuilderInfoTypes(t *testing.T) {
ctx := models.NewQueryContext(context.Background(),
tally.NoopScope, cost.NoopChainedEnforcer(),
models.QueryContextOptions{})

builder := NewColumnBlockBuilder(ctx, Metadata{}, []SeriesMeta{})
block := builder.Build()
assert.Equal(t, BlockDecompressed, block.Info().blockType)

block = builder.BuildAsType(BlockScalar)
assert.Equal(t, BlockScalar, block.Info().blockType)
}
4 changes: 4 additions & 0 deletions src/query/block/container.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,10 @@ func (b *containerBlock) AddBlock(bl Block) error {
return nil
}

func (c *containerBlock) Info() BlockInfo {
return NewBlockInfo(BlockContainer)
}

func (b *containerBlock) Close() error {
multiErr := xerrors.NewMultiError()
multiErr = multiErr.Add(b.err)
Expand Down
4 changes: 4 additions & 0 deletions src/query/block/empty.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,10 @@ func NewEmptyBlock(meta Metadata) Block {

func (b *emptyBlock) Close() error { return nil }

func (b *emptyBlock) Info() BlockInfo {
return NewBlockInfo(BlockEmpty)
}

func (b *emptyBlock) WithMetadata(meta Metadata, _ []SeriesMeta) (Block, error) {
return NewEmptyBlock(meta), nil
}
Expand Down
88 changes: 88 additions & 0 deletions src/query/block/info.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
// Copyright (c) 2019 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.

package block

func (t BlockType) String() string {
switch t {
case BlockM3TSZCompressed:
return "compressed_m3tsz"
case BlockDecompressed:
return "decompressed"
case BlockScalar:
return "scalar"
case BlockLazy:
return "lazy"
case BlockTime:
return "time"
case BlockContainer:
return "container"
case BlockEmpty:
return "empty"
case BlockMultiSeries:
return "multiseries"
case BlockConsolidated:
return "consolidated"
}

return "unknown"
}

type BlockInfo struct {
blockType BlockType
inner []BlockType
}

func NewBlockInfo(blockType BlockType) BlockInfo {
return BlockInfo{blockType: blockType}
}

func NewWrappedBlockInfo(
blockType BlockType,
wrap BlockInfo,
) BlockInfo {
inner := make([]BlockType, len(wrap.inner)+1)
copy(inner[:1], wrap.inner)
inner[0] = wrap.blockType
return BlockInfo{
blockType: blockType,
inner: inner,
}
}

func (b BlockInfo) Type() BlockType {
return b.blockType
}

func (b BlockInfo) InnerType() BlockType {
if b.inner == nil {
return b.Type()
}

return b.inner[0]
}

func (b BlockInfo) BaseType() BlockType {
if b.inner == nil {
return b.Type()
}

return b.inner[len(b.inner)-1]
}
4 changes: 4 additions & 0 deletions src/query/block/lazy.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,10 @@ func NewLazyBlock(block Block, opts LazyOptions) Block {
}
}

func (c *lazyBlock) Info() BlockInfo {
return NewWrappedBlockInfo(BlockLazy, c.block.Info())
}

func (b *lazyBlock) Close() error { return b.block.Close() }

func (b *lazyBlock) WithMetadata(
Expand Down
5 changes: 5 additions & 0 deletions src/query/block/lazy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,11 @@ func TestValidOffset(t *testing.T) {
offset := time.Minute
off := NewLazyBlock(b, testLazyOpts(offset, 1.0))

b.EXPECT().Info().Return(NewBlockInfo(BlockM3TSZCompressed))
info := off.Info()
assert.Equal(t, BlockLazy, info.Type())
assert.Equal(t, BlockM3TSZCompressed, info.BaseType())

// ensure functions are marshalled to the underlying block.
b.EXPECT().Close().Return(nil)
err := off.Close()
Expand Down
Loading

0 comments on commit f04a2e1

Please sign in to comment.