diff --git a/scripts/docker-integration-tests/aggregator_legacy/m3aggregator.yml b/scripts/docker-integration-tests/aggregator_legacy/m3aggregator.yml index 58d0370d4e..e70caa4b59 100644 --- a/scripts/docker-integration-tests/aggregator_legacy/m3aggregator.yml +++ b/scripts/docker-integration-tests/aggregator_legacy/m3aggregator.yml @@ -26,28 +26,7 @@ rawtcp: maxBackoff: 1s forever: true jitter: true - readBufferSize: 1440 - msgpackIterator: - ignoreHigherVersion: false - readerBufferSize: 1440 - largeFloatsSize: 1024 - largeFloatsPool: - buckets: - - count: 1024 - capacity: 2048 - - count: 512 - capacity: 4096 - - count: 256 - capacity: 8192 - - count: 128 - capacity: 16384 - - count: 64 - capacity: 32768 - - count: 32 - capacity: 65536 - watermark: - low: 0.001 - high: 0.002 + readBufferSize: 65536 protobufIterator: initBufferSize: 1440 maxMessageSize: 50000000 # max message size is 50MB diff --git a/src/aggregator/client/writer_test.go b/src/aggregator/client/writer_test.go index 1dcd471508..cf29b4c633 100644 --- a/src/aggregator/client/writer_test.go +++ b/src/aggregator/client/writer_test.go @@ -33,7 +33,6 @@ import ( "time" "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/migration" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric" @@ -974,7 +973,7 @@ func testWriterConcurrentWriteStress( ) for i := 0; i < len(results); i++ { buf := bytes.NewBuffer(results[i]) - iter := migration.NewUnaggregatedIterator(buf, nil, protobuf.NewUnaggregatedOptions()) + iter := protobuf.NewUnaggregatedIterator(buf, protobuf.NewUnaggregatedOptions()) for iter.Next() { msgResult := iter.Current() switch msgResult.Type { diff --git a/src/aggregator/config/m3aggregator.yml b/src/aggregator/config/m3aggregator.yml index 9cdd5eb0e2..366581d4e2 100644 --- a/src/aggregator/config/m3aggregator.yml +++ b/src/aggregator/config/m3aggregator.yml @@ -28,28 +28,7 @@ rawtcp: maxBackoff: 1s forever: true jitter: true - readBufferSize: 1440 - msgpackIterator: - ignoreHigherVersion: false - readerBufferSize: 1440 - largeFloatsSize: 1024 - largeFloatsPool: - buckets: - - count: 1024 - capacity: 2048 - - count: 512 - capacity: 4096 - - count: 256 - capacity: 8192 - - count: 128 - capacity: 16384 - - count: 64 - capacity: 32768 - - count: 32 - capacity: 65536 - watermark: - low: 0.001 - high: 0.002 + readBufferSize: 65536 protobufIterator: initBufferSize: 1440 maxMessageSize: 50000000 # max message size is 50MB diff --git a/src/aggregator/integration/client.go b/src/aggregator/integration/client.go index b83394a813..c5e88f3c21 100644 --- a/src/aggregator/integration/client.go +++ b/src/aggregator/integration/client.go @@ -26,7 +26,6 @@ import ( "time" "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/msgpack" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric" @@ -40,7 +39,6 @@ type client struct { address string batchSize int connectTimeout time.Duration - msgpackEncoder msgpack.UnaggregatedEncoder protobufEncoder protobuf.UnaggregatedEncoder conn net.Conn } @@ -54,7 +52,6 @@ func newClient( address: address, batchSize: batchSize, connectTimeout: connectTimeout, - msgpackEncoder: msgpack.NewUnaggregatedEncoder(msgpack.NewPooledBufferedEncoder(nil)), protobufEncoder: protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()), } } @@ -77,55 +74,6 @@ func (c *client) testConnection() bool { return true } -func (c *client) writeUntimedMetricWithPoliciesList( - mu unaggregated.MetricUnion, - pl policy.PoliciesList, -) error { - encoder := c.msgpackEncoder.Encoder() - sizeBefore := encoder.Buffer().Len() - var err error - switch mu.Type { - case metric.CounterType: - err = c.msgpackEncoder.EncodeCounterWithPoliciesList(unaggregated.CounterWithPoliciesList{ - Counter: mu.Counter(), - PoliciesList: pl, - }) - case metric.TimerType: - err = c.msgpackEncoder.EncodeBatchTimerWithPoliciesList(unaggregated.BatchTimerWithPoliciesList{ - BatchTimer: mu.BatchTimer(), - PoliciesList: pl, - }) - case metric.GaugeType: - err = c.msgpackEncoder.EncodeGaugeWithPoliciesList(unaggregated.GaugeWithPoliciesList{ - Gauge: mu.Gauge(), - PoliciesList: pl, - }) - default: - err = fmt.Errorf("unrecognized metric type %v", mu.Type) - } - if err != nil { - encoder.Buffer().Truncate(sizeBefore) - c.msgpackEncoder.Reset(encoder) - return err - } - sizeAfter := encoder.Buffer().Len() - // If the buffer size is not big enough, do nothing. - if sizeAfter < c.batchSize { - return nil - } - // Otherwise we get a new buffer and copy the bytes exceeding the max - // flush size to it, swap the new buffer with the old one, and flush out - // the old buffer. - encoder2 := msgpack.NewPooledBufferedEncoder(nil) - data := encoder.Bytes() - encoder2.Buffer().Write(data[sizeBefore:sizeAfter]) - c.msgpackEncoder.Reset(encoder2) - encoder.Buffer().Truncate(sizeBefore) - _, err = c.conn.Write(encoder.Bytes()) - encoder.Close() - return err -} - func (c *client) writeUntimedMetricWithMetadatas( mu unaggregated.MetricUnion, sm metadata.StagedMetadatas, @@ -227,24 +175,6 @@ func (c *client) writeUnaggregatedMessage( } func (c *client) flush() error { - if err := c.flushMsgpackEncoder(); err != nil { - return err - } - return c.flushProtobufEncoder() -} - -func (c *client) flushMsgpackEncoder() error { - encoder := c.msgpackEncoder.Encoder() - if len(encoder.Bytes()) == 0 { - return nil - } - c.msgpackEncoder.Reset(msgpack.NewPooledBufferedEncoder(nil)) - _, err := c.conn.Write(encoder.Bytes()) - encoder.Close() - return err -} - -func (c *client) flushProtobufEncoder() error { encoder := c.protobufEncoder if encoder.Len() == 0 { return nil diff --git a/src/aggregator/integration/custom_aggregations_test.go b/src/aggregator/integration/custom_aggregations_test.go index e87cd6fcca..dbddd43982 100644 --- a/src/aggregator/integration/custom_aggregations_test.go +++ b/src/aggregator/integration/custom_aggregations_test.go @@ -36,36 +36,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestCustomAggregationWithPoliciesList(t *testing.T) { - metadataFns := [4]metadataFn{ - func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - }, - func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesListWithCustomAggregation1, - } - }, - func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesListWithCustomAggregation2, - } - }, - func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - }, - } - testCustomAggregations(t, metadataFns) -} - func TestCustomAggregationWithStagedMetadatas(t *testing.T) { metadataFns := [4]metadataFn{ func(int) metadataUnion { @@ -212,11 +182,7 @@ func testCustomAggregations(t *testing.T, metadataFns [4]metadataFn) { for _, data := range dataset { setNowFn(data.timestamp) for _, mm := range data.metricWithMetadatas { - if mm.metadata.mType == policiesListType { - require.NoError(t, client.writeUntimedMetricWithPoliciesList(mm.metric.untimed, mm.metadata.policiesList)) - } else { - require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) - } + require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) } require.NoError(t, client.flush()) diff --git a/src/aggregator/integration/integration_data.go b/src/aggregator/integration/integration_data.go index 220f8119e4..9db4ae1cf7 100644 --- a/src/aggregator/integration/integration_data.go +++ b/src/aggregator/integration/integration_data.go @@ -781,8 +781,7 @@ func (mu metricUnion) ID() metricid.RawID { type metadataType int const ( - policiesListType metadataType = iota - stagedMetadatasType + stagedMetadatasType metadataType = iota forwardMetadataType timedMetadataType passthroughMetadataType @@ -792,7 +791,6 @@ type metadataFn func(idx int) metadataUnion type metadataUnion struct { mType metadataType - policiesList policy.PoliciesList stagedMetadatas metadata.StagedMetadatas forwardMetadata metadata.ForwardMetadata timedMetadata metadata.TimedMetadata @@ -804,8 +802,6 @@ func (mu metadataUnion) expectedAggregationKeys( defaultStoragePolicies []policy.StoragePolicy, ) (aggregationKeys, error) { switch mu.mType { - case policiesListType: - return computeExpectedAggregationKeysFromPoliciesList(now, mu.policiesList, defaultStoragePolicies) case stagedMetadatasType: return computeExpectedAggregationKeysFromStagedMetadatas(now, mu.stagedMetadatas, defaultStoragePolicies) case forwardMetadataType: diff --git a/src/aggregator/integration/metadata_change_test.go b/src/aggregator/integration/metadata_change_test.go index 2eaef2fa51..36ef853f62 100644 --- a/src/aggregator/integration/metadata_change_test.go +++ b/src/aggregator/integration/metadata_change_test.go @@ -35,22 +35,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestMetadataChangeWithPoliciesList(t *testing.T) { - oldMetadataFn := func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - } - newMetadataFn := func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testUpdatedPoliciesList, - } - } - testMetadataChange(t, oldMetadataFn, newMetadataFn) -} - func TestMetadataChangeWithStagedMetadatas(t *testing.T) { oldMetadataFn := func(int) metadataUnion { return metadataUnion{ @@ -156,11 +140,7 @@ func testMetadataChange(t *testing.T, oldMetadataFn, newMetadataFn metadataFn) { for _, data := range dataset { setNowFn(data.timestamp) for _, mm := range data.metricWithMetadatas { - if mm.metadata.mType == policiesListType { - require.NoError(t, client.writeUntimedMetricWithPoliciesList(mm.metric.untimed, mm.metadata.policiesList)) - } else { - require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) - } + require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) } require.NoError(t, client.flush()) diff --git a/src/aggregator/integration/multi_client_one_type_test.go b/src/aggregator/integration/multi_client_one_type_test.go index a5cc457b79..5314bd87d5 100644 --- a/src/aggregator/integration/multi_client_one_type_test.go +++ b/src/aggregator/integration/multi_client_one_type_test.go @@ -35,16 +35,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestMultiClientOneTypeWithPoliciesList(t *testing.T) { - metadataFn := func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - } - testMultiClientOneType(t, metadataFn) -} - func TestMultiClientOneTypeWithStagedMetadatas(t *testing.T) { metadataFn := func(int) metadataUnion { return metadataUnion{ @@ -137,11 +127,7 @@ func testMultiClientOneType(t *testing.T, metadataFn metadataFn) { for _, mm := range data.metricWithMetadatas { // Randomly pick one client to write the metric. client := clients[rand.Int63n(int64(numClients))] - if mm.metadata.mType == policiesListType { - require.NoError(t, client.writeUntimedMetricWithPoliciesList(mm.metric.untimed, mm.metadata.policiesList)) - } else { - require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) - } + require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) } for _, client := range clients { require.NoError(t, client.flush()) diff --git a/src/aggregator/integration/one_client_multi_type_untimed_test.go b/src/aggregator/integration/one_client_multi_type_untimed_test.go index 5efc634637..75e51c3db6 100644 --- a/src/aggregator/integration/one_client_multi_type_untimed_test.go +++ b/src/aggregator/integration/one_client_multi_type_untimed_test.go @@ -33,16 +33,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestOneClientMultiTypeUntimedMetricsWithPoliciesList(t *testing.T) { - metadataFn := func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - } - testOneClientMultiType(t, metadataFn) -} - func TestOneClientMultiTypeUntimedMetricsWithStagedMetadatas(t *testing.T) { metadataFn := func(int) metadataUnion { return metadataUnion{ @@ -127,11 +117,7 @@ func testOneClientMultiType(t *testing.T, metadataFn metadataFn) { for _, data := range dataset { setNowFn(data.timestamp) for _, mm := range data.metricWithMetadatas { - if mm.metadata.mType == policiesListType { - require.NoError(t, client.writeUntimedMetricWithPoliciesList(mm.metric.untimed, mm.metadata.policiesList)) - } else { - require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) - } + require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) } require.NoError(t, client.flush()) diff --git a/src/aggregator/integration/same_id_multi_type_test.go b/src/aggregator/integration/same_id_multi_type_test.go index 23bb4f0d9f..394038befd 100644 --- a/src/aggregator/integration/same_id_multi_type_test.go +++ b/src/aggregator/integration/same_id_multi_type_test.go @@ -34,16 +34,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestSameIDMultiTypeWithPoliciesList(t *testing.T) { - metadataFn := func(int) metadataUnion { - return metadataUnion{ - mType: policiesListType, - policiesList: testPoliciesList, - } - } - testSameIDMultiType(t, metadataFn) -} - func TestSameIDMultiTypeWithStagedMetadatas(t *testing.T) { metadataFn := func(int) metadataUnion { return metadataUnion{ @@ -151,11 +141,7 @@ func testSameIDMultiType(t *testing.T, metadataFn metadataFn) { for _, data := range dataset { setNowFn(data.timestamp) for _, mm := range data.metricWithMetadatas { - if mm.metadata.mType == policiesListType { - require.NoError(t, client.writeUntimedMetricWithPoliciesList(mm.metric.untimed, mm.metadata.policiesList)) - } else { - require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) - } + require.NoError(t, client.writeUntimedMetricWithMetadatas(mm.metric.untimed, mm.metadata.stagedMetadatas)) } require.NoError(t, client.flush()) diff --git a/src/aggregator/server/rawtcp/options.go b/src/aggregator/server/rawtcp/options.go index 314c6c7a82..0baec13d1c 100644 --- a/src/aggregator/server/rawtcp/options.go +++ b/src/aggregator/server/rawtcp/options.go @@ -21,7 +21,6 @@ package rawtcp import ( - "github.com/m3db/m3/src/metrics/encoding/msgpack" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/x/clock" "github.com/m3db/m3/src/x/instrument" @@ -34,7 +33,7 @@ const ( defaultErrorLogLimitPerSecond = 0 // The default read buffer size for raw TCP connections. - defaultReadBufferSize = 1440 + defaultReadBufferSize = 65536 ) // Options provide a set of server options. @@ -57,12 +56,6 @@ type Options interface { // ServerOptiosn returns the server options. ServerOptions() server.Options - // SetMsgpackUnaggregatedIteratorOptions sets the msgpack unaggregated iterator options. - SetMsgpackUnaggregatedIteratorOptions(value msgpack.UnaggregatedIteratorOptions) Options - - // MsgpackUnaggregatedIteratorOptions returns the msgpack unaggregated iterator options. - MsgpackUnaggregatedIteratorOptions() msgpack.UnaggregatedIteratorOptions - // SetProtobufUnaggregatedIteratorOptions sets the protobuf unaggregated iterator options. SetProtobufUnaggregatedIteratorOptions(value protobuf.UnaggregatedOptions) Options @@ -92,7 +85,6 @@ type options struct { clockOpts clock.Options instrumentOpts instrument.Options serverOpts server.Options - msgpackItOpts msgpack.UnaggregatedIteratorOptions protobufItOpts protobuf.UnaggregatedOptions readBufferSize int errLogLimitPerSecond int64 @@ -105,7 +97,6 @@ func NewOptions() Options { clockOpts: clock.NewOptions(), instrumentOpts: instrument.NewOptions(), serverOpts: server.NewOptions(), - msgpackItOpts: msgpack.NewUnaggregatedIteratorOptions(), protobufItOpts: protobuf.NewUnaggregatedOptions(), readBufferSize: defaultReadBufferSize, errLogLimitPerSecond: defaultErrorLogLimitPerSecond, @@ -143,16 +134,6 @@ func (o *options) ServerOptions() server.Options { return o.serverOpts } -func (o *options) SetMsgpackUnaggregatedIteratorOptions(value msgpack.UnaggregatedIteratorOptions) Options { - opts := *o - opts.msgpackItOpts = value - return &opts -} - -func (o *options) MsgpackUnaggregatedIteratorOptions() msgpack.UnaggregatedIteratorOptions { - return o.msgpackItOpts -} - func (o *options) SetProtobufUnaggregatedIteratorOptions(value protobuf.UnaggregatedOptions) Options { opts := *o opts.protobufItOpts = value diff --git a/src/aggregator/server/rawtcp/server.go b/src/aggregator/server/rawtcp/server.go index 22c2be6aa0..506ed33057 100644 --- a/src/aggregator/server/rawtcp/server.go +++ b/src/aggregator/server/rawtcp/server.go @@ -24,7 +24,6 @@ import ( "bufio" "fmt" "io" - "math/rand" "net" "sync" "time" @@ -32,8 +31,6 @@ import ( "github.com/m3db/m3/src/aggregator/aggregator" "github.com/m3db/m3/src/aggregator/rate" "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/migration" - "github.com/m3db/m3/src/metrics/encoding/msgpack" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric/aggregated" @@ -88,11 +85,9 @@ type handler struct { aggregator aggregator.Aggregator log *zap.Logger readBufferSize int - msgpackItOpts msgpack.UnaggregatedIteratorOptions protobufItOpts protobuf.UnaggregatedOptions errLogRateLimiter *rate.Limiter - rand *rand.Rand metrics handlerMetrics opts Options @@ -110,10 +105,8 @@ func NewHandler(aggregator aggregator.Aggregator, opts Options) xserver.Handler aggregator: aggregator, log: iOpts.Logger(), readBufferSize: opts.ReadBufferSize(), - msgpackItOpts: opts.MsgpackUnaggregatedIteratorOptions(), protobufItOpts: opts.ProtobufUnaggregatedIteratorOptions(), errLogRateLimiter: limiter, - rand: rand.New(rand.NewSource(nowFn().UnixNano())), metrics: newHandlerMetrics(iOpts.MetricsScope()), opts: opts, } @@ -128,7 +121,7 @@ func (s *handler) Handle(conn net.Conn) { rOpts := xio.ResettableReaderOptions{ReadBufferSize: s.readBufferSize} read := s.opts.RWOptions().ResettableReaderFn()(conn, rOpts) reader := bufio.NewReaderSize(read, s.readBufferSize) - it := migration.NewUnaggregatedIterator(reader, s.msgpackItOpts, s.protobufItOpts) + it := protobuf.NewUnaggregatedIterator(reader, s.protobufItOpts) defer it.Close() // Iterate over the incoming metrics stream and queue up metrics. diff --git a/src/aggregator/server/rawtcp/server_test.go b/src/aggregator/server/rawtcp/server_test.go index 4e16f2c365..c99a8ada68 100644 --- a/src/aggregator/server/rawtcp/server_test.go +++ b/src/aggregator/server/rawtcp/server_test.go @@ -29,7 +29,6 @@ import ( "github.com/m3db/m3/src/aggregator/aggregator/capture" "github.com/m3db/m3/src/metrics/aggregation" "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/msgpack" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric" @@ -141,11 +140,7 @@ var ( SourceID: 1234, NumForwardedTimes: 3, } - testPassthroughStoragePolicy = policy.NewStoragePolicy(time.Minute, xtime.Minute, 12*time.Hour) - testCounterWithPoliciesList = unaggregated.CounterWithPoliciesList{ - Counter: testCounter.Counter(), - PoliciesList: testDefaultPoliciesList, - } + testPassthroughStoragePolicy = policy.NewStoragePolicy(time.Minute, xtime.Minute, 12*time.Hour) testBatchTimerWithPoliciesList = unaggregated.BatchTimerWithPoliciesList{ BatchTimer: testBatchTimer.BatchTimer(), PoliciesList: testCustomPoliciesList, @@ -184,27 +179,7 @@ var ( } ) -func TestRawTCPServerHandleUnaggregatedMsgpackEncoding(t *testing.T) { - testRawTCPServerHandleUnaggregated(t, func(int) encodingProtocol { return msgpackEncoding }) -} - func TestRawTCPServerHandleUnaggregatedProtobufEncoding(t *testing.T) { - testRawTCPServerHandleUnaggregated(t, func(int) encodingProtocol { return protobufEncoding }) -} - -func TestRawTCPServerHandleUnaggregatedMixedEncoding(t *testing.T) { - testRawTCPServerHandleUnaggregated(t, func(workerID int) encodingProtocol { - if workerID%2 == 0 { - return msgpackEncoding - } - return protobufEncoding - }) -} - -func testRawTCPServerHandleUnaggregated( - t *testing.T, - protocolSelector func(int) encodingProtocol, -) { agg := capture.NewAggregator() h := NewHandler(agg, testServerOptions()) @@ -224,23 +199,16 @@ func testRawTCPServerHandleUnaggregated( // Now establish multiple connections and send data to the server. var expectedTotalMetrics int for i := 0; i < numClients; i++ { - i := i wgClient.Add(1) // Add test metrics to expected result. expectedResult.CountersWithMetadatas = append(expectedResult.CountersWithMetadatas, testCounterWithMetadatas) expectedResult.BatchTimersWithMetadatas = append(expectedResult.BatchTimersWithMetadatas, testBatchTimerWithMetadatas) expectedResult.GaugesWithMetadatas = append(expectedResult.GaugesWithMetadatas, testGaugeWithMetadatas) - - protocol := protocolSelector(i) - if protocol == protobufEncoding { - expectedResult.TimedMetricWithMetadata = append(expectedResult.TimedMetricWithMetadata, testTimedMetricWithMetadata) - expectedResult.PassthroughMetricWithMetadata = append(expectedResult.PassthroughMetricWithMetadata, testPassthroughMetricWithMetadata) - expectedResult.ForwardedMetricsWithMetadata = append(expectedResult.ForwardedMetricsWithMetadata, testForwardedMetricWithMetadata) - expectedTotalMetrics += 5 - } else { - expectedTotalMetrics += 3 - } + expectedResult.TimedMetricWithMetadata = append(expectedResult.TimedMetricWithMetadata, testTimedMetricWithMetadata) + expectedResult.PassthroughMetricWithMetadata = append(expectedResult.PassthroughMetricWithMetadata, testPassthroughMetricWithMetadata) + expectedResult.ForwardedMetricsWithMetadata = append(expectedResult.ForwardedMetricsWithMetadata, testForwardedMetricWithMetadata) + expectedTotalMetrics += 5 go func() { defer wgClient.Done() @@ -248,44 +216,33 @@ func testRawTCPServerHandleUnaggregated( conn, err := net.Dial("tcp", listener.Addr().String()) require.NoError(t, err) - var stream []byte - switch protocol { - case msgpackEncoding: - encoder := msgpack.NewUnaggregatedEncoder(msgpack.NewPooledBufferedEncoder(nil)) - require.NoError(t, encoder.EncodeCounterWithPoliciesList(testCounterWithPoliciesList)) - require.NoError(t, encoder.EncodeBatchTimerWithPoliciesList(testBatchTimerWithPoliciesList)) - require.NoError(t, encoder.EncodeGaugeWithPoliciesList(testGaugeWithPoliciesList)) - stream = encoder.Encoder().Bytes() - case protobufEncoding: - encoder := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: testCounterWithMetadatas, - })) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: testBatchTimerWithMetadatas, - })) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: testGaugeWithMetadatas, - })) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.TimedMetricWithMetadataType, - TimedMetricWithMetadata: testTimedMetricWithMetadata, - })) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.PassthroughMetricWithMetadataType, - PassthroughMetricWithMetadata: testPassthroughMetricWithMetadata, - })) - require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.ForwardedMetricWithMetadataType, - ForwardedMetricWithMetadata: testForwardedMetricWithMetadata, - })) - buf := encoder.Relinquish() - stream = buf.Bytes() - } - _, err = conn.Write(stream) + encoder := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.CounterWithMetadatasType, + CounterWithMetadatas: testCounterWithMetadatas, + })) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.BatchTimerWithMetadatasType, + BatchTimerWithMetadatas: testBatchTimerWithMetadatas, + })) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.GaugeWithMetadatasType, + GaugeWithMetadatas: testGaugeWithMetadatas, + })) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.TimedMetricWithMetadataType, + TimedMetricWithMetadata: testTimedMetricWithMetadata, + })) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.PassthroughMetricWithMetadataType, + PassthroughMetricWithMetadata: testPassthroughMetricWithMetadata, + })) + require.NoError(t, encoder.EncodeMessage(encoding.UnaggregatedMessageUnion{ + Type: encoding.ForwardedMetricWithMetadataType, + ForwardedMetricWithMetadata: testForwardedMetricWithMetadata, + })) + + _, err = conn.Write(encoder.Relinquish().Bytes()) require.NoError(t, err) }() } @@ -310,10 +267,3 @@ func testServerOptions() Options { serverOpts := xserver.NewOptions().SetRetryOptions(retry.NewOptions().SetMaxRetries(2)) return opts.SetInstrumentOptions(instrumentOpts).SetServerOptions(serverOpts) } - -type encodingProtocol int - -const ( - msgpackEncoding encodingProtocol = iota - protobufEncoding -) diff --git a/src/cmd/services/m3aggregator/config/server.go b/src/cmd/services/m3aggregator/config/server.go index d979c749f5..e063c6c549 100644 --- a/src/cmd/services/m3aggregator/config/server.go +++ b/src/cmd/services/m3aggregator/config/server.go @@ -26,7 +26,6 @@ import ( "github.com/m3db/m3/src/aggregator/server/http" "github.com/m3db/m3/src/aggregator/server/m3msg" "github.com/m3db/m3/src/aggregator/server/rawtcp" - "github.com/m3db/m3/src/metrics/encoding/msgpack" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/msg/consumer" "github.com/m3db/m3/src/x/instrument" @@ -78,9 +77,6 @@ type RawTCPServerConfiguration struct { // Read buffer size. ReadBufferSize *int `yaml:"readBufferSize"` - // Msgpack iterator configuration. - MsgpackIterator msgpackUnaggregatedIteratorConfiguration `yaml:"msgpackIterator"` - // Protobuf iterator configuration. ProtobufIterator protobufUnaggregatedIteratorConfiguration `yaml:"protobufIterator"` } @@ -103,10 +99,6 @@ func (c *RawTCPServerConfiguration) NewServerOptions( } opts = opts.SetServerOptions(serverOpts) - // Set msgpack iterator options. - msgpackItOpts := c.MsgpackIterator.NewOptions(instrumentOpts) - opts = opts.SetMsgpackUnaggregatedIteratorOptions(msgpackItOpts) - // Set protobuf iterator options. protobufItOpts := c.ProtobufIterator.NewOptions(instrumentOpts) opts = opts.SetProtobufUnaggregatedIteratorOptions(protobufItOpts) @@ -120,48 +112,6 @@ func (c *RawTCPServerConfiguration) NewServerOptions( return opts } -// msgpackUnaggregatedIteratorConfiguration contains configuration for msgpack unaggregated iterator. -type msgpackUnaggregatedIteratorConfiguration struct { - // Whether to ignore encoded data streams whose version is higher than the current known version. - IgnoreHigherVersion *bool `yaml:"ignoreHigherVersion"` - - // Reader buffer size. - ReaderBufferSize *int `yaml:"readerBufferSize"` - - // Whether a float slice is considered a "large" slice and therefore resort to - // the large floats pool for allocating that slice. - LargeFloatsSize *int `yaml:"largeFloatsSize"` - - // Pool of large float slices. - LargeFloatsPool pool.BucketizedPoolConfiguration `yaml:"largeFloatsPool"` -} - -func (c *msgpackUnaggregatedIteratorConfiguration) NewOptions( - instrumentOpts instrument.Options, -) msgpack.UnaggregatedIteratorOptions { - scope := instrumentOpts.MetricsScope() - opts := msgpack.NewUnaggregatedIteratorOptions() - if c.IgnoreHigherVersion != nil { - opts = opts.SetIgnoreHigherVersion(*c.IgnoreHigherVersion) - } - if c.ReaderBufferSize != nil { - opts = opts.SetReaderBufferSize(*c.ReaderBufferSize) - } - if c.LargeFloatsSize != nil { - opts = opts.SetLargeFloatsSize(*c.LargeFloatsSize) - } - - // NB(xichen): intentionally not using the same floats pool used for computing - // timer quantiles to accommodate different usage patterns and reduce contention. - iOpts := instrumentOpts.SetMetricsScope(scope.SubScope("large-floats-pool")) - largeFloatsPoolOpts := c.LargeFloatsPool.NewObjectPoolOptions(iOpts) - largeFloatsPool := pool.NewFloatsPool(c.LargeFloatsPool.NewBuckets(), largeFloatsPoolOpts) - opts = opts.SetLargeFloatsPool(largeFloatsPool) - largeFloatsPool.Init() - - return opts -} - // protobufUnaggregatedIteratorConfiguration contains configuration for protobuf unaggregated iterator. type protobufUnaggregatedIteratorConfiguration struct { // Initial buffer size. diff --git a/src/collector/integration/server/server.go b/src/collector/integration/server/server.go index 3738fcf651..1da99d82c3 100644 --- a/src/collector/integration/server/server.go +++ b/src/collector/integration/server/server.go @@ -26,7 +26,6 @@ import ( "net" "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/migration" "github.com/m3db/m3/src/metrics/encoding/protobuf" "github.com/m3db/m3/src/metrics/metadata" "github.com/m3db/m3/src/metrics/metric/unaggregated" @@ -59,7 +58,7 @@ func newHandler(opts HandlerOptions) *handler { func (h *handler) Handle(conn net.Conn) { reader := bufio.NewReaderSize(conn, h.readBufferSize) - it := migration.NewUnaggregatedIterator(reader, nil, h.itOpts) + it := protobuf.NewUnaggregatedIterator(reader, h.itOpts) defer it.Close() for it.Next() { diff --git a/src/metrics/encoding/migration/convert.go b/src/metrics/encoding/migration/convert.go deleted file mode 100644 index a7b14c3447..0000000000 --- a/src/metrics/encoding/migration/convert.go +++ /dev/null @@ -1,131 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package migration - -import ( - "fmt" - - "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/metadata" - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" -) - -func toUnaggregatedMessageUnion( - metricUnion unaggregated.MetricUnion, - policiesList policy.PoliciesList, -) (encoding.UnaggregatedMessageUnion, error) { - metadatas := ToStagedMetadatas(policiesList) - switch metricUnion.Type { - case metric.CounterType: - return encoding.UnaggregatedMessageUnion{ - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: unaggregated.CounterWithMetadatas{ - Counter: metricUnion.Counter(), - StagedMetadatas: metadatas, - }, - }, nil - case metric.TimerType: - return encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: unaggregated.BatchTimerWithMetadatas{ - BatchTimer: metricUnion.BatchTimer(), - StagedMetadatas: metadatas, - }, - }, nil - case metric.GaugeType: - return encoding.UnaggregatedMessageUnion{ - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: unaggregated.GaugeWithMetadatas{ - Gauge: metricUnion.Gauge(), - StagedMetadatas: metadatas, - }, - }, nil - default: - return encoding.UnaggregatedMessageUnion{}, fmt.Errorf("unknown metric type: %v", metricUnion.Type) - } -} - -// ToStagedMetadatas converts a list of policies to staged metadatas. -// TODO: look into reuse metadatas during conversion. -func ToStagedMetadatas( - policiesList policy.PoliciesList, -) metadata.StagedMetadatas { - numStagedPolicies := len(policiesList) - res := make(metadata.StagedMetadatas, 0, numStagedPolicies) - for _, sp := range policiesList { - sm := metadata.StagedMetadata{} - sm.CutoverNanos = sp.CutoverNanos - sm.Tombstoned = sp.Tombstoned - policies, isDefault := sp.Policies() - if isDefault { - sm.Metadata = metadata.DefaultMetadata - } else { - sm.Metadata = toMetadata(policies) - } - res = append(res, sm) - } - return res -} - -// TODO: look into reuse metadata during conversion. -func toMetadata(policies []policy.Policy) metadata.Metadata { - res := metadata.Metadata{} - for _, p := range policies { - // Find if there is an existing pipeline in the set of metadatas - // with the same aggregation ID. - pipelineIdx := -1 - for i := 0; i < len(res.Pipelines); i++ { - if p.AggregationID == res.Pipelines[i].AggregationID { - pipelineIdx = i - break - } - } - // If there is no existing pipeline with the same aggregation ID, - // create a new pipeline with the aggregation ID. - if pipelineIdx == -1 { - res.Pipelines = append(res.Pipelines, metadata.PipelineMetadata{ - AggregationID: p.AggregationID, - }) - pipelineIdx = len(res.Pipelines) - 1 - } - - // Find if the storage policy already exists in the set of storage - // policies in the corresponding pipeline. - pipelines := res.Pipelines - policyIdx := -1 - for i := 0; i < len(pipelines[pipelineIdx].StoragePolicies); i++ { - if pipelines[pipelineIdx].StoragePolicies[i] == p.StoragePolicy { - policyIdx = i - break - } - } - // If the policy already exists in the pipeline, nothing to do. - if policyIdx != -1 { - continue - } - // Otherwise we append the policy to the end. This maintains the original - // ordering (if any) of the policies within each pipeline. - pipelines[pipelineIdx].StoragePolicies = append(pipelines[pipelineIdx].StoragePolicies, p.StoragePolicy) - } - return res -} diff --git a/src/metrics/encoding/migration/convert_test.go b/src/metrics/encoding/migration/convert_test.go deleted file mode 100644 index 3b017c16ec..0000000000 --- a/src/metrics/encoding/migration/convert_test.go +++ /dev/null @@ -1,235 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package migration - -import ( - "strings" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/metadata" - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -var ( - testConvertCounterUnion = unaggregated.MetricUnion{ - Type: metric.CounterType, - ID: []byte("testConvertCounter"), - CounterVal: 1234, - } - testConvertBatchTimerUnion = unaggregated.MetricUnion{ - Type: metric.TimerType, - ID: []byte("testConvertBatchTimer"), - BatchTimerVal: []float64{222.22, 345.67, 901.23345}, - } - testConvertGaugeUnion = unaggregated.MetricUnion{ - Type: metric.GaugeType, - ID: []byte("testConvertGauge"), - GaugeVal: 123.456, - } - testConvertPoliciesList = policy.PoliciesList{ - // Default staged policies. - policy.DefaultStagedPolicies, - - // Staged policies with default policies lists. - policy.NewStagedPolicies( - 123, - false, - nil, - ), - - // Single pipeline. - policy.NewStagedPolicies( - 1234, - false, - []policy.Policy{ - policy.NewPolicy( - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - aggregation.DefaultID, - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - aggregation.DefaultID, - ), - // A duplicate policy. - policy.NewPolicy( - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - aggregation.DefaultID, - ), - }, - ), - - // Multiple pipelines. - policy.NewStagedPolicies( - 5678, - true, - []policy.Policy{ - policy.NewPolicy( - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - aggregation.DefaultID, - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - ), - policy.NewPolicy( - policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 24*time.Hour), - aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Hour, xtime.Hour, 30*24*time.Hour), - aggregation.MustCompressTypes(aggregation.Sum), - ), - }, - ), - } - testConvertCounter = unaggregated.Counter{ - ID: []byte("testConvertCounter"), - Value: 1234, - } - testConvertBatchTimer = unaggregated.BatchTimer{ - ID: []byte("testConvertBatchTimer"), - Values: []float64{222.22, 345.67, 901.23345}, - } - testConvertGauge = unaggregated.Gauge{ - ID: []byte("testConvertGauge"), - Value: 123.456, - } - testConvertStagedMetadatas = metadata.StagedMetadatas{ - metadata.DefaultStagedMetadata, - metadata.StagedMetadata{ - CutoverNanos: 123, - Tombstoned: false, - Metadata: metadata.DefaultMetadata, - }, - metadata.StagedMetadata{ - CutoverNanos: 1234, - Tombstoned: false, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - }, - }, - }, - }, - }, - metadata.StagedMetadata{ - CutoverNanos: 5678, - Tombstoned: true, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - }, - }, - { - AggregationID: aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 24*time.Hour), - }, - }, - { - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Hour, xtime.Hour, 30*24*time.Hour), - }, - }, - }, - }, - }, - } -) - -func TestToStagedMetadatas(t *testing.T) { - require.Equal(t, testConvertStagedMetadatas, ToStagedMetadatas(testConvertPoliciesList)) -} - -func TestToUnaggregatedMessageUnion(t *testing.T) { - inputs := []struct { - metricUnion unaggregated.MetricUnion - policiesList policy.PoliciesList - }{ - { - metricUnion: testConvertCounterUnion, - policiesList: testConvertPoliciesList, - }, - { - metricUnion: testConvertBatchTimerUnion, - policiesList: testConvertPoliciesList, - }, - { - metricUnion: testConvertGaugeUnion, - policiesList: testConvertPoliciesList, - }, - } - expected := []encoding.UnaggregatedMessageUnion{ - { - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: unaggregated.CounterWithMetadatas{ - Counter: testConvertCounter, - StagedMetadatas: testConvertStagedMetadatas, - }, - }, - { - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: unaggregated.BatchTimerWithMetadatas{ - BatchTimer: testConvertBatchTimer, - StagedMetadatas: testConvertStagedMetadatas, - }, - }, - { - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: unaggregated.GaugeWithMetadatas{ - Gauge: testConvertGauge, - StagedMetadatas: testConvertStagedMetadatas, - }, - }, - } - - for i, input := range inputs { - res, err := toUnaggregatedMessageUnion(input.metricUnion, input.policiesList) - require.NoError(t, err) - require.Equal(t, expected[i], res) - } -} - -func TestToUnaggregatedMessageUnionError(t *testing.T) { - invalidMetric := unaggregated.MetricUnion{ - Type: metric.UnknownType, - } - _, err := toUnaggregatedMessageUnion(invalidMetric, testConvertPoliciesList) - require.Error(t, err) - require.True(t, strings.Contains(err.Error(), "unknown metric type")) -} diff --git a/src/metrics/encoding/migration/unaggregated_iterator.go b/src/metrics/encoding/migration/unaggregated_iterator.go deleted file mode 100644 index e99d0107c9..0000000000 --- a/src/metrics/encoding/migration/unaggregated_iterator.go +++ /dev/null @@ -1,163 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package migration - -import ( - "fmt" - - "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/msgpack" - "github.com/m3db/m3/src/metrics/encoding/protobuf" -) - -// UnaggregatedIterator decodes unaggregated metrics. -type UnaggregatedIterator interface { - // Next returns true if there are more items to decode. - Next() bool - - // Current returns the current decoded value. - Current() encoding.UnaggregatedMessageUnion - - // Err returns the error encountered during decoding, if any. - Err() error - - // Close closes the iterator. - Close() -} - -type unaggregatedIterator struct { - reader encoding.ByteReadScanner - - msgpackIt msgpack.UnaggregatedIterator - protobufIt protobuf.UnaggregatedIterator - msg encoding.UnaggregatedMessageUnion - closed bool - err error -} - -// NewUnaggregatedIterator creates a new unaggregated iterator. -func NewUnaggregatedIterator( - reader encoding.ByteReadScanner, - msgpackItOpts msgpack.UnaggregatedIteratorOptions, - protobufItOpts protobuf.UnaggregatedOptions, -) UnaggregatedIterator { - msgpackIt := msgpack.NewUnaggregatedIterator(reader, msgpackItOpts) - protobufIt := protobuf.NewUnaggregatedIterator(reader, protobufItOpts) - return &unaggregatedIterator{ - reader: reader, - msgpackIt: msgpackIt, - protobufIt: protobufIt, - } -} - -func (it *unaggregatedIterator) Next() bool { - if it.closed || it.Err() != nil { - return false - } - protocol, err := it.decodeProtocolType() - if err != nil { - return false - } - switch protocol { - case msgpackProtocol: - if !it.msgpackIt.Next() { - return false - } - metric := it.msgpackIt.Metric() - policiesList := it.msgpackIt.PoliciesList() - msg, err := toUnaggregatedMessageUnion(metric, policiesList) - if err != nil { - it.err = err - return false - } - it.msg = msg - return true - case protobufProtocol: - if !it.protobufIt.Next() { - return false - } - it.msg = it.protobufIt.Current() - return true - default: - it.err = fmt.Errorf("unexpected protocol type: %v", protocol) - return false - } -} - -func (it *unaggregatedIterator) Current() encoding.UnaggregatedMessageUnion { return it.msg } - -func (it *unaggregatedIterator) Err() error { - if it.err != nil { - return it.err - } - if err := it.msgpackIt.Err(); err != nil { - return err - } - return it.protobufIt.Err() -} - -func (it *unaggregatedIterator) Close() { - if it.closed { - return - } - it.closed = true - it.msgpackIt.Close() - it.msgpackIt = nil - it.protobufIt.Close() - it.protobufIt = nil - it.msg = encoding.UnaggregatedMessageUnion{} - it.err = nil -} - -func (it *unaggregatedIterator) decodeProtocolType() (protocolType, error) { - // Peek the first byte of the message. - b, err := it.reader.ReadByte() - if err != nil { - it.err = err - return unknownProtocolType, err - } - - // Put the byte back. - if err := it.reader.UnreadByte(); err != nil { - it.err = err - return unknownProtocolType, err - } - - // NB: currently the msgpack encoder always encodes the version first as the first byte - // of the msgpack message, and since we are currently at version 1, the first byte will - // always be an odd number (i.e., 1). On the other hand, the protobuf encoder always - // encodes the message length as a varint, in which case the first byte will always - // be an even number. As such we can use the last bit of the byte to distinguish the - // different protocols. - protocol := msgpackProtocol - if (b & 1) == 0 { - protocol = protobufProtocol - } - return protocol, nil -} - -type protocolType int - -const ( - unknownProtocolType protocolType = iota - msgpackProtocol - protobufProtocol -) diff --git a/src/metrics/encoding/migration/unaggregated_iterator_test.go b/src/metrics/encoding/migration/unaggregated_iterator_test.go deleted file mode 100644 index c0328f7ec7..0000000000 --- a/src/metrics/encoding/migration/unaggregated_iterator_test.go +++ /dev/null @@ -1,841 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package migration - -import ( - "bytes" - "io" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/encoding" - "github.com/m3db/m3/src/metrics/encoding/msgpack" - "github.com/m3db/m3/src/metrics/encoding/protobuf" - "github.com/m3db/m3/src/metrics/metadata" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/pipeline" - "github.com/m3db/m3/src/metrics/pipeline/applied" - "github.com/m3db/m3/src/metrics/policy" - "github.com/m3db/m3/src/metrics/transformation" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/google/go-cmp/cmp" - "github.com/google/go-cmp/cmp/cmpopts" - "github.com/stretchr/testify/require" -) - -var ( - testCounter1 = unaggregated.Counter{ - ID: []byte("testCounter1"), - Value: 123, - } - testCounter2 = unaggregated.Counter{ - ID: []byte("testCounter2"), - Value: 456, - } - testBatchTimer1 = unaggregated.BatchTimer{ - ID: []byte("testBatchTimer1"), - Values: []float64{3.67, -9.38}, - } - testBatchTimer2 = unaggregated.BatchTimer{ - ID: []byte("testBatchTimer2"), - Values: []float64{4.57, 189234.01}, - } - testGauge1 = unaggregated.Gauge{ - ID: []byte("testGauge1"), - Value: 845.23, - } - testGauge2 = unaggregated.Gauge{ - ID: []byte("testGauge2"), - Value: 234231.345, - } - testPoliciesList1 = policy.DefaultPoliciesList - testPoliciesList2 = policy.PoliciesList{ - // Default staged policies. - policy.DefaultStagedPolicies, - - // Single pipeline, - policy.NewStagedPolicies( - 1234, - false, - []policy.Policy{ - policy.NewPolicy( - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - aggregation.DefaultID, - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - aggregation.DefaultID, - ), - }, - ), - - // Multiple pipelines. - policy.NewStagedPolicies( - 5678, - true, - []policy.Policy{ - policy.NewPolicy( - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - aggregation.DefaultID, - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - ), - policy.NewPolicy( - policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 24*time.Hour), - aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - ), - policy.NewPolicy( - policy.NewStoragePolicy(time.Hour, xtime.Hour, 30*24*time.Hour), - aggregation.MustCompressTypes(aggregation.Sum), - ), - }, - ), - } - testMetadatas1 = metadata.StagedMetadatas{ - { - CutoverNanos: 1234, - Tombstoned: false, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - }, - }, - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(10*time.Second, xtime.Second, time.Hour), - }, - Pipeline: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.RollupOpType, - Rollup: applied.RollupOp{ - ID: []byte("baz"), - AggregationID: aggregation.MustCompressTypes(aggregation.Mean), - }, - }, - }), - }, - }, - }, - }, - } - testMetadatas2 = metadata.StagedMetadatas{ - { - CutoverNanos: 4567, - Tombstoned: false, - }, - { - CutoverNanos: 7890, - Tombstoned: true, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.MustCompressTypes(aggregation.Count), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - }, - }, - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - policy.NewStoragePolicy(time.Hour, xtime.Hour, 30*24*time.Hour), - }, - Pipeline: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{ - Type: transformation.Absolute, - }, - }, - { - Type: pipeline.RollupOpType, - Rollup: applied.RollupOp{ - ID: []byte("foo"), - AggregationID: aggregation.MustCompressTypes(aggregation.Last, aggregation.Sum), - }, - }, - }), - }, - }, - }, - }, - { - CutoverNanos: 32768, - Tombstoned: false, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.DefaultID, - Pipeline: applied.NewPipeline([]applied.OpUnion{ - { - Type: pipeline.TransformationOpType, - Transformation: pipeline.TransformationOp{ - Type: transformation.PerSecond, - }, - }, - { - Type: pipeline.RollupOpType, - Rollup: applied.RollupOp{ - ID: []byte("bar"), - AggregationID: aggregation.MustCompressTypes(aggregation.P99), - }, - }, - }), - }, - }, - }, - }, - } - testConvertedMetadatas1 = metadata.DefaultStagedMetadatas - testConvertedMetadatas2 = metadata.StagedMetadatas{ - metadata.DefaultStagedMetadata, - metadata.StagedMetadata{ - CutoverNanos: 1234, - Tombstoned: false, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - }, - }, - }, - }, - }, - metadata.StagedMetadata{ - CutoverNanos: 5678, - Tombstoned: true, - Metadata: metadata.Metadata{ - Pipelines: []metadata.PipelineMetadata{ - { - AggregationID: aggregation.DefaultID, - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), - }, - }, - { - AggregationID: aggregation.MustCompressTypes(aggregation.Count, aggregation.Last), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Minute, xtime.Minute, 6*time.Hour), - policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 24*time.Hour), - }, - }, - { - AggregationID: aggregation.MustCompressTypes(aggregation.Sum), - StoragePolicies: []policy.StoragePolicy{ - policy.NewStoragePolicy(time.Hour, xtime.Hour, 30*24*time.Hour), - }, - }, - }, - }, - }, - } - testCmpOpts = []cmp.Option{ - cmpopts.EquateEmpty(), - cmp.AllowUnexported(policy.StoragePolicy{}), - } -) - -func TestUnaggregatedIteratorDecodeCounterWithPoliciesList(t *testing.T) { - inputs := []unaggregated.CounterWithPoliciesList{ - { - Counter: testCounter1, - PoliciesList: testPoliciesList1, - }, - { - Counter: testCounter1, - PoliciesList: testPoliciesList2, - }, - { - Counter: testCounter2, - PoliciesList: testPoliciesList1, - }, - { - Counter: testCounter2, - PoliciesList: testPoliciesList2, - }, - } - expected := []unaggregated.CounterWithMetadatas{ - { - Counter: testCounter1, - StagedMetadatas: testConvertedMetadatas1, - }, - { - Counter: testCounter1, - StagedMetadatas: testConvertedMetadatas2, - }, - { - Counter: testCounter2, - StagedMetadatas: testConvertedMetadatas1, - }, - { - Counter: testCounter2, - StagedMetadatas: testConvertedMetadatas2, - }, - } - - encoder := msgpack.NewUnaggregatedEncoder(msgpack.NewBufferedEncoder()) - for _, input := range inputs { - require.NoError(t, encoder.EncodeCounterWithPoliciesList(input)) - } - var ( - i int - stream = bytes.NewReader(encoder.Encoder().Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.CounterWithMetadatasType, res.Type) - require.Equal(t, expected[i], res.CounterWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeBatchTimerWithPoliciesList(t *testing.T) { - inputs := []unaggregated.BatchTimerWithPoliciesList{ - { - BatchTimer: testBatchTimer1, - PoliciesList: testPoliciesList1, - }, - { - BatchTimer: testBatchTimer1, - PoliciesList: testPoliciesList2, - }, - { - BatchTimer: testBatchTimer2, - PoliciesList: testPoliciesList1, - }, - { - BatchTimer: testBatchTimer2, - PoliciesList: testPoliciesList2, - }, - } - expected := []unaggregated.BatchTimerWithMetadatas{ - { - BatchTimer: testBatchTimer1, - StagedMetadatas: testConvertedMetadatas1, - }, - { - BatchTimer: testBatchTimer1, - StagedMetadatas: testConvertedMetadatas2, - }, - { - BatchTimer: testBatchTimer2, - StagedMetadatas: testConvertedMetadatas1, - }, - { - BatchTimer: testBatchTimer2, - StagedMetadatas: testConvertedMetadatas2, - }, - } - - encoder := msgpack.NewUnaggregatedEncoder(msgpack.NewBufferedEncoder()) - for _, input := range inputs { - require.NoError(t, encoder.EncodeBatchTimerWithPoliciesList(input)) - } - var ( - i int - stream = bytes.NewReader(encoder.Encoder().Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.BatchTimerWithMetadatasType, res.Type) - require.Equal(t, expected[i], res.BatchTimerWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeGaugeWithPoliciesList(t *testing.T) { - inputs := []unaggregated.GaugeWithPoliciesList{ - { - Gauge: testGauge1, - PoliciesList: testPoliciesList1, - }, - { - Gauge: testGauge1, - PoliciesList: testPoliciesList2, - }, - { - Gauge: testGauge2, - PoliciesList: testPoliciesList1, - }, - { - Gauge: testGauge2, - PoliciesList: testPoliciesList2, - }, - } - expected := []unaggregated.GaugeWithMetadatas{ - { - Gauge: testGauge1, - StagedMetadatas: testConvertedMetadatas1, - }, - { - Gauge: testGauge1, - StagedMetadatas: testConvertedMetadatas2, - }, - { - Gauge: testGauge2, - StagedMetadatas: testConvertedMetadatas1, - }, - { - Gauge: testGauge2, - StagedMetadatas: testConvertedMetadatas2, - }, - } - - encoder := msgpack.NewUnaggregatedEncoder(msgpack.NewBufferedEncoder()) - for _, input := range inputs { - require.NoError(t, encoder.EncodeGaugeWithPoliciesList(input)) - } - var ( - i int - stream = bytes.NewReader(encoder.Encoder().Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.GaugeWithMetadatasType, res.Type) - require.Equal(t, expected[i], res.GaugeWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeCounterWithMetadatas(t *testing.T) { - inputs := []unaggregated.CounterWithMetadatas{ - { - Counter: testCounter1, - StagedMetadatas: testMetadatas1, - }, - { - Counter: testCounter2, - StagedMetadatas: testMetadatas1, - }, - { - Counter: testCounter1, - StagedMetadatas: testMetadatas2, - }, - { - Counter: testCounter2, - StagedMetadatas: testMetadatas2, - }, - } - - enc := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) - for _, input := range inputs { - require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: input, - })) - } - dataBuf := enc.Relinquish() - defer dataBuf.Close() - - var ( - i int - stream = bytes.NewReader(dataBuf.Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.CounterWithMetadatasType, res.Type) - require.Equal(t, inputs[i], res.CounterWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeBatchTimerWithMetadatas(t *testing.T) { - inputs := []unaggregated.BatchTimerWithMetadatas{ - { - BatchTimer: testBatchTimer1, - StagedMetadatas: testMetadatas1, - }, - { - BatchTimer: testBatchTimer2, - StagedMetadatas: testMetadatas1, - }, - { - BatchTimer: testBatchTimer1, - StagedMetadatas: testMetadatas2, - }, - { - BatchTimer: testBatchTimer2, - StagedMetadatas: testMetadatas2, - }, - } - - enc := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) - for _, input := range inputs { - require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: input, - })) - } - dataBuf := enc.Relinquish() - defer dataBuf.Close() - - var ( - i int - stream = bytes.NewReader(dataBuf.Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.BatchTimerWithMetadatasType, res.Type) - require.Equal(t, inputs[i], res.BatchTimerWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeGaugeWithMetadatas(t *testing.T) { - inputs := []unaggregated.GaugeWithMetadatas{ - { - Gauge: testGauge1, - StagedMetadatas: testMetadatas1, - }, - { - Gauge: testGauge2, - StagedMetadatas: testMetadatas1, - }, - { - Gauge: testGauge1, - StagedMetadatas: testMetadatas2, - }, - { - Gauge: testGauge2, - StagedMetadatas: testMetadatas2, - }, - } - - enc := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) - for _, input := range inputs { - require.NoError(t, enc.EncodeMessage(encoding.UnaggregatedMessageUnion{ - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: input, - })) - } - dataBuf := enc.Relinquish() - defer dataBuf.Close() - - var ( - i int - stream = bytes.NewReader(dataBuf.Bytes()) - ) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - require.Equal(t, encoding.GaugeWithMetadatasType, res.Type) - require.Equal(t, inputs[i], res.GaugeWithMetadatas) - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, len(inputs), i) -} - -func TestUnaggregatedIteratorDecodeStress(t *testing.T) { - var ( - numIter = 1000 - testCounters = []unaggregated.Counter{testCounter1, testCounter2} - testBatchTimers = []unaggregated.BatchTimer{testBatchTimer1, testBatchTimer2} - testGauges = []unaggregated.Gauge{testGauge1, testGauge2} - testPoliciesLists = []policy.PoliciesList{testPoliciesList1, testPoliciesList2} - testMetadataLists = []metadata.StagedMetadatas{testMetadatas1, testMetadatas2} - testConvertedMetadataLists = []metadata.StagedMetadatas{testConvertedMetadatas1, testConvertedMetadatas2} - msgpackInputs []interface{} - expectedMsgpackOutputs []encoding.UnaggregatedMessageUnion - protobufInputs []interface{} - ) - - for metricIdx := 0; metricIdx < 2; metricIdx++ { - for metadataIdx := 0; metadataIdx < 2; metadataIdx++ { - msgpackInputs = append(msgpackInputs, unaggregated.CounterWithPoliciesList{ - Counter: testCounters[metricIdx], - PoliciesList: testPoliciesLists[metadataIdx], - }) - msgpackInputs = append(msgpackInputs, unaggregated.BatchTimerWithPoliciesList{ - BatchTimer: testBatchTimers[metricIdx], - PoliciesList: testPoliciesLists[metadataIdx], - }) - msgpackInputs = append(msgpackInputs, unaggregated.GaugeWithPoliciesList{ - Gauge: testGauges[metricIdx], - PoliciesList: testPoliciesLists[metadataIdx], - }) - expectedMsgpackOutputs = append(expectedMsgpackOutputs, encoding.UnaggregatedMessageUnion{ - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: unaggregated.CounterWithMetadatas{ - Counter: testCounters[metricIdx], - StagedMetadatas: testConvertedMetadataLists[metadataIdx], - }, - }) - expectedMsgpackOutputs = append(expectedMsgpackOutputs, encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: unaggregated.BatchTimerWithMetadatas{ - BatchTimer: testBatchTimers[metricIdx], - StagedMetadatas: testConvertedMetadataLists[metadataIdx], - }, - }) - expectedMsgpackOutputs = append(expectedMsgpackOutputs, encoding.UnaggregatedMessageUnion{ - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: unaggregated.GaugeWithMetadatas{ - Gauge: testGauges[metricIdx], - StagedMetadatas: testConvertedMetadataLists[metadataIdx], - }, - }) - } - } - - for metricIdx := 0; metricIdx < 2; metricIdx++ { - for metadataIdx := 0; metadataIdx < 2; metadataIdx++ { - protobufInputs = append(protobufInputs, unaggregated.CounterWithMetadatas{ - Counter: testCounters[metricIdx], - StagedMetadatas: testMetadataLists[metadataIdx], - }) - protobufInputs = append(protobufInputs, unaggregated.BatchTimerWithMetadatas{ - BatchTimer: testBatchTimers[metricIdx], - StagedMetadatas: testMetadataLists[metadataIdx], - }) - protobufInputs = append(protobufInputs, unaggregated.GaugeWithMetadatas{ - Gauge: testGauges[metricIdx], - StagedMetadatas: testMetadataLists[metadataIdx], - }) - } - } - - var stream bytes.Buffer - for iter := 0; iter < numIter; iter++ { - msgpackEncoder := msgpack.NewUnaggregatedEncoder(msgpack.NewBufferedEncoder()) - protobufEncoder := protobuf.NewUnaggregatedEncoder(protobuf.NewUnaggregatedOptions()) - for _, input := range msgpackInputs { - switch input := input.(type) { - case unaggregated.CounterWithPoliciesList: - require.NoError(t, msgpackEncoder.EncodeCounterWithPoliciesList(input)) - case unaggregated.BatchTimerWithPoliciesList: - require.NoError(t, msgpackEncoder.EncodeBatchTimerWithPoliciesList(input)) - case unaggregated.GaugeWithPoliciesList: - require.NoError(t, msgpackEncoder.EncodeGaugeWithPoliciesList(input)) - default: - require.Fail(t, "unrecognized type %T", input) - } - } - for _, input := range protobufInputs { - var msg encoding.UnaggregatedMessageUnion - switch input := input.(type) { - case unaggregated.CounterWithMetadatas: - msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.CounterWithMetadatasType, - CounterWithMetadatas: input, - } - case unaggregated.BatchTimerWithMetadatas: - msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.BatchTimerWithMetadatasType, - BatchTimerWithMetadatas: input, - } - case unaggregated.GaugeWithMetadatas: - msg = encoding.UnaggregatedMessageUnion{ - Type: encoding.GaugeWithMetadatasType, - GaugeWithMetadatas: input, - } - default: - require.Fail(t, "unrecognized type %T", input) - } - require.NoError(t, protobufEncoder.EncodeMessage(msg)) - } - _, err := stream.Write(msgpackEncoder.Encoder().Bytes()) - require.NoError(t, err) - dataBuf := protobufEncoder.Relinquish() - _, err = stream.Write(dataBuf.Bytes()) - require.NoError(t, err) - dataBuf.Close() - } - - var ( - i int - reader = bytes.NewReader(stream.Bytes()) - ) - it := NewUnaggregatedIterator( - reader, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - for it.Next() { - res := it.Current() - j := i % (len(msgpackInputs) + len(protobufInputs)) - if j < len(msgpackInputs) { - require.True(t, cmp.Equal(expectedMsgpackOutputs[j], res, testCmpOpts...)) - } else { - // Protobuf encoded data. - j -= len(msgpackInputs) - switch expectedRes := protobufInputs[j].(type) { - case unaggregated.CounterWithMetadatas: - require.Equal(t, encoding.CounterWithMetadatasType, res.Type) - require.True(t, cmp.Equal(expectedRes, res.CounterWithMetadatas, testCmpOpts...)) - case unaggregated.BatchTimerWithMetadatas: - require.Equal(t, encoding.BatchTimerWithMetadatasType, res.Type) - require.True(t, cmp.Equal(expectedRes, res.BatchTimerWithMetadatas, testCmpOpts...)) - case unaggregated.GaugeWithMetadatas: - require.Equal(t, encoding.GaugeWithMetadatasType, res.Type) - require.True(t, cmp.Equal(expectedRes, res.GaugeWithMetadatas, testCmpOpts...)) - default: - require.Fail(t, "unknown input type: %T", protobufInputs[j]) - } - } - i++ - } - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, (len(msgpackInputs)+len(protobufInputs))*numIter, i) -} - -func TestUnaggregatedIteratorDecodeMsgpackError(t *testing.T) { - reader := bytes.NewReader([]byte{0x1, 0x2}) - it := NewUnaggregatedIterator( - reader, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - i := 0 - for it.Next() { - i++ - } - require.NotEqual(t, io.EOF, it.Err()) - require.Equal(t, it.Err(), it.(*unaggregatedIterator).msgpackIt.Err()) - require.Equal(t, 0, i) - - // Verify calling Next() still returns false. - require.False(t, it.Next()) - require.Equal(t, it.Err(), it.(*unaggregatedIterator).msgpackIt.Err()) -} - -func TestUnaggregatedIteratorDecodeProtobufError(t *testing.T) { - reader := bytes.NewReader([]byte{0x2, 0x2}) - it := NewUnaggregatedIterator( - reader, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - defer it.Close() - i := 0 - for it.Next() { - i++ - } - require.NotEqual(t, io.EOF, it.Err()) - require.Equal(t, it.Err(), it.(*unaggregatedIterator).protobufIt.Err()) - require.Equal(t, 0, i) - - // Verify calling Next() still returns false. - require.False(t, it.Next()) - require.Equal(t, it.Err(), it.(*unaggregatedIterator).protobufIt.Err()) -} - -func TestUnaggregatedIteratorNextOnClose(t *testing.T) { - stream := bytes.NewReader([]byte{0x1, 0x2, 0x3}) - it := NewUnaggregatedIterator( - stream, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - iterator := it.(*unaggregatedIterator) - require.False(t, iterator.closed) - require.Nil(t, it.Err()) - - // Verify that closing the iterator cleans up the state. - it.Close() - require.False(t, it.Next()) - require.False(t, it.Next()) - require.True(t, iterator.closed) - require.Equal(t, encoding.UnaggregatedMessageUnion{}, iterator.msg) - - // Verify that closing a second time is a no op. - it.Close() -} - -func TestUnaggregatedIteratorClose(t *testing.T) { - it := NewUnaggregatedIterator( - nil, - msgpack.NewUnaggregatedIteratorOptions(), - protobuf.NewUnaggregatedOptions(), - ) - require.False(t, it.(*unaggregatedIterator).closed) - require.NotNil(t, it.(*unaggregatedIterator).msgpackIt) - require.NotNil(t, it.(*unaggregatedIterator).protobufIt) - - it.Close() - require.True(t, it.(*unaggregatedIterator).closed) - require.Nil(t, it.(*unaggregatedIterator).msgpackIt) - require.Nil(t, it.(*unaggregatedIterator).protobufIt) - - // Verify that closing a second time is a no op. - it.Close() - require.True(t, it.(*unaggregatedIterator).closed) - require.Nil(t, it.(*unaggregatedIterator).msgpackIt) - require.Nil(t, it.(*unaggregatedIterator).protobufIt) -} diff --git a/src/metrics/encoding/msgpack/aggregated_encoder.go b/src/metrics/encoding/msgpack/aggregated_encoder.go deleted file mode 100644 index 8e9af130d7..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_encoder.go +++ /dev/null @@ -1,171 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/policy" -) - -type encodeRawMetricWithStoragePolicyFn func(data []byte, p policy.StoragePolicy) -type encodeRawMetricWithStoragePolicyAndEncodeTimeFn func(data []byte, p policy.StoragePolicy, encodedAtNanos int64) -type encodeRawMetricFn func(data []byte) -type encodeMetricAsRawFn func(m aggregated.Metric) []byte -type encodeChunkedMetricAsRawFn func(m aggregated.ChunkedMetric) []byte - -// aggregatedEncoder uses MessagePack for encoding aggregated metrics. -// It is not thread-safe. -type aggregatedEncoder struct { - encoderBase - - buf encoderBase - encodeRootObjectFn encodeRootObjectFn - encodeRawMetricWithStoragePolicyFn encodeRawMetricWithStoragePolicyFn - encodeRawMetricWithStoragePolicyAndEncodeTimeFn encodeRawMetricWithStoragePolicyAndEncodeTimeFn - encodeRawMetricFn encodeRawMetricFn - encodeMetricAsRawFn encodeMetricAsRawFn - encodeChunkedMetricAsRawFn encodeChunkedMetricAsRawFn -} - -// NewAggregatedEncoder creates an aggregated encoder. -func NewAggregatedEncoder(encoder BufferedEncoder) AggregatedEncoder { - enc := &aggregatedEncoder{ - encoderBase: newBaseEncoder(encoder), - buf: newBaseEncoder(NewBufferedEncoder()), - } - - enc.encodeRootObjectFn = enc.encodeRootObject - enc.encodeRawMetricWithStoragePolicyFn = enc.encodeRawMetricWithStoragePolicy - enc.encodeRawMetricWithStoragePolicyAndEncodeTimeFn = enc.encodeRawMetricWithStoragePolicyAndEncodeTime - enc.encodeRawMetricFn = enc.encodeRawMetric - enc.encodeMetricAsRawFn = enc.encodeMetricAsRaw - enc.encodeChunkedMetricAsRawFn = enc.encodeChunkedMetricAsRaw - - return enc -} - -func (enc *aggregatedEncoder) Encoder() BufferedEncoder { return enc.encoder() } -func (enc *aggregatedEncoder) Reset(encoder BufferedEncoder) { enc.reset(encoder) } - -// NB(xichen): we encode metric as a raw metric so the decoder can inspect the encoded byte stream -// and apply filters to the encode bytes as needed without fully decoding the entire payload. -func (enc *aggregatedEncoder) EncodeMetricWithStoragePolicy( - mp aggregated.MetricWithStoragePolicy, -) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(rawMetricWithStoragePolicyType) - data := enc.encodeMetricAsRawFn(mp.Metric) - enc.encodeRawMetricWithStoragePolicyFn(data, mp.StoragePolicy) - return enc.err() -} - -func (enc *aggregatedEncoder) EncodeMetricWithStoragePolicyAndEncodeTime( - mp aggregated.MetricWithStoragePolicy, - encodedAtNanos int64, -) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(rawMetricWithStoragePolicyAndEncodeTimeType) - data := enc.encodeMetricAsRawFn(mp.Metric) - enc.encodeRawMetricWithStoragePolicyAndEncodeTimeFn(data, mp.StoragePolicy, encodedAtNanos) - return enc.err() -} - -func (enc *aggregatedEncoder) EncodeChunkedMetricWithStoragePolicy( - cmp aggregated.ChunkedMetricWithStoragePolicy, -) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(rawMetricWithStoragePolicyType) - data := enc.encodeChunkedMetricAsRawFn(cmp.ChunkedMetric) - enc.encodeRawMetricWithStoragePolicyFn(data, cmp.StoragePolicy) - return enc.err() -} - -func (enc *aggregatedEncoder) EncodeChunkedMetricWithStoragePolicyAndEncodeTime( - cmp aggregated.ChunkedMetricWithStoragePolicy, - encodedAtNanos int64, -) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(rawMetricWithStoragePolicyAndEncodeTimeType) - data := enc.encodeChunkedMetricAsRawFn(cmp.ChunkedMetric) - enc.encodeRawMetricWithStoragePolicyAndEncodeTimeFn(data, cmp.StoragePolicy, encodedAtNanos) - return enc.err() -} - -func (enc *aggregatedEncoder) encodeRootObject(objType objectType) { - enc.encodeVersion(aggregatedVersion) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType)) - enc.encodeObjectType(objType) -} - -func (enc *aggregatedEncoder) encodeMetricAsRaw(m aggregated.Metric) []byte { - enc.buf.resetData() - enc.encodeMetricProlog() - enc.buf.encodeRawID(m.ID) - enc.buf.encodeVarint(m.TimeNanos) - enc.buf.encodeFloat64(m.Value) - return enc.buf.encoder().Bytes() -} - -func (enc *aggregatedEncoder) encodeChunkedMetricAsRaw(m aggregated.ChunkedMetric) []byte { - enc.buf.resetData() - enc.encodeMetricProlog() - enc.buf.encodeChunkedID(m.ChunkedID) - enc.buf.encodeVarint(m.TimeNanos) - enc.buf.encodeFloat64(m.Value) - return enc.buf.encoder().Bytes() -} - -func (enc *aggregatedEncoder) encodeMetricProlog() { - enc.buf.encodeVersion(metricVersion) - enc.buf.encodeNumObjectFields(numFieldsForType(metricType)) -} - -func (enc *aggregatedEncoder) encodeRawMetricWithStoragePolicy( - data []byte, - p policy.StoragePolicy, -) { - enc.encodeNumObjectFields(numFieldsForType(rawMetricWithStoragePolicyType)) - enc.encodeRawMetricFn(data) - enc.encodeStoragePolicy(p) -} - -func (enc *aggregatedEncoder) encodeRawMetricWithStoragePolicyAndEncodeTime( - data []byte, - p policy.StoragePolicy, - encodedAtNanos int64, -) { - enc.encodeNumObjectFields(numFieldsForType(rawMetricWithStoragePolicyAndEncodeTimeType)) - enc.encodeRawMetricFn(data) - enc.encodeStoragePolicy(p) - enc.encodeVarint(encodedAtNanos) -} - -func (enc *aggregatedEncoder) encodeRawMetric(data []byte) { - enc.encodeBytes(data) -} diff --git a/src/metrics/encoding/msgpack/aggregated_encoder_test.go b/src/metrics/encoding/msgpack/aggregated_encoder_test.go deleted file mode 100644 index aad6366255..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_encoder_test.go +++ /dev/null @@ -1,180 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "testing" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/policy" - - "github.com/stretchr/testify/require" -) - -func testCapturingAggregatedEncoder() (AggregatedEncoder, *[]interface{}) { - encoder := testAggregatedEncoder().(*aggregatedEncoder) - result := testCapturingBaseEncoder(encoder.encoderBase) - return encoder, result -} - -func expectedResultsForRawMetricWithPolicy( - m aggregated.RawMetric, - p policy.StoragePolicy, -) []interface{} { - results := []interface{}{ - numFieldsForType(rawMetricWithStoragePolicyType), - m.Bytes(), - } - results = append(results, expectedResultsForPolicy(p)...) - return results -} - -func expectedResultsForAggregatedMetricWithPolicy( - t *testing.T, - m interface{}, - p policy.StoragePolicy, -) []interface{} { - results := []interface{}{ - int64(aggregatedVersion), - numFieldsForType(rootObjectType), - int64(rawMetricWithStoragePolicyType), - } - switch m := m.(type) { - case aggregated.Metric: - rm := toRawMetric(t, m) - results = append(results, expectedResultsForRawMetricWithPolicy(rm, p)...) - case aggregated.ChunkedMetric: - rm := toRawMetric(t, m) - results = append(results, expectedResultsForRawMetricWithPolicy(rm, p)...) - default: - require.Fail(t, "unrecognized input type %T", m) - } - return results -} - -func expectedResultsForRawMetricWithPolicyAndEncodeTime( - m aggregated.RawMetric, - p policy.StoragePolicy, - encodedAtNanos int64, -) []interface{} { - results := []interface{}{ - numFieldsForType(rawMetricWithStoragePolicyAndEncodeTimeType), - m.Bytes(), - } - results = append(results, expectedResultsForPolicy(p)...) - results = append(results, encodedAtNanos) - return results -} - -func expectedResultsForAggregatedMetricWithPolicyAndEncodeTime( - t *testing.T, - m interface{}, - p policy.StoragePolicy, - encodedAtNanos int64, -) []interface{} { - results := []interface{}{ - int64(aggregatedVersion), - numFieldsForType(rootObjectType), - int64(rawMetricWithStoragePolicyAndEncodeTimeType), - } - switch m := m.(type) { - case aggregated.Metric: - rm := toRawMetric(t, m) - res := expectedResultsForRawMetricWithPolicyAndEncodeTime(rm, p, encodedAtNanos) - results = append(results, res...) - case aggregated.ChunkedMetric: - rm := toRawMetric(t, m) - res := expectedResultsForRawMetricWithPolicyAndEncodeTime(rm, p, encodedAtNanos) - results = append(results, res...) - default: - require.Fail(t, "unrecognized input type %T", m) - } - return results -} - -func TestAggregatedEncodeMetric(t *testing.T) { - encoder := testAggregatedEncoder().(*aggregatedEncoder) - result := testCapturingBaseEncoder(encoder.buf) - encoder.encodeMetricAsRaw(testMetric) - expected := []interface{}{ - int64(metricVersion), - int(numFieldsForType(metricType)), - []byte(testMetric.ID), - testMetric.TimeNanos, - testMetric.Value, - } - require.Equal(t, expected, *result) -} - -func TestAggregatedEncodeMetricWithPolicy(t *testing.T) { - encoder, results := testCapturingAggregatedEncoder() - require.NoError(t, testAggregatedEncodeMetricWithPolicy(encoder, testMetric, testPolicy)) - expected := expectedResultsForAggregatedMetricWithPolicy(t, testMetric, testPolicy) - require.Equal(t, expected, *results) -} - -func TestAggregatedEncodeMetricWithPolicyAndEncodeTime(t *testing.T) { - encoder, results := testCapturingAggregatedEncoder() - err := testAggregatedEncodeMetricWithPolicyAndEncodeTime(encoder, testMetric, testPolicy, testEncodedAtNanos) - require.NoError(t, err) - expected := expectedResultsForAggregatedMetricWithPolicyAndEncodeTime(t, testMetric, testPolicy, testEncodedAtNanos) - require.Equal(t, expected, *results) -} - -func TestAggregatedEncodeChunkedMetricWithPolicy(t *testing.T) { - encoder, results := testCapturingAggregatedEncoder() - require.NoError(t, testAggregatedEncodeMetricWithPolicy(encoder, testChunkedMetric, testPolicy)) - expected := expectedResultsForAggregatedMetricWithPolicy(t, testChunkedMetric, testPolicy) - require.Equal(t, expected, *results) -} - -func TestAggregatedEncodeChunkedMetricWithPolicyAndEncodeTime(t *testing.T) { - encoder, results := testCapturingAggregatedEncoder() - err := testAggregatedEncodeMetricWithPolicyAndEncodeTime(encoder, testChunkedMetric, testPolicy, testEncodedAtNanos) - require.NoError(t, err) - expected := expectedResultsForAggregatedMetricWithPolicyAndEncodeTime(t, testChunkedMetric, testPolicy, testEncodedAtNanos) - require.Equal(t, expected, *results) -} - -func TestAggregatedEncodeError(t *testing.T) { - // Intentionally return an error when encoding varint. - encoder := testAggregatedEncoder().(*aggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeVarintFn = func(value int64) { - baseEncoder.encodeErr = errTestVarint - } - - // Assert the error is expected. - require.Equal(t, errTestVarint, testAggregatedEncodeMetricWithPolicy(encoder, testMetric, testPolicy)) - - // Assert re-encoding doesn't change the error. - require.Equal(t, errTestVarint, testAggregatedEncodeMetricWithPolicy(encoder, testMetric, testPolicy)) -} - -func TestAggregatedEncoderReset(t *testing.T) { - encoder := testAggregatedEncoder().(*aggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeErr = errTestVarint - require.Equal(t, errTestVarint, testAggregatedEncodeMetricWithPolicy(encoder, testMetric, testPolicy)) - - encoder.Reset(NewBufferedEncoder()) - require.NoError(t, testAggregatedEncodeMetricWithPolicy(encoder, testMetric, testPolicy)) -} diff --git a/src/metrics/encoding/msgpack/aggregated_iterator.go b/src/metrics/encoding/msgpack/aggregated_iterator.go deleted file mode 100644 index ea3fbab829..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_iterator.go +++ /dev/null @@ -1,148 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "fmt" - "io" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/policy" -) - -// aggregatedIterator is an iterator for decoding aggregated metrics. -type aggregatedIterator struct { - iteratorBase - - ignoreHigherVersion bool - closed bool - iteratorPool AggregatedIteratorPool - metric aggregated.RawMetric - storagePolicy policy.StoragePolicy - encodedAtNanos int64 -} - -// NewAggregatedIterator creates a new aggregated iterator. -func NewAggregatedIterator(reader io.Reader, opts AggregatedIteratorOptions) AggregatedIterator { - if opts == nil { - opts = NewAggregatedIteratorOptions() - } - readerBufferSize := opts.ReaderBufferSize() - return &aggregatedIterator{ - ignoreHigherVersion: opts.IgnoreHigherVersion(), - iteratorBase: newBaseIterator(reader, readerBufferSize), - metric: NewRawMetric(nil, readerBufferSize), - iteratorPool: opts.IteratorPool(), - } -} - -func (it *aggregatedIterator) Err() error { return it.err() } - -func (it *aggregatedIterator) Reset(reader io.Reader) { - it.closed = false - it.reset(reader) -} - -func (it *aggregatedIterator) Value() (aggregated.RawMetric, policy.StoragePolicy, int64) { - return it.metric, it.storagePolicy, it.encodedAtNanos -} - -func (it *aggregatedIterator) Next() bool { - if it.err() != nil || it.closed { - return false - } - return it.decodeRootObject() -} - -func (it *aggregatedIterator) Close() { - if it.closed { - return - } - it.closed = true - it.reset(emptyReader) - it.metric.Reset(nil) - if it.iteratorPool != nil { - it.iteratorPool.Put(it) - } -} - -func (it *aggregatedIterator) decodeRootObject() bool { - version := it.decodeVersion() - if it.err() != nil { - return false - } - // If the actual version is higher than supported version, we skip - // the data for this metric and continue to the next. - if version > aggregatedVersion { - if it.ignoreHigherVersion { - it.skip(it.decodeNumObjectFields()) - return it.Next() - } - it.setErr(fmt.Errorf("received version %d is higher than supported version %d", version, aggregatedVersion)) - return false - } - // Otherwise we proceed to decoding normally. - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(rootObjectType) - if !ok { - return false - } - objType := it.decodeObjectType() - if it.err() != nil { - return false - } - switch objType { - case rawMetricWithStoragePolicyType: - it.decodeRawMetricWithStoragePolicy() - case rawMetricWithStoragePolicyAndEncodeTimeType: - it.decodeRawMetricWithStoragePolicyAndEncodeTime() - default: - it.setErr(fmt.Errorf("unrecognized object type %v", objType)) - } - it.skip(numActualFields - numExpectedFields) - - return it.err() == nil -} - -func (it *aggregatedIterator) decodeRawMetricWithStoragePolicy() { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(rawMetricWithStoragePolicyType) - if !ok { - return - } - it.metric.Reset(it.decodeRawMetric()) - it.storagePolicy = it.decodeStoragePolicy() - it.encodedAtNanos = 0 - it.skip(numActualFields - numExpectedFields) -} - -func (it *aggregatedIterator) decodeRawMetricWithStoragePolicyAndEncodeTime() { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(rawMetricWithStoragePolicyAndEncodeTimeType) - if !ok { - return - } - it.metric.Reset(it.decodeRawMetric()) - it.storagePolicy = it.decodeStoragePolicy() - it.encodedAtNanos = it.decodeVarint() - it.skip(numActualFields - numExpectedFields) -} - -func (it *aggregatedIterator) decodeRawMetric() []byte { - return it.decodeBytes() -} diff --git a/src/metrics/encoding/msgpack/aggregated_iterator_pool.go b/src/metrics/encoding/msgpack/aggregated_iterator_pool.go deleted file mode 100644 index 34b50cd04d..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_iterator_pool.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import "github.com/m3db/m3/src/x/pool" - -type aggregatedIteratorPool struct { - pool pool.ObjectPool -} - -// NewAggregatedIteratorPool creates a new pool for aggregated iterators. -func NewAggregatedIteratorPool(opts pool.ObjectPoolOptions) AggregatedIteratorPool { - return &aggregatedIteratorPool{pool: pool.NewObjectPool(opts)} -} - -func (p *aggregatedIteratorPool) Init(alloc AggregatedIteratorAlloc) { - p.pool.Init(func() interface{} { - return alloc() - }) -} - -func (p *aggregatedIteratorPool) Get() AggregatedIterator { - return p.pool.Get().(AggregatedIterator) -} - -func (p *aggregatedIteratorPool) Put(it AggregatedIterator) { - p.pool.Put(it) -} diff --git a/src/metrics/encoding/msgpack/aggregated_iterator_pool_test.go b/src/metrics/encoding/msgpack/aggregated_iterator_pool_test.go deleted file mode 100644 index 48ff512d75..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_iterator_pool_test.go +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "testing" - - "github.com/m3db/m3/src/x/pool" - - "github.com/stretchr/testify/require" -) - -func TestAggregatedIteratorPool(t *testing.T) { - p := NewAggregatedIteratorPool(pool.NewObjectPoolOptions().SetSize(1)) - itOpts := NewAggregatedIteratorOptions().SetIteratorPool(p) - p.Init(func() AggregatedIterator { - return NewAggregatedIterator(nil, itOpts) - }) - - // Retrieve an iterator from the pool. - it := p.Get() - it.Reset(bytes.NewBuffer([]byte{0x1, 0x2})) - - // Closing the iterator should put it back to the pool. - it.Close() - require.True(t, it.(*aggregatedIterator).closed) - - // Retrieve the iterator and assert it's the same iterator. - it = p.Get() - require.True(t, it.(*aggregatedIterator).closed) - - // Reset the iterator and assert it's been reset. - it.Reset(nil) - require.False(t, it.(*aggregatedIterator).closed) -} diff --git a/src/metrics/encoding/msgpack/aggregated_iterator_test.go b/src/metrics/encoding/msgpack/aggregated_iterator_test.go deleted file mode 100644 index f3646c4891..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_iterator_test.go +++ /dev/null @@ -1,207 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "io" - "testing" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/policy" - - "github.com/stretchr/testify/require" -) - -func validateAggregatedDecodeResults( - t *testing.T, - it AggregatedIterator, - expectedResults []metricWithPolicyAndEncodeTime, - expectedErr error, -) { - var results []metricWithPolicyAndEncodeTime - for it.Next() { - value, policy, encodedAtNanos := it.Value() - m, err := value.Metric() - require.NoError(t, err) - results = append(results, metricWithPolicyAndEncodeTime{ - metric: m, - policy: policy, - encodedAtNanos: encodedAtNanos, - }) - } - require.Equal(t, expectedErr, it.Err()) - require.Equal(t, expectedResults, results) -} - -func TestAggregatedIteratorDecodeNewerVersionThanSupported(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Version encoded is higher than supported version. - enc.encodeRootObjectFn = func(objType objectType) { - enc.encodeVersion(aggregatedVersion + 1) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType)) - enc.encodeObjectType(objType) - } - require.NoError(t, testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy)) - - // Now restore the encode top-level function and encode another metric. - enc.encodeRootObjectFn = enc.encodeRootObject - require.NoError(t, testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy)) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - it.(*aggregatedIterator).ignoreHigherVersion = true - - // Check that we skipped the first metric and successfully decoded the second metric. - validateAggregatedDecodeResults(t, it, []metricWithPolicyAndEncodeTime{input}, io.EOF) -} - -func TestAggregatedIteratorDecodeRootObjectMoreFieldsThanExpected(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Pretend we added an extra int field to the root object. - enc.encodeRootObjectFn = func(objType objectType) { - enc.encodeVersion(unaggregatedVersion) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType) + 1) - enc.encodeObjectType(objType) - } - err := testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy) - require.NoError(t, err) - enc.encodeVarint(0) - require.NoError(t, enc.err()) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the metric. - validateAggregatedDecodeResults(t, it, []metricWithPolicyAndEncodeTime{input}, io.EOF) -} - -func TestAggregatedIteratorDecodeRawMetricMoreFieldsThanExpected(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Pretend we added an extra int field to the raw metric with policy object. - enc.encodeRawMetricWithStoragePolicyFn = func(data []byte, p policy.StoragePolicy) { - enc.encodeNumObjectFields(numFieldsForType(rawMetricWithStoragePolicyType) + 1) - enc.encodeRawMetricFn(data) - enc.encodeStoragePolicy(p) - } - err := testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy) - require.NoError(t, err) - enc.encodeVarint(0) - require.NoError(t, enc.err()) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the metric. - validateAggregatedDecodeResults(t, it, []metricWithPolicyAndEncodeTime{input}, io.EOF) -} - -func TestAggregatedIteratorDecodeRawMetricWithEncodeTimeMoreFieldsThanExpected(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - encodedAtNanos: testEncodedAtNanos, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Pretend we added an extra int field to the raw metric with policy object. - enc.encodeRawMetricWithStoragePolicyAndEncodeTimeFn = func(data []byte, p policy.StoragePolicy, encodedAtNanos int64) { - enc.encodeNumObjectFields(numFieldsForType(rawMetricWithStoragePolicyAndEncodeTimeType) + 1) - enc.encodeRawMetricFn(data) - enc.encodeStoragePolicy(p) - enc.encodeVarint(encodedAtNanos) - } - err := testAggregatedEncodeMetricWithPolicyAndEncodeTime(enc, input.metric.(aggregated.Metric), input.policy, input.encodedAtNanos) - require.NoError(t, err) - enc.encodeVarint(0) - require.NoError(t, enc.err()) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the metric. - validateAggregatedDecodeResults(t, it, []metricWithPolicyAndEncodeTime{input}, io.EOF) -} - -func TestAggregatedIteratorDecodeMetricHigherVersionThanSupported(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Pretend we added an extra int field to the raw metric object. - enc.encodeMetricAsRawFn = func(m aggregated.Metric) []byte { - enc.buf.resetData() - enc.buf.encodeVersion(metricVersion + 1) - return enc.buf.encoder().Bytes() - } - err := testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy) - require.NoError(t, err) - require.NoError(t, enc.err()) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - require.True(t, it.Next()) - rawMetric, _, _ := it.Value() - _, err = rawMetric.Value() - require.Error(t, err) -} - -func TestAggregatedIteratorDecodeMetricMoreFieldsThanExpected(t *testing.T) { - input := metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - } - enc := testAggregatedEncoder().(*aggregatedEncoder) - - // Pretend we added an extra int field to the raw metric object. - enc.encodeMetricAsRawFn = func(m aggregated.Metric) []byte { - enc.encodeMetricAsRaw(m) - enc.buf.encodeVarint(0) - return enc.buf.encoder().Bytes() - } - err := testAggregatedEncodeMetricWithPolicy(enc, input.metric.(aggregated.Metric), input.policy) - require.NoError(t, err) - require.NoError(t, enc.err()) - - it := testAggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the metric. - validateAggregatedDecodeResults(t, it, []metricWithPolicyAndEncodeTime{input}, io.EOF) -} - -func TestAggregatedIteratorClose(t *testing.T) { - it := NewAggregatedIterator(nil, nil) - it.Close() - require.False(t, it.Next()) - require.NoError(t, it.Err()) - require.True(t, it.(*aggregatedIterator).closed) -} diff --git a/src/metrics/encoding/msgpack/aggregated_roundtrip_test.go b/src/metrics/encoding/msgpack/aggregated_roundtrip_test.go deleted file mode 100644 index df004848a0..0000000000 --- a/src/metrics/encoding/msgpack/aggregated_roundtrip_test.go +++ /dev/null @@ -1,274 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "fmt" - "io" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -var ( - testMetric = aggregated.Metric{ - ID: id.RawID("foo"), - TimeNanos: time.Now().UnixNano(), - Value: 123.45, - } - testChunkedMetric = aggregated.ChunkedMetric{ - ChunkedID: id.ChunkedID{ - Prefix: []byte("foo."), - Data: []byte("bar"), - Suffix: []byte(".baz"), - }, - TimeNanos: time.Now().UnixNano(), - Value: 123.45, - } - testPolicy = policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour) - testEncodedAtNanos = time.Now().UnixNano() -) - -type metricWithPolicyAndEncodeTime struct { - metric interface{} - policy policy.StoragePolicy - encodedAtNanos int64 -} - -func testAggregatedEncoder() AggregatedEncoder { - return NewAggregatedEncoder(NewBufferedEncoder()) -} - -func testAggregatedIterator(reader io.Reader) AggregatedIterator { - return NewAggregatedIterator(reader, NewAggregatedIteratorOptions()) -} - -func testAggregatedEncodeMetricWithPolicy( - encoder AggregatedEncoder, - m interface{}, - p policy.StoragePolicy, -) error { - switch m := m.(type) { - case aggregated.Metric: - return encoder.EncodeMetricWithStoragePolicy(aggregated.MetricWithStoragePolicy{ - Metric: m, - StoragePolicy: p, - }) - case aggregated.ChunkedMetric: - return encoder.EncodeChunkedMetricWithStoragePolicy(aggregated.ChunkedMetricWithStoragePolicy{ - ChunkedMetric: m, - StoragePolicy: p, - }) - default: - return fmt.Errorf("unrecognized metric type: %T", m) - } -} - -func testAggregatedEncodeMetricWithPolicyAndEncodeTime( - encoder AggregatedEncoder, - m interface{}, - p policy.StoragePolicy, - encodedAtNanos int64, -) error { - switch m := m.(type) { - case aggregated.Metric: - input := aggregated.MetricWithStoragePolicy{ - Metric: m, - StoragePolicy: p, - } - return encoder.EncodeMetricWithStoragePolicyAndEncodeTime(input, encodedAtNanos) - case aggregated.ChunkedMetric: - input := aggregated.ChunkedMetricWithStoragePolicy{ - ChunkedMetric: m, - StoragePolicy: p, - } - return encoder.EncodeChunkedMetricWithStoragePolicyAndEncodeTime(input, encodedAtNanos) - default: - return fmt.Errorf("unrecognized metric type: %T", m) - } -} - -func toRawMetric(t *testing.T, m interface{}) aggregated.RawMetric { - encoder := NewAggregatedEncoder(NewBufferedEncoder()).(*aggregatedEncoder) - var data []byte - switch m := m.(type) { - case aggregated.Metric: - data = encoder.encodeMetricAsRaw(m) - case aggregated.ChunkedMetric: - data = encoder.encodeChunkedMetricAsRaw(m) - default: - require.Fail(t, "unrecognized metric type %T", m) - } - require.NoError(t, encoder.err()) - return NewRawMetric(data, 16) -} - -func validateAggregatedRoundtrip(t *testing.T, inputs ...metricWithPolicyAndEncodeTime) { - encoder := testAggregatedEncoder() - it := testAggregatedIterator(nil) - validateAggregatedRoundtripWithEncoderAndIterator(t, encoder, it, inputs...) -} - -func validateAggregatedRoundtripWithEncoderAndIterator( - t *testing.T, - encoder AggregatedEncoder, - it AggregatedIterator, - inputs ...metricWithPolicyAndEncodeTime, -) { - var ( - expected []metricWithPolicyAndEncodeTime - results []metricWithPolicyAndEncodeTime - ) - - // Encode the batch of metrics. - encoder.Reset(NewBufferedEncoder()) - for _, input := range inputs { - switch inputMetric := input.metric.(type) { - case aggregated.Metric: - expected = append(expected, metricWithPolicyAndEncodeTime{ - metric: inputMetric, - policy: input.policy, - encodedAtNanos: input.encodedAtNanos, - }) - if input.encodedAtNanos == 0 { - require.NoError(t, testAggregatedEncodeMetricWithPolicy(encoder, inputMetric, input.policy)) - } else { - require.NoError(t, testAggregatedEncodeMetricWithPolicyAndEncodeTime(encoder, inputMetric, input.policy, input.encodedAtNanos)) - } - case aggregated.ChunkedMetric: - var id id.RawID - id = append(id, inputMetric.ChunkedID.Prefix...) - id = append(id, inputMetric.ChunkedID.Data...) - id = append(id, inputMetric.ChunkedID.Suffix...) - expected = append(expected, metricWithPolicyAndEncodeTime{ - metric: aggregated.Metric{ - ID: id, - TimeNanos: inputMetric.TimeNanos, - Value: inputMetric.Value, - }, - policy: input.policy, - encodedAtNanos: input.encodedAtNanos, - }) - if input.encodedAtNanos == 0 { - require.NoError(t, testAggregatedEncodeMetricWithPolicy(encoder, inputMetric, input.policy)) - } else { - require.NoError(t, testAggregatedEncodeMetricWithPolicyAndEncodeTime(encoder, inputMetric, input.policy, input.encodedAtNanos)) - } - default: - require.Fail(t, "unrecognized input type %T", inputMetric) - } - } - - // Decode the batch of metrics. - encodedBytes := bytes.NewBuffer(encoder.Encoder().Bytes()) - it.Reset(encodedBytes) - for it.Next() { - metric, p, encodedAtNanos := it.Value() - m, err := metric.Metric() - require.NoError(t, err) - results = append(results, metricWithPolicyAndEncodeTime{ - metric: m, - policy: p, - encodedAtNanos: encodedAtNanos, - }) - } - - // Assert the results match expectations. - require.Equal(t, io.EOF, it.Err()) - require.Equal(t, expected, results) -} - -func TestAggregatedEncodeDecodeMetricWithPolicy(t *testing.T) { - validateAggregatedRoundtrip(t, metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - }) -} - -func TestAggregatedEncodeDecodeMetricWithPolicyAndEncodeTime(t *testing.T) { - validateAggregatedRoundtrip(t, metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - encodedAtNanos: testEncodedAtNanos, - }) -} - -func TestAggregatedEncodeDecodeChunkedMetricWithPolicy(t *testing.T) { - validateAggregatedRoundtrip(t, metricWithPolicyAndEncodeTime{ - metric: testChunkedMetric, - policy: testPolicy, - }) -} - -func TestAggregatedEncodeDecodeChunkedMetricWithPolicyAndEncodeTime(t *testing.T) { - validateAggregatedRoundtrip(t, metricWithPolicyAndEncodeTime{ - metric: testChunkedMetric, - policy: testPolicy, - encodedAtNanos: testEncodedAtNanos, - }) -} - -func TestAggregatedEncodeDecodeStress(t *testing.T) { - var ( - numIter = 10 - numMetrics = 10000 - encoder = testAggregatedEncoder() - iterator = testAggregatedIterator(nil) - ) - - for i := 0; i < numIter; i++ { - var inputs []metricWithPolicyAndEncodeTime - for j := 0; j < numMetrics; j++ { - switch j % 4 { - case 0: - inputs = append(inputs, metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - }) - case 1: - inputs = append(inputs, metricWithPolicyAndEncodeTime{ - metric: testMetric, - policy: testPolicy, - encodedAtNanos: testEncodedAtNanos, - }) - case 2: - inputs = append(inputs, metricWithPolicyAndEncodeTime{ - metric: testChunkedMetric, - policy: testPolicy, - }) - case 3: - inputs = append(inputs, metricWithPolicyAndEncodeTime{ - metric: testChunkedMetric, - policy: testPolicy, - encodedAtNanos: testEncodedAtNanos, - }) - } - } - validateAggregatedRoundtripWithEncoderAndIterator(t, encoder, iterator, inputs...) - } -} diff --git a/src/metrics/encoding/msgpack/base_encoder.go b/src/metrics/encoding/msgpack/base_encoder.go deleted file mode 100644 index c8f1a85c28..0000000000 --- a/src/metrics/encoding/msgpack/base_encoder.go +++ /dev/null @@ -1,217 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/policy" -) - -type encodeVarintFn func(value int64) -type encodeBoolFn func(value bool) -type encodeFloat64Fn func(value float64) -type encodeBytesFn func(value []byte) -type encodeBytesLenFn func(value int) -type encodeArrayLenFn func(value int) -type encodeStoragePolicyFn func(p policy.StoragePolicy) -type encodePolicyFn func(p policy.Policy) - -// baseEncoder is the base encoder that provides common encoding APIs. -type baseEncoder struct { - bufEncoder BufferedEncoder - encodeErr error - encodeVarintFn encodeVarintFn - encodeBoolFn encodeBoolFn - encodeFloat64Fn encodeFloat64Fn - encodeBytesFn encodeBytesFn - encodeBytesLenFn encodeBytesLenFn - encodeArrayLenFn encodeArrayLenFn - encodeStoragePolicyFn encodeStoragePolicyFn - encodePolicyFn encodePolicyFn -} - -func newBaseEncoder(encoder BufferedEncoder) encoderBase { - enc := &baseEncoder{bufEncoder: encoder} - - enc.encodeVarintFn = enc.encodeVarintInternal - enc.encodeBoolFn = enc.encodeBoolInternal - enc.encodeFloat64Fn = enc.encodeFloat64Internal - enc.encodeBytesFn = enc.encodeBytesInternal - enc.encodeBytesLenFn = enc.encodeBytesLenInternal - enc.encodeArrayLenFn = enc.encodeArrayLenInternal - enc.encodeStoragePolicyFn = enc.encodeStoragePolicyInternal - enc.encodePolicyFn = enc.encodePolicyInternal - - return enc -} - -func (enc *baseEncoder) encoder() BufferedEncoder { return enc.bufEncoder } -func (enc *baseEncoder) err() error { return enc.encodeErr } -func (enc *baseEncoder) resetData() { enc.bufEncoder.Reset() } -func (enc *baseEncoder) encodeVersion(version int) { enc.encodeVarint(int64(version)) } -func (enc *baseEncoder) encodeObjectType(objType objectType) { enc.encodeVarint(int64(objType)) } -func (enc *baseEncoder) encodeNumObjectFields(numFields int) { enc.encodeArrayLen(numFields) } -func (enc *baseEncoder) encodeRawID(id id.RawID) { enc.encodeBytes([]byte(id)) } -func (enc *baseEncoder) encodeVarint(value int64) { enc.encodeVarintFn(value) } -func (enc *baseEncoder) encodeBool(value bool) { enc.encodeBoolFn(value) } -func (enc *baseEncoder) encodeFloat64(value float64) { enc.encodeFloat64Fn(value) } -func (enc *baseEncoder) encodeBytes(value []byte) { enc.encodeBytesFn(value) } -func (enc *baseEncoder) encodeBytesLen(value int) { enc.encodeBytesLenFn(value) } -func (enc *baseEncoder) encodeArrayLen(value int) { enc.encodeArrayLenFn(value) } -func (enc *baseEncoder) encodeStoragePolicy(p policy.StoragePolicy) { enc.encodeStoragePolicyFn(p) } -func (enc *baseEncoder) encodePolicy(p policy.Policy) { enc.encodePolicyFn(p) } - -func (enc *baseEncoder) reset(encoder BufferedEncoder) { - enc.bufEncoder = encoder - enc.encodeErr = nil -} - -func (enc *baseEncoder) encodeChunkedID(id id.ChunkedID) { - enc.encodeBytesLen(len(id.Prefix) + len(id.Data) + len(id.Suffix)) - enc.writeRaw(id.Prefix) - enc.writeRaw(id.Data) - enc.writeRaw(id.Suffix) -} - -func (enc *baseEncoder) encodePolicyInternal(p policy.Policy) { - enc.encodeNumObjectFields(numFieldsForType(policyType)) - enc.encodeStoragePolicyFn(p.StoragePolicy) - enc.encodeCompressedAggregationTypes(p.AggregationID) -} - -func (enc *baseEncoder) encodeCompressedAggregationTypes(aggTypes aggregation.ID) { - if aggTypes.IsDefault() { - enc.encodeNumObjectFields(numFieldsForType(defaultAggregationID)) - enc.encodeObjectType(defaultAggregationID) - return - } - - if aggregation.IDLen == 1 { - enc.encodeNumObjectFields(numFieldsForType(shortAggregationID)) - enc.encodeObjectType(shortAggregationID) - enc.encodeVarintFn(int64(aggTypes[0])) - return - } - - // NB(cw): Only reachable after we start to support more than 63 aggregation types - enc.encodeNumObjectFields(numFieldsForType(longAggregationID)) - enc.encodeObjectType(longAggregationID) - enc.encodeArrayLen(aggregation.IDLen) - for _, v := range aggTypes { - enc.encodeVarint(int64(v)) - } -} - -func (enc *baseEncoder) encodeStoragePolicyInternal(p policy.StoragePolicy) { - enc.encodeNumObjectFields(numFieldsForType(storagePolicyType)) - enc.encodeResolution(p.Resolution()) - enc.encodeRetention(p.Retention()) -} - -func (enc *baseEncoder) encodeResolution(resolution policy.Resolution) { - if enc.encodeErr != nil { - return - } - // If this is a known resolution, only encode its corresponding value. - if resolutionValue, err := policy.ValueFromResolution(resolution); err == nil { - enc.encodeNumObjectFields(numFieldsForType(knownResolutionType)) - enc.encodeObjectType(knownResolutionType) - enc.encodeVarintFn(int64(resolutionValue)) - return - } - // Otherwise encode the entire resolution object. - // TODO(xichen): validate the resolution before putting it on the wire. - enc.encodeNumObjectFields(numFieldsForType(unknownResolutionType)) - enc.encodeObjectType(unknownResolutionType) - enc.encodeVarintFn(int64(resolution.Window)) - enc.encodeVarintFn(int64(resolution.Precision)) -} - -func (enc *baseEncoder) encodeRetention(retention policy.Retention) { - if enc.encodeErr != nil { - return - } - // If this is a known retention, only encode its corresponding value. - if retentionValue, err := policy.ValueFromRetention(retention); err == nil { - enc.encodeNumObjectFields(numFieldsForType(knownRetentionType)) - enc.encodeObjectType(knownRetentionType) - enc.encodeVarintFn(int64(retentionValue)) - return - } - // Otherwise encode the entire retention object. - // TODO(xichen): validate the retention before putting it on the wire. - enc.encodeNumObjectFields(numFieldsForType(unknownRetentionType)) - enc.encodeObjectType(unknownRetentionType) - enc.encodeVarintFn(int64(retention)) -} - -// NB(xichen): the underlying msgpack encoder implementation -// always cast an integer value to an int64 and encodes integer -// values as varints, regardless of the actual integer type. -func (enc *baseEncoder) encodeVarintInternal(value int64) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeInt64(value) -} - -func (enc *baseEncoder) encodeBoolInternal(value bool) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeBool(value) -} - -func (enc *baseEncoder) encodeFloat64Internal(value float64) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeFloat64(value) -} - -func (enc *baseEncoder) encodeBytesInternal(value []byte) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeBytes(value) -} - -func (enc *baseEncoder) encodeBytesLenInternal(value int) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeBytesLen(value) -} - -func (enc *baseEncoder) encodeArrayLenInternal(value int) { - if enc.encodeErr != nil { - return - } - enc.encodeErr = enc.bufEncoder.EncodeArrayLen(value) -} - -func (enc *baseEncoder) writeRaw(buf []byte) { - if enc.encodeErr != nil { - return - } - _, enc.encodeErr = enc.bufEncoder.Buffer().Write(buf) -} diff --git a/src/metrics/encoding/msgpack/base_iterator.go b/src/metrics/encoding/msgpack/base_iterator.go deleted file mode 100644 index e86045dd23..0000000000 --- a/src/metrics/encoding/msgpack/base_iterator.go +++ /dev/null @@ -1,342 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bufio" - "bytes" - "fmt" - "io" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - msgpack "gopkg.in/vmihailenco/msgpack.v2" -) - -var ( - emptyReader *bytes.Buffer -) - -// baseIterator is the base iterator that provides common decoding APIs. -type baseIterator struct { - readerBufferSize int - bufReader bufReader - decoder *msgpack.Decoder - decodeErr error -} - -func newBaseIterator(reader io.Reader, readerBufferSize int) iteratorBase { - // NB(xichen): if reader is not a bufReader, the underlying msgpack decoder - // creates a bufio.Reader wrapping the reader. By converting the reader to a - // bufReader, it is guaranteed that the reader passed to the decoder is the one - // used for reading and buffering the underlying data. - bufReader := toBufReader(reader, readerBufferSize) - return &baseIterator{ - readerBufferSize: readerBufferSize, - bufReader: bufReader, - decoder: msgpack.NewDecoder(bufReader), - } -} - -func (it *baseIterator) reset(reader io.Reader) { - bufReader := toBufReader(reader, it.readerBufferSize) - it.bufReader = bufReader - it.decoder.Reset(bufReader) // nolint: errcheck - it.decodeErr = nil -} - -func (it *baseIterator) err() error { return it.decodeErr } -func (it *baseIterator) setErr(err error) { it.decodeErr = err } -func (it *baseIterator) reader() bufReader { return it.bufReader } - -func (it *baseIterator) decodePolicy() policy.Policy { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(policyType) - if !ok { - return policy.DefaultPolicy - } - sp := it.decodeStoragePolicy() - aggTypes := it.decodeCompressedAggregationTypes() - it.skip(numActualFields - numExpectedFields) - return policy.NewPolicy(sp, aggTypes) -} - -func (it *baseIterator) decodeCompressedAggregationTypes() aggregation.ID { - numActualFields := it.decodeNumObjectFields() - aggregationEncodeType := it.decodeObjectType() - numExpectedFields, ok := it.checkExpectedNumFieldsForType( - aggregationEncodeType, - numActualFields, - ) - if !ok { - return aggregation.DefaultID - } - - var aggTypes aggregation.ID - switch aggregationEncodeType { - case defaultAggregationID: - case shortAggregationID: - value := it.decodeVarint() - aggTypes[0] = uint64(value) - case longAggregationID: - numValues := it.decodeArrayLen() - if numValues > aggregation.IDLen { - it.decodeErr = fmt.Errorf("invalid CompressedAggregationType length: %d", numValues) - return aggTypes - } - - for i := 0; i < numValues; i++ { - aggTypes[i] = uint64(it.decodeVarint()) - } - default: - it.decodeErr = fmt.Errorf("unrecognized aggregation encode type %v", aggregationEncodeType) - return aggTypes - } - it.skip(numActualFields - numExpectedFields) - return aggTypes -} - -func (it *baseIterator) decodeStoragePolicy() policy.StoragePolicy { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(storagePolicyType) - if !ok { - return policy.EmptyStoragePolicy - } - resolution := it.decodeResolution() - retention := it.decodeRetention() - sp := policy.NewStoragePolicy(resolution.Window, resolution.Precision, time.Duration(retention)) - it.skip(numActualFields - numExpectedFields) - return sp -} - -func (it *baseIterator) decodeResolution() policy.Resolution { - numActualFields := it.decodeNumObjectFields() - resolutionType := it.decodeObjectType() - numExpectedFields, ok := it.checkExpectedNumFieldsForType( - resolutionType, - numActualFields, - ) - if !ok { - return policy.EmptyResolution - } - switch resolutionType { - case knownResolutionType: - resolutionValue := policy.ResolutionValue(it.decodeVarint()) - if !resolutionValue.IsValid() { - it.decodeErr = fmt.Errorf("invalid resolution value %v", resolutionValue) - return policy.EmptyResolution - } - it.skip(numActualFields - numExpectedFields) - if it.decodeErr != nil { - return policy.EmptyResolution - } - resolution, err := resolutionValue.Resolution() - it.decodeErr = err - return resolution - case unknownResolutionType: - window := time.Duration(it.decodeVarint()) - precision := xtime.Unit(it.decodeVarint()) - if it.decodeErr != nil { - return policy.EmptyResolution - } - if !precision.IsValid() { - it.decodeErr = fmt.Errorf("invalid precision %v", precision) - return policy.EmptyResolution - } - it.skip(numActualFields - numExpectedFields) - return policy.Resolution{Window: window, Precision: precision} - default: - it.decodeErr = fmt.Errorf("unrecognized resolution type %v", resolutionType) - return policy.EmptyResolution - } -} - -func (it *baseIterator) decodeRetention() policy.Retention { - numActualFields := it.decodeNumObjectFields() - retentionType := it.decodeObjectType() - numExpectedFields, ok := it.checkExpectedNumFieldsForType( - retentionType, - numActualFields, - ) - if !ok { - return policy.EmptyRetention - } - switch retentionType { - case knownRetentionType: - retentionValue := policy.RetentionValue(it.decodeVarint()) - if !retentionValue.IsValid() { - it.decodeErr = fmt.Errorf("invalid retention value %v", retentionValue) - return policy.EmptyRetention - } - it.skip(numActualFields - numExpectedFields) - if it.decodeErr != nil { - return policy.EmptyRetention - } - retention, err := retentionValue.Retention() - it.decodeErr = err - return retention - case unknownRetentionType: - retention := policy.Retention(it.decodeVarint()) - it.skip(numActualFields - numExpectedFields) - return retention - default: - it.decodeErr = fmt.Errorf("unrecognized retention type %v", retentionType) - return policy.EmptyRetention - } -} - -func (it *baseIterator) decodeVersion() int { - return int(it.decodeVarint()) -} - -func (it *baseIterator) decodeObjectType() objectType { - ot := objectType(it.decodeVarint()) - if it.decodeErr != nil { - return unknownType - } - if !ot.isValid() { - it.decodeErr = fmt.Errorf("invalid object type %v", ot) - return unknownType - } - return ot -} - -func (it *baseIterator) decodeNumObjectFields() int { - return it.decodeArrayLen() -} - -func (it *baseIterator) decodeRawID() id.RawID { - return id.RawID(it.decodeBytes()) -} - -// NB(xichen): the underlying msgpack decoder implementation -// always decodes an int64 and looks at the actual decoded -// value to determine the width of the integer (a.k.a. varint -// decoding). -func (it *baseIterator) decodeVarint() int64 { - if it.decodeErr != nil { - return 0 - } - value, err := it.decoder.DecodeInt64() - it.decodeErr = err - return value -} - -func (it *baseIterator) decodeBool() bool { - if it.decodeErr != nil { - return false - } - value, err := it.decoder.DecodeBool() - it.decodeErr = err - return value -} - -func (it *baseIterator) decodeFloat64() float64 { - if it.decodeErr != nil { - return 0.0 - } - value, err := it.decoder.DecodeFloat64() - it.decodeErr = err - return value -} - -func (it *baseIterator) decodeBytes() []byte { - if it.decodeErr != nil { - return nil - } - value, err := it.decoder.DecodeBytes() - it.decodeErr = err - return value -} - -func (it *baseIterator) decodeBytesLen() int { - if it.decodeErr != nil { - return 0 - } - bytesLen, err := it.decoder.DecodeBytesLen() - it.decodeErr = err - return bytesLen -} - -func (it *baseIterator) decodeArrayLen() int { - if it.decodeErr != nil { - return 0 - } - value, err := it.decoder.DecodeArrayLen() - it.decodeErr = err - return value -} - -func (it *baseIterator) skip(numFields int) { - if it.decodeErr != nil { - return - } - if numFields < 0 { - it.decodeErr = fmt.Errorf("number of fields to skip is %d", numFields) - return - } - // Otherwise we skip any unexpected extra fields. - for i := 0; i < numFields; i++ { - if err := it.decoder.Skip(); err != nil { - it.decodeErr = err - return - } - } -} - -func (it *baseIterator) checkNumFieldsForType(objType objectType) (int, int, bool) { - numActualFields := it.decodeNumObjectFields() - numExpectedFields, ok := it.checkExpectedNumFieldsForType(objType, numActualFields) - return numExpectedFields, numActualFields, ok -} - -func (it *baseIterator) checkExpectedNumFieldsForType( - objType objectType, - numActualFields int, -) (int, bool) { - if it.decodeErr != nil { - return 0, false - } - numExpectedFields := numFieldsForType(objType) - if numExpectedFields > numActualFields { - it.decodeErr = fmt.Errorf("number of fields mismatch: expected %d actual %d", numExpectedFields, numActualFields) - return 0, false - } - return numExpectedFields, true -} - -// bufReader is a buffered reader. -type bufReader interface { - io.Reader - - ReadByte() (byte, error) - UnreadByte() error -} - -func toBufReader(reader io.Reader, readerBufferSize int) bufReader { - bufReader, ok := reader.(bufReader) - if ok { - return bufReader - } - return bufio.NewReaderSize(reader, readerBufferSize) -} diff --git a/src/metrics/encoding/msgpack/base_test.go b/src/metrics/encoding/msgpack/base_test.go deleted file mode 100644 index fb2fd39170..0000000000 --- a/src/metrics/encoding/msgpack/base_test.go +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright (c) 2018 Uber Technologies, Inc. -// -// Permission is hereby granted, free of charge, to any person obtaining a copy -// of this software and associated documentation files (the "Software"), to deal -// in the Software without restriction, including without limitation the rights -// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -// copies of the Software, and to permit persons to whom the Software is -// furnished to do so, subject to the following conditions: -// -// The above copyright notice and this permission notice shall be included in -// all copies or substantial portions of the Software. -// -// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -// THE SOFTWARE. - -package msgpack - -import ( - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -func TestAggregationTypesRoundTrip(t *testing.T) { - inputs := []aggregation.ID{ - aggregation.DefaultID, - aggregation.ID{5}, - aggregation.ID{100}, - aggregation.ID{12345}, - } - - for _, input := range inputs { - enc := newBaseEncoder(NewBufferedEncoder()).(*baseEncoder) - it := newBaseIterator(enc.bufEncoder.Buffer(), 16).(*baseIterator) - - enc.encodeCompressedAggregationTypes(input) - r := it.decodeCompressedAggregationTypes() - require.Equal(t, input, r) - } -} - -func TestUnaggregatedPolicyRoundTrip(t *testing.T) { - inputs := []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(10*time.Second, xtime.Second, 24*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Second, xtime.Second, 2*24*time.Hour), aggregation.ID{8}), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Second, xtime.Second, 24*time.Hour), aggregation.ID{100}), - } - - for _, input := range inputs { - enc := newBaseEncoder(NewBufferedEncoder()).(*baseEncoder) - enc.encodePolicy(input) - - it := newBaseIterator(enc.bufEncoder.Buffer(), 16).(*baseIterator) - r := it.decodePolicy() - require.Equal(t, input, r) - } -} diff --git a/src/metrics/encoding/msgpack/buffered_encoder.go b/src/metrics/encoding/msgpack/buffered_encoder.go deleted file mode 100644 index 1bbcf1d2b7..0000000000 --- a/src/metrics/encoding/msgpack/buffered_encoder.go +++ /dev/null @@ -1,74 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - - msgpack "gopkg.in/vmihailenco/msgpack.v2" -) - -type bufferedEncoder struct { - *msgpack.Encoder - - buf bytes.Buffer - closed bool - pool BufferedEncoderPool -} - -// NewBufferedEncoder creates a new buffered encoder. -func NewBufferedEncoder() BufferedEncoder { - return NewPooledBufferedEncoder(nil) -} - -// NewPooledBufferedEncoder creates a new pooled buffered encoder. -func NewPooledBufferedEncoder(p BufferedEncoderPool) BufferedEncoder { - return NewPooledBufferedEncoderSize(p, 0) -} - -// NewPooledBufferedEncoderSize creates a new pooled buffered encoder -// with an initial buffer size. -func NewPooledBufferedEncoderSize(p BufferedEncoderPool, size int) BufferedEncoder { - var enc bufferedEncoder - enc.buf.Grow(size) - enc.Encoder = msgpack.NewEncoder(&enc.buf) - enc.pool = p - return &enc -} - -func (enc *bufferedEncoder) Buffer() *bytes.Buffer { return &enc.buf } - -func (enc *bufferedEncoder) Bytes() []byte { return enc.buf.Bytes() } - -func (enc *bufferedEncoder) Reset() { - enc.closed = false - enc.buf.Truncate(0) -} - -func (enc *bufferedEncoder) Close() { - if enc.closed { - return - } - enc.closed = true - if enc.pool != nil { - enc.pool.Put(enc) - } -} diff --git a/src/metrics/encoding/msgpack/buffered_encoder_pool.go b/src/metrics/encoding/msgpack/buffered_encoder_pool.go deleted file mode 100644 index 93256d96aa..0000000000 --- a/src/metrics/encoding/msgpack/buffered_encoder_pool.go +++ /dev/null @@ -1,56 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import "github.com/m3db/m3/src/x/pool" - -type bufferedEncoderPool struct { - maxCapacity int - pool pool.ObjectPool -} - -// NewBufferedEncoderPool creates a new pool for buffered encoders. -func NewBufferedEncoderPool(opts BufferedEncoderPoolOptions) BufferedEncoderPool { - if opts == nil { - opts = NewBufferedEncoderPoolOptions() - } - return &bufferedEncoderPool{ - maxCapacity: opts.MaxCapacity(), - pool: pool.NewObjectPool(opts.ObjectPoolOptions()), - } -} - -func (p *bufferedEncoderPool) Init(alloc BufferedEncoderAlloc) { - p.pool.Init(func() interface{} { - return alloc() - }) -} - -func (p *bufferedEncoderPool) Get() BufferedEncoder { - return p.pool.Get().(BufferedEncoder) -} - -func (p *bufferedEncoderPool) Put(encoder BufferedEncoder) { - if encoder.Buffer().Cap() > p.maxCapacity { - return - } - p.pool.Put(encoder) -} diff --git a/src/metrics/encoding/msgpack/buffered_encoder_pool_test.go b/src/metrics/encoding/msgpack/buffered_encoder_pool_test.go deleted file mode 100644 index f6bf39226d..0000000000 --- a/src/metrics/encoding/msgpack/buffered_encoder_pool_test.go +++ /dev/null @@ -1,81 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "testing" - - "github.com/m3db/m3/src/x/pool" - - "github.com/stretchr/testify/require" -) - -func TestBufferedEncoderPool(t *testing.T) { - poolOpts := pool.NewObjectPoolOptions().SetSize(1) - opts := NewBufferedEncoderPoolOptions(). - SetObjectPoolOptions(poolOpts) - - p := NewBufferedEncoderPool(opts) - p.Init(func() BufferedEncoder { - return NewPooledBufferedEncoder(p) - }) - - // Retrieve an encoder from the pool. - encoder := p.Get() - encoder.Buffer().Write([]byte{1, 2, 3}) - require.Equal(t, 3, encoder.Buffer().Len()) - - // Closing the encoder should put it back to the pool. - encoder.Close() - - // Retrieve the encoder and assert it's the same encoder. - encoder = p.Get() - require.Equal(t, 3, encoder.Buffer().Len()) - - // Reset the encoder and assert it's been reset. - encoder.Reset() - require.Equal(t, 0, encoder.Buffer().Len()) -} - -func TestBufferedEncoderPoolMaxCapacity(t *testing.T) { - poolOpts := pool.NewObjectPoolOptions().SetSize(1) - opts := NewBufferedEncoderPoolOptions(). - SetMaxCapacity(2). - SetObjectPoolOptions(poolOpts) - - p := NewBufferedEncoderPool(opts) - p.Init(func() BufferedEncoder { - return NewPooledBufferedEncoder(p) - }) - - // Retrieve an encoder from the pool. - encoder := p.Get() - encoder.Buffer().Write([]byte{1, 2, 3}) - require.Equal(t, 3, encoder.Buffer().Len()) - - // Closing the encoder should put it back to the pool. - encoder.Close() - - // Retrieve an encoder and assert it's a different encoder since - // the previous one exceeded the maximum capacity of the pool. - encoder = p.Get() - require.Equal(t, 0, encoder.Buffer().Cap()) -} diff --git a/src/metrics/encoding/msgpack/buffered_encoder_test.go b/src/metrics/encoding/msgpack/buffered_encoder_test.go deleted file mode 100644 index cb929a271d..0000000000 --- a/src/metrics/encoding/msgpack/buffered_encoder_test.go +++ /dev/null @@ -1,72 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "testing" - - "github.com/stretchr/testify/require" -) - -func TestBufferedEncoderReset(t *testing.T) { - encoder := testBufferedEncoder() - inputs := []interface{}{1, 2.0, "foo", byte(8)} - - // Encode for the first time. - for _, input := range inputs { - err := encoder.Encode(input) - require.NoError(t, err) - } - encoded := encoder.Bytes() - results := make([]byte, len(encoded)) - copy(results, encoded) - - // Reset the encoder. - encoder.Reset() - - // Encode for the second time. - for _, input := range inputs { - err := encoder.Encode(input) - require.NoError(t, err) - } - encoded = encoder.Bytes() - results2 := make([]byte, len(encoded)) - copy(results2, encoded) - - require.Equal(t, results, results2) -} - -func TestBufferedEncoderClose(t *testing.T) { - encoder := testBufferedEncoder() - require.False(t, encoder.closed) - - // Close the encoder should set the flag. - encoder.Close() - require.True(t, encoder.closed) - - // Close the encoder again should be a no-op. - encoder.Close() - require.True(t, encoder.closed) -} - -func testBufferedEncoder() *bufferedEncoder { - return NewBufferedEncoder().(*bufferedEncoder) -} diff --git a/src/metrics/encoding/msgpack/options.go b/src/metrics/encoding/msgpack/options.go deleted file mode 100644 index 3c315bf071..0000000000 --- a/src/metrics/encoding/msgpack/options.go +++ /dev/null @@ -1,199 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "math" - - xpool "github.com/m3db/m3/src/x/pool" -) - -const ( - // The maximum capacity of buffers that can be returned to the buffered - // encoder pool. - defaultBufferedEncoderPoolMaxCapacity = math.MaxInt64 - - // Whether the iterator should ignore higher-than-supported version - // by default for unaggregated iterator. - defaultUnaggregatedIgnoreHigherVersion = false - - // Default reader buffer size for the unaggregated iterator. - defaultUnaggregatedReaderBufferSize = 1440 - - // Whether a float slice is considered a "large" slice and therefore - // resort to the pool for allocating that slice. - defaultLargeFloatsSize = 1024 - - // Whether the iterator should ignore higher-than-supported version - // by default for aggregated iterator. - defaultAggregatedIgnoreHigherVersion = false - - // Default reader buffer size for the aggregated iterator. - defaultAggregatedReaderBufferSize = 1440 -) - -type bufferedEncoderPoolOptions struct { - maxCapacity int - poolOpts xpool.ObjectPoolOptions -} - -// NewBufferedEncoderPoolOptions creates a new set of buffered encoder pool options. -func NewBufferedEncoderPoolOptions() BufferedEncoderPoolOptions { - return &bufferedEncoderPoolOptions{ - maxCapacity: defaultBufferedEncoderPoolMaxCapacity, - poolOpts: xpool.NewObjectPoolOptions(), - } -} - -func (o *bufferedEncoderPoolOptions) SetMaxCapacity(value int) BufferedEncoderPoolOptions { - opts := *o - opts.maxCapacity = value - return &opts -} - -func (o *bufferedEncoderPoolOptions) MaxCapacity() int { - return o.maxCapacity -} - -func (o *bufferedEncoderPoolOptions) SetObjectPoolOptions(value xpool.ObjectPoolOptions) BufferedEncoderPoolOptions { - opts := *o - opts.poolOpts = value - return &opts -} - -func (o *bufferedEncoderPoolOptions) ObjectPoolOptions() xpool.ObjectPoolOptions { - return o.poolOpts -} - -type unaggregatedIteratorOptions struct { - ignoreHigherVersion bool - readerBufferSize int - largeFloatsSize int - largeFloatsPool xpool.FloatsPool - iteratorPool UnaggregatedIteratorPool -} - -// NewUnaggregatedIteratorOptions creates a new set of unaggregated iterator options. -func NewUnaggregatedIteratorOptions() UnaggregatedIteratorOptions { - largeFloatsPool := xpool.NewFloatsPool(nil, nil) - largeFloatsPool.Init() - - return &unaggregatedIteratorOptions{ - ignoreHigherVersion: defaultUnaggregatedIgnoreHigherVersion, - readerBufferSize: defaultUnaggregatedReaderBufferSize, - largeFloatsSize: defaultLargeFloatsSize, - largeFloatsPool: largeFloatsPool, - } -} - -func (o *unaggregatedIteratorOptions) SetIgnoreHigherVersion(value bool) UnaggregatedIteratorOptions { - opts := *o - opts.ignoreHigherVersion = value - return &opts -} - -func (o *unaggregatedIteratorOptions) IgnoreHigherVersion() bool { - return o.ignoreHigherVersion -} - -func (o *unaggregatedIteratorOptions) SetReaderBufferSize(value int) UnaggregatedIteratorOptions { - opts := *o - opts.readerBufferSize = value - return &opts -} - -func (o *unaggregatedIteratorOptions) ReaderBufferSize() int { - return o.readerBufferSize -} - -func (o *unaggregatedIteratorOptions) SetLargeFloatsSize(value int) UnaggregatedIteratorOptions { - opts := *o - opts.largeFloatsSize = value - return &opts -} - -func (o *unaggregatedIteratorOptions) LargeFloatsSize() int { - return o.largeFloatsSize -} - -func (o *unaggregatedIteratorOptions) SetLargeFloatsPool(value xpool.FloatsPool) UnaggregatedIteratorOptions { - opts := *o - opts.largeFloatsPool = value - return &opts -} - -func (o *unaggregatedIteratorOptions) LargeFloatsPool() xpool.FloatsPool { - return o.largeFloatsPool -} - -func (o *unaggregatedIteratorOptions) SetIteratorPool(value UnaggregatedIteratorPool) UnaggregatedIteratorOptions { - opts := *o - opts.iteratorPool = value - return &opts -} - -func (o *unaggregatedIteratorOptions) IteratorPool() UnaggregatedIteratorPool { - return o.iteratorPool -} - -type aggregatedIteratorOptions struct { - ignoreHigherVersion bool - readerBufferSize int - iteratorPool AggregatedIteratorPool -} - -// NewAggregatedIteratorOptions creates a new set of aggregated iterator options. -func NewAggregatedIteratorOptions() AggregatedIteratorOptions { - return &aggregatedIteratorOptions{ - ignoreHigherVersion: defaultAggregatedIgnoreHigherVersion, - readerBufferSize: defaultAggregatedReaderBufferSize, - } -} - -func (o *aggregatedIteratorOptions) SetIgnoreHigherVersion(value bool) AggregatedIteratorOptions { - opts := *o - opts.ignoreHigherVersion = value - return &opts -} - -func (o *aggregatedIteratorOptions) IgnoreHigherVersion() bool { - return o.ignoreHigherVersion -} - -func (o *aggregatedIteratorOptions) SetReaderBufferSize(value int) AggregatedIteratorOptions { - opts := *o - opts.readerBufferSize = value - return &opts -} - -func (o *aggregatedIteratorOptions) ReaderBufferSize() int { - return o.readerBufferSize -} - -func (o *aggregatedIteratorOptions) SetIteratorPool(value AggregatedIteratorPool) AggregatedIteratorOptions { - opts := *o - opts.iteratorPool = value - return &opts -} - -func (o *aggregatedIteratorOptions) IteratorPool() AggregatedIteratorPool { - return o.iteratorPool -} diff --git a/src/metrics/encoding/msgpack/raw_metric.go b/src/metrics/encoding/msgpack/raw_metric.go deleted file mode 100644 index f81944235e..0000000000 --- a/src/metrics/encoding/msgpack/raw_metric.go +++ /dev/null @@ -1,192 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "fmt" - "io" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/metric/id" -) - -var ( - emptyMetric aggregated.Metric -) - -type readBytesFn func(start int, n int) []byte - -// rawMetric is a raw metric. -type rawMetric struct { - data []byte // raw data containing encoded metric. - it iteratorBase // base iterator for lazily decoding metric fields. - metric aggregated.Metric // current metric. - idDecoded bool // whether id has been decoded. - timeDecoded bool // whether time has been decoded. - valueDecoded bool // whether value has been decoded. - readBytesFn readBytesFn // reading bytes function. -} - -// NewRawMetric creates a new raw metric. -func NewRawMetric(data []byte, readerBufferSize int) aggregated.RawMetric { - reader := bytes.NewReader(data) - m := &rawMetric{ - data: data, - it: newBaseIterator(reader, readerBufferSize), - } - m.readBytesFn = m.readBytes - return m -} - -func (m *rawMetric) ID() (id.RawID, error) { - m.decodeID() - if err := m.it.err(); err != nil { - return nil, err - } - return m.metric.ID, nil -} - -func (m *rawMetric) TimeNanos() (int64, error) { - m.decodeID() - m.decodeTime() - if err := m.it.err(); err != nil { - return 0, err - } - return m.metric.TimeNanos, nil -} - -func (m *rawMetric) Value() (float64, error) { - m.decodeID() - m.decodeTime() - m.decodeValue() - if err := m.it.err(); err != nil { - return 0.0, err - } - return m.metric.Value, nil -} - -func (m *rawMetric) Metric() (aggregated.Metric, error) { - m.decodeID() - m.decodeTime() - m.decodeValue() - if err := m.it.err(); err != nil { - return emptyMetric, err - } - return m.metric, nil -} - -func (m *rawMetric) Bytes() []byte { - return m.data -} - -func (m *rawMetric) Reset(data []byte) { - m.metric = emptyMetric - m.idDecoded = false - m.timeDecoded = false - m.valueDecoded = false - m.data = data - m.reader().Reset(data) - m.it.reset(m.reader()) -} - -// NB(xichen): decodeID decodes the ID without making a copy -// of the bytes stored in the buffer. The decoded ID is a slice -// of the internal buffer, which remains valid until the buffered -// data become invalid (e.g. when Reset() is called). -func (m *rawMetric) decodeID() { - if m.it.err() != nil || m.idDecoded { - return - } - version := m.it.decodeVersion() - if m.it.err() != nil { - return - } - if version > metricVersion { - err := fmt.Errorf("metric version received %d is higher than supported version %d", version, metricVersion) - m.it.setErr(err) - return - } - _, _, ok := m.it.checkNumFieldsForType(metricType) - if !ok { - return - } - idLen := m.it.decodeBytesLen() - if m.it.err() != nil { - return - } - // NB(xichen): DecodeBytesLen() returns -1 if the byte slice is nil. - if idLen == -1 { - m.metric.ID = nil - m.idDecoded = true - return - } - numRead := len(m.data) - m.reader().Len() - m.metric.ID = m.readBytesFn(numRead, idLen) - if m.it.err() != nil { - return - } - m.idDecoded = true -} - -func (m *rawMetric) decodeTime() { - if m.it.err() != nil || m.timeDecoded { - return - } - timeNanos := m.it.decodeVarint() - if m.it.err() != nil { - return - } - m.metric.TimeNanos = timeNanos - m.timeDecoded = true -} - -func (m *rawMetric) decodeValue() { - if m.it.err() != nil || m.valueDecoded { - return - } - v := m.it.decodeFloat64() - if m.it.err() != nil { - return - } - m.metric.Value = v - m.valueDecoded = true -} - -func (m *rawMetric) reader() *bytes.Reader { - return m.it.reader().(*bytes.Reader) -} - -func (m *rawMetric) readBytes(start int, n int) []byte { - numBytes := len(m.data) - if n < 0 || start < 0 || start+n > numBytes { - err := fmt.Errorf("invalid start %d and length %d, numBytes=%d", start, n, numBytes) - m.it.setErr(err) - return nil - } - // Advance the internal buffer index. - _, err := m.reader().Seek(int64(n), io.SeekCurrent) - if err != nil { - m.it.setErr(err) - return nil - } - return m.data[start : start+n] -} diff --git a/src/metrics/encoding/msgpack/raw_metric_test.go b/src/metrics/encoding/msgpack/raw_metric_test.go deleted file mode 100644 index 4afb1e9317..0000000000 --- a/src/metrics/encoding/msgpack/raw_metric_test.go +++ /dev/null @@ -1,283 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "errors" - "io" - "testing" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/policy" - - "github.com/stretchr/testify/require" -) - -var ( - testRawMetricData = []byte("foodg") - errTestDecodeRawMetric = errors.New("foo") -) - -func TestRawMetricDecodeIDExistingError(t *testing.T) { - m := testRawMetric() - m.it.setErr(errTestDecodeRawMetric) - _, err := m.ID() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeIDVersionError(t *testing.T) { - m := testRawMetric() - m.it.(*mockBaseIterator).decodeVersionFn = func() int { - return metricVersion + 1 - } - _, err := m.ID() - require.Error(t, err) -} - -func TestRawMetricDecodeIDBytesLenDecodeError(t *testing.T) { - m := testRawMetric() - m.it.(*mockBaseIterator).decodeBytesLenFn = func() int { - m.it.setErr(errTestDecodeRawMetric) - return 0 - } - _, err := m.ID() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeIDBytesLenOutOfRange(t *testing.T) { - m := testRawMetric() - m.it.(*mockBaseIterator).decodeBytesLenFn = func() int { return -100 } - _, err := m.ID() - require.Error(t, err) - - m = testRawMetric() - m.it.(*mockBaseIterator).decodeBytesLenFn = func() int { - return len(testRawMetricData) + 1 - } - _, err = m.ID() - require.Error(t, err) -} - -func TestRawMetricDecodeIDSuccess(t *testing.T) { - m := testRawMetric() - id, err := m.ID() - require.NoError(t, err) - require.Equal(t, testMetric.ID, id) - require.True(t, m.idDecoded) - - // Get ID again to make sure we don't re-decode the ID. - id, err = m.ID() - require.NoError(t, err) - require.Equal(t, testMetric.ID, id) -} - -func TestRawMetricDecodeTimestampExistingError(t *testing.T) { - m := testRawMetric() - m.it.setErr(errTestDecodeRawMetric) - _, err := m.TimeNanos() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeTimestampDecodeError(t *testing.T) { - m := testRawMetric() - m.it.(*mockBaseIterator).decodeVarintFn = func() int64 { - m.it.setErr(errTestDecodeRawMetric) - return 0 - } - _, err := m.TimeNanos() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeTimestampSuccess(t *testing.T) { - m := testRawMetric() - timeNanos, err := m.TimeNanos() - require.NoError(t, err) - require.Equal(t, testMetric.TimeNanos, timeNanos) - require.True(t, m.timeDecoded) - - // Get timestamp again to make sure we don't re-decode the timestamp. - require.NoError(t, err) - require.Equal(t, testMetric.TimeNanos, timeNanos) -} - -func TestRawMetricDecodeValueExistingError(t *testing.T) { - m := testRawMetric() - m.it.setErr(errTestDecodeRawMetric) - _, err := m.Value() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeValueDecodeError(t *testing.T) { - m := testRawMetric() - m.it.(*mockBaseIterator).decodeFloat64Fn = func() float64 { - m.it.setErr(errTestDecodeRawMetric) - return 0 - } - _, err := m.Value() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeValueSuccess(t *testing.T) { - m := testRawMetric() - value, err := m.Value() - require.NoError(t, err) - require.Equal(t, testMetric.Value, value) - require.True(t, m.valueDecoded) - - value, err = m.Value() - require.NoError(t, err) - require.Equal(t, testMetric.Value, value) -} - -func TestRawMetricDecodeMetricExistingError(t *testing.T) { - m := testRawMetric() - m.it.setErr(errTestDecodeRawMetric) - _, err := m.Metric() - require.Equal(t, errTestDecodeRawMetric, err) -} - -func TestRawMetricDecodeMetricSuccess(t *testing.T) { - m := testRawMetric() - metric, err := m.Metric() - require.NoError(t, err) - require.Equal(t, testMetric, metric) - require.True(t, m.idDecoded) - require.True(t, m.timeDecoded) - require.True(t, m.valueDecoded) - - // Get metric again to make sure we don't re-decode the metric. - require.NoError(t, err) - require.Equal(t, testMetric, metric) -} - -func TestRawMetricBytes(t *testing.T) { - m := testRawMetric() - require.Equal(t, m.data, m.Bytes()) -} - -func TestRawMetricNilID(t *testing.T) { - r := NewRawMetric(nil, 16) - r.Reset(toRawMetric(t, emptyMetric).Bytes()) - decoded, err := r.ID() - require.NoError(t, err) - require.Nil(t, decoded) - require.True(t, r.(*rawMetric).idDecoded) -} - -func TestRawMetricReset(t *testing.T) { - metrics := []aggregated.Metric{ - {ID: id.RawID("foo"), TimeNanos: testMetric.TimeNanos, Value: 1.0}, - {ID: id.RawID("bar"), TimeNanos: testMetric.TimeNanos, Value: 2.3}, - {ID: id.RawID("baz"), TimeNanos: testMetric.TimeNanos, Value: 4234.234}, - } - rawMetric := NewRawMetric(nil, 16) - for i := 0; i < len(metrics); i++ { - rawMetric.Reset(toRawMetric(t, metrics[i]).Bytes()) - decoded, err := rawMetric.Metric() - require.NoError(t, err) - require.Equal(t, metrics[i], decoded) - } -} - -func TestRawMetricRoundtripStress(t *testing.T) { - metrics := []aggregated.Metric{ - {ID: id.RawID("foo"), TimeNanos: testMetric.TimeNanos, Value: 1.0}, - {ID: id.RawID("bar"), TimeNanos: testMetric.TimeNanos, Value: 2.3}, - {ID: id.RawID("baz"), TimeNanos: testMetric.TimeNanos, Value: 4234.234}, - } - var ( - inputs []aggregated.Metric - results []aggregated.Metric - numIter = 2 - ) - for i := 0; i < numIter; i++ { - input := metrics[i%len(metrics)] - inputs = append(inputs, input) - rawMetric := toRawMetric(t, input) - decoded, err := rawMetric.Metric() - require.NoError(t, err) - results = append(results, decoded) - } - require.Equal(t, inputs, results) -} - -type decodeVersionFn func() int -type decodeBytesLenFn func() int -type decodeVarintFn func() int64 -type decodeFloat64Fn func() float64 - -type mockBaseIterator struct { - bufReader bufReader - itErr error - decodeVersionFn decodeVersionFn - decodeBytesLenFn decodeBytesLenFn - decodeVarintFn decodeVarintFn - decodeFloat64Fn decodeFloat64Fn -} - -func (it *mockBaseIterator) reset(reader io.Reader) {} -func (it *mockBaseIterator) err() error { return it.itErr } -func (it *mockBaseIterator) setErr(err error) { it.itErr = err } -func (it *mockBaseIterator) reader() bufReader { return it.bufReader } -func (it *mockBaseIterator) decodeStoragePolicy() policy.StoragePolicy { - return policy.EmptyStoragePolicy -} -func (it *mockBaseIterator) decodeVersion() int { return it.decodeVersionFn() } -func (it *mockBaseIterator) decodeObjectType() objectType { return unknownType } -func (it *mockBaseIterator) decodeNumObjectFields() int { return 0 } -func (it *mockBaseIterator) decodeRawID() id.RawID { return nil } -func (it *mockBaseIterator) decodeVarint() int64 { return it.decodeVarintFn() } -func (it *mockBaseIterator) decodeBool() bool { return false } -func (it *mockBaseIterator) decodeFloat64() float64 { return it.decodeFloat64Fn() } -func (it *mockBaseIterator) decodeBytes() []byte { return nil } -func (it *mockBaseIterator) decodeBytesLen() int { return it.decodeBytesLenFn() } -func (it *mockBaseIterator) decodeArrayLen() int { return 0 } -func (it *mockBaseIterator) skip(numFields int) {} -func (it *mockBaseIterator) decodePolicy() policy.Policy { - return policy.DefaultPolicy -} - -func (it *mockBaseIterator) checkNumFieldsForType(objType objectType) (int, int, bool) { - return 0, 0, true -} - -func (it *mockBaseIterator) checkExpectedNumFieldsForType( - objType objectType, - numActualFields int, -) (int, bool) { - return 0, true -} - -func testRawMetric() *rawMetric { - mockIt := &mockBaseIterator{} - mockIt.decodeVersionFn = func() int { return metricVersion } - mockIt.decodeBytesLenFn = func() int { return len(testMetric.ID) } - mockIt.decodeVarintFn = func() int64 { return testMetric.TimeNanos } - mockIt.decodeFloat64Fn = func() float64 { return testMetric.Value } - mockIt.bufReader = bytes.NewReader(testRawMetricData) - - m := NewRawMetric(testRawMetricData, 16).(*rawMetric) - m.it = mockIt - - return m -} diff --git a/src/metrics/encoding/msgpack/schema.go b/src/metrics/encoding/msgpack/schema.go deleted file mode 100644 index 6aed5c2785..0000000000 --- a/src/metrics/encoding/msgpack/schema.go +++ /dev/null @@ -1,138 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -type objectType int - -const ( - // Current version for encoding unaggregated metrics. - unaggregatedVersion int = 1 - - // Current version for encoding aggregated metrics. - aggregatedVersion int = 1 - - // Current metric version. - metricVersion int = 1 -) - -// nolint: deadcode -const ( - unknownType objectType = iota - - // Root object type. - rootObjectType - - // Object types exposed to the encoder interface. - counterWithPoliciesListType - batchTimerWithPoliciesListType - gaugeWithPoliciesListType - rawMetricWithStoragePolicyType - - // Object types not exposed to the encoder interface. - counterType - timerType - gaugeType - metricType - defaultPoliciesListType - customPoliciesListType - stagedPoliciesType - storagePolicyType - knownResolutionType - unknownResolutionType - knownRetentionType - unknownRetentionType - defaultAggregationID - shortAggregationID - longAggregationID - policyType - - // Additional object types. - rawMetricWithStoragePolicyAndEncodeTimeType - - // Total number of object types. - numObjectTypes = iota - 1 -) - -const ( - numRootObjectFields = 2 - numCounterWithPoliciesListFields = 2 - numBatchTimerWithPoliciesListFields = 2 - numGaugeWithPoliciesListFields = 2 - numRawMetricWithStoragePolicyFields = 2 - numRawMetricWithStoragePolicyAndEncodeTimeFields = 3 - numCounterFields = 2 - numBatchTimerFields = 2 - numGaugeFields = 2 - numMetricFields = 3 - numDefaultStagedPoliciesListFields = 1 - numCustomStagedPoliciesListFields = 2 - numStagedPoliciesFields = 3 - numStoragePolicyFields = 2 - numKnownResolutionFields = 2 - numUnknownResolutionFields = 3 - numKnownRetentionFields = 2 - numDefaultAggregationIDFields = 1 - numShortAggregationIDFields = 2 - numLongAggregationIDFields = 2 - numPolicyFields = 2 -) - -func (ot objectType) isValid() bool { - return ot > unknownType && ot <= numObjectTypes -} - -// NB(xichen): use a slice instead of a map to avoid lookup overhead. -var numObjectFields []int - -func numFieldsForType(objType objectType) int { - return numObjectFields[int(objType)-1] -} - -func setNumFieldsForType(objType objectType, numFields int) { - numObjectFields[int(objType)-1] = numFields -} - -func init() { - numObjectFields = make([]int, int(numObjectTypes)) - - setNumFieldsForType(rootObjectType, numRootObjectFields) - setNumFieldsForType(counterWithPoliciesListType, numCounterWithPoliciesListFields) - setNumFieldsForType(batchTimerWithPoliciesListType, numBatchTimerWithPoliciesListFields) - setNumFieldsForType(gaugeWithPoliciesListType, numGaugeWithPoliciesListFields) - setNumFieldsForType(rawMetricWithStoragePolicyType, numRawMetricWithStoragePolicyFields) - setNumFieldsForType(rawMetricWithStoragePolicyAndEncodeTimeType, numRawMetricWithStoragePolicyAndEncodeTimeFields) - setNumFieldsForType(counterType, numCounterFields) - setNumFieldsForType(timerType, numBatchTimerFields) - setNumFieldsForType(gaugeType, numGaugeFields) - setNumFieldsForType(metricType, numMetricFields) - setNumFieldsForType(defaultPoliciesListType, numDefaultStagedPoliciesListFields) - setNumFieldsForType(customPoliciesListType, numCustomStagedPoliciesListFields) - setNumFieldsForType(stagedPoliciesType, numStagedPoliciesFields) - setNumFieldsForType(storagePolicyType, numStoragePolicyFields) - setNumFieldsForType(knownResolutionType, numKnownResolutionFields) - setNumFieldsForType(unknownResolutionType, numUnknownResolutionFields) - setNumFieldsForType(knownRetentionType, numKnownRetentionFields) - setNumFieldsForType(unknownRetentionType, numKnownRetentionFields) - setNumFieldsForType(defaultAggregationID, numDefaultAggregationIDFields) - setNumFieldsForType(shortAggregationID, numShortAggregationIDFields) - setNumFieldsForType(longAggregationID, numLongAggregationIDFields) - setNumFieldsForType(policyType, numPolicyFields) -} diff --git a/src/metrics/encoding/msgpack/types.go b/src/metrics/encoding/msgpack/types.go deleted file mode 100644 index b3b4b43d4f..0000000000 --- a/src/metrics/encoding/msgpack/types.go +++ /dev/null @@ -1,408 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "io" - - "github.com/m3db/m3/src/metrics/metric/aggregated" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - "github.com/m3db/m3/src/x/pool" -) - -// Buffer is a byte buffer. -type Buffer interface { - // Buffer returns the bytes buffer. - Buffer() *bytes.Buffer - - // Bytes returns the buffered bytes. - Bytes() []byte - - // Reset resets the buffer. - Reset() - - // Close closes the buffer. - Close() -} - -// Encoder is an encoder. -type Encoder interface { - // EncodeInt64 encodes an int64 value. - EncodeInt64(value int64) error - - // EncodeBool encodes a boolean value. - EncodeBool(value bool) error - - // EncodeFloat64 encodes a float64 value. - EncodeFloat64(value float64) error - - // EncodeBytes encodes a byte slice. - EncodeBytes(value []byte) error - - // EncodeBytesLen encodes the length of a byte slice. - EncodeBytesLen(value int) error - - // EncodeArrayLen encodes the length of an array. - EncodeArrayLen(value int) error -} - -// BufferedEncoder is an encoder backed by byte buffers. -type BufferedEncoder interface { - Buffer - Encoder -} - -// BufferedEncoderAlloc allocates a bufferer encoder. -type BufferedEncoderAlloc func() BufferedEncoder - -// BufferedEncoderPool is a pool of buffered encoders. -type BufferedEncoderPool interface { - // Init initializes the buffered encoder pool. - Init(alloc BufferedEncoderAlloc) - - // Get returns a buffered encoder from the pool. - Get() BufferedEncoder - - // Put puts a buffered encoder into the pool. - Put(enc BufferedEncoder) -} - -// BufferedEncoderPoolOptions provides options for buffered encoder pools. -type BufferedEncoderPoolOptions interface { - // SetMaxCapacity sets the maximum capacity of buffers that can be returned to the pool. - SetMaxCapacity(value int) BufferedEncoderPoolOptions - - // MaxBufferCapacity returns the maximum capacity of buffers that can be returned to the pool. - MaxCapacity() int - - // SetObjectPoolOptions sets the object pool options. - SetObjectPoolOptions(value pool.ObjectPoolOptions) BufferedEncoderPoolOptions - - // ObjectPoolOptions returns the object pool options. - ObjectPoolOptions() pool.ObjectPoolOptions -} - -// encoderBase is the base encoder interface. -type encoderBase interface { - // Encoder returns the encoder. - encoder() BufferedEncoder - - // err returns the error encountered during encoding, if any. - err() error - - // reset resets the encoder. - reset(encoder BufferedEncoder) - - // resetData resets the encoder data. - resetData() - - // encodeStoragePolicy encodes a storage policy. - encodeStoragePolicy(p policy.StoragePolicy) - - // encodePolicy encodes a policy. - encodePolicy(p policy.Policy) - - // encodeVersion encodes a version. - encodeVersion(version int) - - // encodeObjectType encodes an object type. - encodeObjectType(objType objectType) - - // encodeNumObjectFields encodes the number of object fields. - encodeNumObjectFields(numFields int) - - // encodeRawID encodes a raw ID. - encodeRawID(id id.RawID) - - // encodeChunkedID encodes a chunked ID. - encodeChunkedID(id id.ChunkedID) - - // encodeVarint encodes an integer value as varint. - encodeVarint(value int64) - - // encodeBool encodes a boolean value. - encodeBool(value bool) - - // encodeFloat64 encodes a float64 value. - encodeFloat64(value float64) - - // encodeBytes encodes a byte slice. - encodeBytes(value []byte) - - // encodeBytesLen encodes the length of a byte slice. - encodeBytesLen(value int) - - // encodeArrayLen encodes the length of an array. - encodeArrayLen(value int) -} - -// iteratorBase is the base iterator interface. -type iteratorBase interface { - // Reset resets the iterator. - reset(reader io.Reader) - - // err returns the error encountered during decoding, if any. - err() error - - // setErr sets the iterator error. - setErr(err error) - - // reader returns the buffered reader. - reader() bufReader - - // decodeStoragePolicy decodes a storage policy. - decodeStoragePolicy() policy.StoragePolicy - - // decodePolicy decodes a policy. - decodePolicy() policy.Policy - - // decodeVersion decodes a version. - decodeVersion() int - - // decodeObjectType decodes an object type. - decodeObjectType() objectType - - // decodeNumObjectFields decodes the number of object fields. - decodeNumObjectFields() int - - // decodeRawID decodes a raw ID. - decodeRawID() id.RawID - - // decodeVarint decodes a variable-width integer value. - decodeVarint() int64 - - // decodeBool decodes a boolean value. - decodeBool() bool - - // decodeFloat64 decodes a float64 value. - decodeFloat64() float64 - - // decodeBytes decodes a byte slice. - decodeBytes() []byte - - // decodeBytesLen decodes the length of a byte slice. - decodeBytesLen() int - - // decodeArrayLen decodes the length of an array. - decodeArrayLen() int - - // skip skips given number of fields if applicable. - skip(numFields int) - - // checkNumFieldsForType decodes and compares the number of actual fields with - // the number of expected fields for a given object type. - checkNumFieldsForType(objType objectType) (int, int, bool) - - // checkExpectedNumFieldsForType compares the given number of actual fields with - // the number of expected fields for a given object type. - checkExpectedNumFieldsForType(objType objectType, numActualFields int) (int, bool) -} - -// UnaggregatedEncoder is an encoder for encoding different types of unaggregated metrics. -type UnaggregatedEncoder interface { - // EncodeCounter encodes a counter. - EncodeCounter(c unaggregated.Counter) error - - // EncodeBatchTimer encodes a batch timer. - EncodeBatchTimer(bt unaggregated.BatchTimer) error - - // EncodeGauge encodes a gauge. - EncodeGauge(g unaggregated.Gauge) error - - // EncodeCounterWithPoliciesList encodes a counter with applicable policies list. - EncodeCounterWithPoliciesList(cp unaggregated.CounterWithPoliciesList) error - - // EncodeBatchTimerWithPoliciesList encodes a batched timer with applicable policies list. - EncodeBatchTimerWithPoliciesList(btp unaggregated.BatchTimerWithPoliciesList) error - - // EncodeGaugeWithPoliciesList encodes a gauge with applicable policies list. - EncodeGaugeWithPoliciesList(gp unaggregated.GaugeWithPoliciesList) error - - // Encoder returns the encoder. - Encoder() BufferedEncoder - - // Reset resets the encoder. - Reset(encoder BufferedEncoder) -} - -// UnaggregatedIterator is an iterator for decoding different types of unaggregated metrics. -type UnaggregatedIterator interface { - // Next returns true if there are more items to decode. - Next() bool - - // Metric returns the current metric. - // The returned value remains valid until the next Next() call. - Metric() unaggregated.MetricUnion - - // PoliciesList returns the current applicable policies list. - // The returned value remains valid until the next Next() call. - PoliciesList() policy.PoliciesList - - // Err returns the error encountered during decoding, if any. - Err() error - - // Reset resets the iterator. - Reset(reader io.Reader) - - // Close closes the iterator. - Close() -} - -// UnaggregatedIteratorOptions provide options for unaggregated iterators. -type UnaggregatedIteratorOptions interface { - // SetIgnoreHigherVersion determines whether the iterator ignores messages - // with higher-than-supported version. - SetIgnoreHigherVersion(value bool) UnaggregatedIteratorOptions - - // IgnoreHigherVersion returns whether the iterator ignores messages with - // higher-than-supported version. - IgnoreHigherVersion() bool - - // SetReaderBufferSize sets the reader buffer size. - SetReaderBufferSize(value int) UnaggregatedIteratorOptions - - // ReaderBufferSize returns the reader buffer size. - ReaderBufferSize() int - - // SetLargeFloatsSize determines whether a float slice is considered a "large" - // slice and therefore resort to the pool for allocating that slice. - SetLargeFloatsSize(value int) UnaggregatedIteratorOptions - - // LargeFloatsSize returns whether a float slice is considered a "large" - // slice and therefore resort to the pool for allocating that slice. - LargeFloatsSize() int - - // SetLargeFloatsPool sets the large floats pool. - SetLargeFloatsPool(value pool.FloatsPool) UnaggregatedIteratorOptions - - // LargeFloatsPool returns the large floats pool. - LargeFloatsPool() pool.FloatsPool - - // SetIteratorPool sets the unaggregated iterator pool. - SetIteratorPool(value UnaggregatedIteratorPool) UnaggregatedIteratorOptions - - // IteratorPool returns the unaggregated iterator pool. - IteratorPool() UnaggregatedIteratorPool -} - -// UnaggregatedIteratorAlloc allocates an unaggregated iterator. -type UnaggregatedIteratorAlloc func() UnaggregatedIterator - -// UnaggregatedIteratorPool is a pool of unaggregated iterators. -type UnaggregatedIteratorPool interface { - // Init initializes the unaggregated iterator pool. - Init(alloc UnaggregatedIteratorAlloc) - - // Get returns an unaggregated iterator from the pool. - Get() UnaggregatedIterator - - // Put puts an unaggregated iterator into the pool. - Put(it UnaggregatedIterator) -} - -// AggregatedEncoder is an encoder for encoding aggregated metrics. -type AggregatedEncoder interface { - // EncodeMetricWithStoragePolicy encodes a metric with an applicable storage policy. - EncodeMetricWithStoragePolicy(mp aggregated.MetricWithStoragePolicy) error - - // EncodeMetricWithStoragePolicyAndEncodeTime encodes a metric with an applicable - // storage policy, alongside the time at which encoding happens. - EncodeMetricWithStoragePolicyAndEncodeTime( - mp aggregated.MetricWithStoragePolicy, - encodedAtNanos int64, - ) error - - // EncodeChunkedMetricWithStoragePolicy encodes a chunked metric with an applicable storage policy. - EncodeChunkedMetricWithStoragePolicy(cmp aggregated.ChunkedMetricWithStoragePolicy) error - - // EncodeChunkedMetricWithStoragePolicyAndEncodeTime encodes a chunked metric with - // an applicable storage policy, alongside the time at which encoding happens. - EncodeChunkedMetricWithStoragePolicyAndEncodeTime( - cmp aggregated.ChunkedMetricWithStoragePolicy, - encodedAtNanos int64, - ) error - - // Encoder returns the encoder. - Encoder() BufferedEncoder - - // Reset resets the encoder. - Reset(encoder BufferedEncoder) -} - -// AggregatedIterator is an iterator for decoding aggregated metrics. -type AggregatedIterator interface { - // Next returns true if there are more metrics to decode. - Next() bool - - // Value returns the current raw metric, the corresponding policy, and timestamp at - // which the metric and the policy were encoded if applicable. - Value() (aggregated.RawMetric, policy.StoragePolicy, int64) - - // Err returns the error encountered during decoding, if any. - Err() error - - // Reset resets the iterator. - Reset(reader io.Reader) - - // Close closes the iterator. - Close() -} - -// AggregatedIteratorOptions provide options for aggregated iterators. -type AggregatedIteratorOptions interface { - // SetIgnoreHigherVersion determines whether the iterator ignores messages - // with higher-than-supported version. - SetIgnoreHigherVersion(value bool) AggregatedIteratorOptions - - // IgnoreHigherVersion returns whether the iterator ignores messages with - // higher-than-supported version. - IgnoreHigherVersion() bool - - // SetReaderBufferSize sets the reader buffer size. - SetReaderBufferSize(value int) AggregatedIteratorOptions - - // ReaderBufferSize returns the reader buffer size. - ReaderBufferSize() int - - // SetIteratorPool sets the aggregated iterator pool. - SetIteratorPool(value AggregatedIteratorPool) AggregatedIteratorOptions - - // IteratorPool returns the aggregated iterator pool. - IteratorPool() AggregatedIteratorPool -} - -// AggregatedIteratorAlloc allocates an aggregated iterator. -type AggregatedIteratorAlloc func() AggregatedIterator - -// AggregatedIteratorPool is a pool of aggregated iterators. -type AggregatedIteratorPool interface { - // Init initializes the aggregated iterator pool. - Init(alloc AggregatedIteratorAlloc) - - // Get returns an aggregated iterator from the pool. - Get() AggregatedIterator - - // Put puts an aggregated iterator into the pool. - Put(it AggregatedIterator) -} diff --git a/src/metrics/encoding/msgpack/unaggregated_encoder.go b/src/metrics/encoding/msgpack/unaggregated_encoder.go deleted file mode 100644 index 8bcd22eeee..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_encoder.go +++ /dev/null @@ -1,195 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" -) - -// Various object-level encoding functions to facilitate testing. -type encodeRootObjectFn func(objType objectType) -type encodeCounterWithPoliciesListFn func(cp unaggregated.CounterWithPoliciesList) -type encodeBatchTimerWithPoliciesListFn func(btp unaggregated.BatchTimerWithPoliciesList) -type encodeGaugeWithPoliciesListFn func(gp unaggregated.GaugeWithPoliciesList) -type encodeCounterFn func(c unaggregated.Counter) -type encodeBatchTimerFn func(bt unaggregated.BatchTimer) -type encodeGaugeFn func(g unaggregated.Gauge) -type encodePoliciesListFn func(spl policy.PoliciesList) - -// unaggregatedEncoder uses MessagePack for encoding different types of unaggregated metrics. -// It is not thread-safe. -type unaggregatedEncoder struct { - encoderBase - - encodeRootObjectFn encodeRootObjectFn - encodeCounterWithPoliciesListFn encodeCounterWithPoliciesListFn - encodeBatchTimerWithPoliciesListFn encodeBatchTimerWithPoliciesListFn - encodeGaugeWithPoliciesListFn encodeGaugeWithPoliciesListFn - encodeCounterFn encodeCounterFn - encodeBatchTimerFn encodeBatchTimerFn - encodeGaugeFn encodeGaugeFn - encodePoliciesListFn encodePoliciesListFn -} - -// NewUnaggregatedEncoder creates a new unaggregated encoder. -func NewUnaggregatedEncoder(encoder BufferedEncoder) UnaggregatedEncoder { - enc := &unaggregatedEncoder{encoderBase: newBaseEncoder(encoder)} - - enc.encodeRootObjectFn = enc.encodeRootObject - enc.encodeCounterWithPoliciesListFn = enc.encodeCounterWithPoliciesList - enc.encodeBatchTimerWithPoliciesListFn = enc.encodeBatchTimerWithPoliciesList - enc.encodeGaugeWithPoliciesListFn = enc.encodeGaugeWithPoliciesList - enc.encodeCounterFn = enc.encodeCounter - enc.encodeBatchTimerFn = enc.encodeBatchTimer - enc.encodeGaugeFn = enc.encodeGauge - enc.encodePoliciesListFn = enc.encodePoliciesList - - return enc -} - -func (enc *unaggregatedEncoder) Encoder() BufferedEncoder { return enc.encoder() } -func (enc *unaggregatedEncoder) Reset(encoder BufferedEncoder) { enc.reset(encoder) } - -func (enc *unaggregatedEncoder) EncodeCounter(c unaggregated.Counter) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(counterType) - enc.encodeCounterFn(c) - return enc.err() -} - -func (enc *unaggregatedEncoder) EncodeBatchTimer(bt unaggregated.BatchTimer) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(timerType) - enc.encodeBatchTimerFn(bt) - return enc.err() -} - -func (enc *unaggregatedEncoder) EncodeGauge(g unaggregated.Gauge) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(gaugeType) - enc.encodeGaugeFn(g) - return enc.err() -} - -func (enc *unaggregatedEncoder) EncodeCounterWithPoliciesList(cp unaggregated.CounterWithPoliciesList) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(counterWithPoliciesListType) - enc.encodeCounterWithPoliciesListFn(cp) - return enc.err() -} - -func (enc *unaggregatedEncoder) EncodeBatchTimerWithPoliciesList(btp unaggregated.BatchTimerWithPoliciesList) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(batchTimerWithPoliciesListType) - enc.encodeBatchTimerWithPoliciesListFn(btp) - return enc.err() -} - -func (enc *unaggregatedEncoder) EncodeGaugeWithPoliciesList(gp unaggregated.GaugeWithPoliciesList) error { - if err := enc.err(); err != nil { - return err - } - enc.encodeRootObjectFn(gaugeWithPoliciesListType) - enc.encodeGaugeWithPoliciesListFn(gp) - return enc.err() -} - -func (enc *unaggregatedEncoder) encodeRootObject(objType objectType) { - enc.encodeVersion(unaggregatedVersion) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType)) - enc.encodeObjectType(objType) -} - -func (enc *unaggregatedEncoder) encodeCounterWithPoliciesList(cp unaggregated.CounterWithPoliciesList) { - enc.encodeNumObjectFields(numFieldsForType(counterWithPoliciesListType)) - enc.encodeCounterFn(cp.Counter) - enc.encodePoliciesListFn(cp.PoliciesList) -} - -func (enc *unaggregatedEncoder) encodeBatchTimerWithPoliciesList(btp unaggregated.BatchTimerWithPoliciesList) { - enc.encodeNumObjectFields(numFieldsForType(batchTimerWithPoliciesListType)) - enc.encodeBatchTimerFn(btp.BatchTimer) - enc.encodePoliciesListFn(btp.PoliciesList) -} - -func (enc *unaggregatedEncoder) encodeGaugeWithPoliciesList(gp unaggregated.GaugeWithPoliciesList) { - enc.encodeNumObjectFields(numFieldsForType(gaugeWithPoliciesListType)) - enc.encodeGaugeFn(gp.Gauge) - enc.encodePoliciesListFn(gp.PoliciesList) -} - -func (enc *unaggregatedEncoder) encodeCounter(c unaggregated.Counter) { - enc.encodeNumObjectFields(numFieldsForType(counterType)) - enc.encodeRawID(c.ID) - enc.encodeVarint(c.Value) -} - -func (enc *unaggregatedEncoder) encodeBatchTimer(bt unaggregated.BatchTimer) { - enc.encodeNumObjectFields(numFieldsForType(timerType)) - enc.encodeRawID(bt.ID) - enc.encodeArrayLen(len(bt.Values)) - for _, v := range bt.Values { - enc.encodeFloat64(v) - } -} - -func (enc *unaggregatedEncoder) encodeGauge(g unaggregated.Gauge) { - enc.encodeNumObjectFields(numFieldsForType(gaugeType)) - enc.encodeRawID(g.ID) - enc.encodeFloat64(g.Value) -} - -func (enc *unaggregatedEncoder) encodePoliciesList(pl policy.PoliciesList) { - if pl.IsDefault() { - enc.encodeNumObjectFields(numFieldsForType(defaultPoliciesListType)) - enc.encodeObjectType(defaultPoliciesListType) - return - } - enc.encodeNumObjectFields(numFieldsForType(customPoliciesListType)) - enc.encodeObjectType(customPoliciesListType) - numPolicies := len(pl) - enc.encodeArrayLen(numPolicies) - for i := 0; i < numPolicies; i++ { - enc.encodeStagedPolicies(pl[i]) - } -} - -func (enc *unaggregatedEncoder) encodeStagedPolicies(sp policy.StagedPolicies) { - enc.encodeNumObjectFields(numFieldsForType(stagedPoliciesType)) - enc.encodeVarint(sp.CutoverNanos) - enc.encodeBool(sp.Tombstoned) - policies, _ := sp.Policies() - enc.encodeArrayLen(len(policies)) - for _, policy := range policies { - enc.encodePolicy(policy) - } -} diff --git a/src/metrics/encoding/msgpack/unaggregated_encoder_test.go b/src/metrics/encoding/msgpack/unaggregated_encoder_test.go deleted file mode 100644 index 07d554716e..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_encoder_test.go +++ /dev/null @@ -1,413 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "errors" - "fmt" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -var ( - errTestVarint = errors.New("test varint error") - errTestFloat64 = errors.New("test float64 error") - errTestBytes = errors.New("test bytes error") - errTestArrayLen = errors.New("test array len error") -) - -func TestUnaggregatedEncodeCounter(t *testing.T) { - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetric(encoder, testCounter)) - expected := expectedResultsForUnaggregatedMetric(t, testCounter) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeBatchTimer(t *testing.T) { - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetric(encoder, testBatchTimer)) - expected := expectedResultsForUnaggregatedMetric(t, testBatchTimer) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeGauge(t *testing.T) { - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetric(encoder, testGauge)) - expected := expectedResultsForUnaggregatedMetric(t, testGauge) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeCounterWithDefaultPoliciesList(t *testing.T) { - policies := testDefaultStagedPoliciesList - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, testCounter, policies)) - expected := expectedResultsForUnaggregatedMetricWithPoliciesList(t, testCounter, policies) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeBatchTimerWithDefaultPoliciesList(t *testing.T) { - policies := testDefaultStagedPoliciesList - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, testBatchTimer, policies)) - expected := expectedResultsForUnaggregatedMetricWithPoliciesList(t, testBatchTimer, policies) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeGaugeWithDefaultPoliciesList(t *testing.T) { - policies := testDefaultStagedPoliciesList - encoder, results := testCapturingUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, testGauge, policies)) - expected := expectedResultsForUnaggregatedMetricWithPoliciesList(t, testGauge, policies) - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeAllMetricTypes(t *testing.T) { - inputs := []unaggregated.MetricUnion{testCounter, testBatchTimer, testGauge} - var expected []interface{} - encoder, results := testCapturingUnaggregatedEncoder() - for _, input := range inputs { - require.NoError(t, testUnaggregatedEncodeMetric(encoder, input)) - expected = append(expected, expectedResultsForUnaggregatedMetric(t, input)...) - } - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeAllTypesWithDefaultPoliciesList(t *testing.T) { - var expected []interface{} - encoder, results := testCapturingUnaggregatedEncoder() - for _, input := range testInputWithAllTypesAndDefaultPoliciesList { - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, input.metric, input.policiesList)) - expected = append(expected, expectedResultsForUnaggregatedMetricWithPoliciesList(t, input.metric, input.policiesList)...) - } - - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeAllTypesWithSingleCustomPoliciesList(t *testing.T) { - var expected []interface{} - encoder, results := testCapturingUnaggregatedEncoder() - for _, input := range testInputWithAllTypesAndSingleCustomPoliciesList { - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, input.metric, input.policiesList)) - expected = append(expected, expectedResultsForUnaggregatedMetricWithPoliciesList(t, input.metric, input.policiesList)...) - } - - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeAllTypesWithMultiCustomPolicies(t *testing.T) { - var expected []interface{} - encoder, results := testCapturingUnaggregatedEncoder() - for _, input := range testInputWithAllTypesAndMultiCustomPoliciesList { - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, input.metric, input.policiesList)) - expected = append(expected, expectedResultsForUnaggregatedMetricWithPoliciesList(t, input.metric, input.policiesList)...) - } - - require.Equal(t, expected, *results) -} - -func TestUnaggregatedEncodeVarintError(t *testing.T) { - counter := testCounter - policies := testDefaultStagedPoliciesList - - // Intentionally return an error when encoding varint. - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeVarintFn = func(value int64) { - baseEncoder.encodeErr = errTestVarint - } - - // Assert the error is expected. - require.Equal(t, errTestVarint, testUnaggregatedEncodeMetricWithPoliciesList(encoder, counter, policies)) - - // Assert re-encoding doesn't change the error. - require.Equal(t, errTestVarint, testUnaggregatedEncodeMetricWithPoliciesList(encoder, counter, policies)) -} - -func TestUnaggregatedEncodeFloat64Error(t *testing.T) { - gauge := testGauge - policies := testDefaultStagedPoliciesList - - // Intentionally return an error when encoding float64. - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeFloat64Fn = func(value float64) { - baseEncoder.encodeErr = errTestFloat64 - } - - // Assert the error is expected. - require.Equal(t, errTestFloat64, testUnaggregatedEncodeMetricWithPoliciesList(encoder, gauge, policies)) - - // Assert re-encoding doesn't change the error. - require.Equal(t, errTestFloat64, testUnaggregatedEncodeMetricWithPoliciesList(encoder, gauge, policies)) -} - -func TestUnaggregatedEncodeBytesError(t *testing.T) { - timer := testBatchTimer - policies := testDefaultStagedPoliciesList - - // Intentionally return an error when encoding array length. - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeBytesFn = func(value []byte) { - baseEncoder.encodeErr = errTestBytes - } - - // Assert the error is expected. - require.Equal(t, errTestBytes, testUnaggregatedEncodeMetricWithPoliciesList(encoder, timer, policies)) - - // Assert re-encoding doesn't change the error. - require.Equal(t, errTestBytes, testUnaggregatedEncodeMetricWithPoliciesList(encoder, timer, policies)) -} - -func TestUnaggregatedEncodeArrayLenError(t *testing.T) { - gauge := testGauge - policies := policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - } - - // Intentionally return an error when encoding array length. - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeArrayLenFn = func(value int) { - baseEncoder.encodeErr = errTestArrayLen - } - - // Assert the error is expected. - require.Equal(t, errTestArrayLen, testUnaggregatedEncodeMetricWithPoliciesList(encoder, gauge, policies)) - - // Assert re-encoding doesn't change the error. - require.Equal(t, errTestArrayLen, testUnaggregatedEncodeMetricWithPoliciesList(encoder, gauge, policies)) -} - -func TestUnaggregatedEncoderReset(t *testing.T) { - metric := testCounter - policies := testDefaultStagedPoliciesList - - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := encoder.encoderBase.(*baseEncoder) - baseEncoder.encodeErr = errTestVarint - require.Equal(t, errTestVarint, testUnaggregatedEncodeMetricWithPoliciesList(encoder, metric, policies)) - - encoder.Reset(NewBufferedEncoder()) - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(encoder, metric, policies)) -} - -func testCapturingUnaggregatedEncoder() (UnaggregatedEncoder, *[]interface{}) { - encoder := testUnaggregatedEncoder().(*unaggregatedEncoder) - result := testCapturingBaseEncoder(encoder.encoderBase) - return encoder, result -} - -func expectedResultsForUnaggregatedPolicy(p policy.Policy) []interface{} { - results := []interface{}{numFieldsForType(policyType)} - - results = append(results, expectedResultsForPolicy(p.StoragePolicy)...) - return append(results, expectedResultsForCompressedAggregationTypes(p.AggregationID)...) -} - -func expectedResultsForCompressedAggregationTypes(compressed aggregation.ID) []interface{} { - results := []interface{}{} - - if compressed.IsDefault() { - return append(results, numFieldsForType(defaultAggregationID), int64(defaultAggregationID)) - } - - if len(compressed) == 1 { - return append(results, numFieldsForType(shortAggregationID), int64(shortAggregationID), int64(compressed[0])) - } - - results = append(results, numFieldsForType(longAggregationID), int64(longAggregationID), int64(len(compressed))) - - for _, code := range compressed { - results = append(results, code) - } - - return results -} - -func expectedResultsForPolicy(p policy.StoragePolicy) []interface{} { - results := []interface{}{numFieldsForType(storagePolicyType)} - - resolutionValue, err := policy.ValueFromResolution(p.Resolution()) - if err == nil { - results = append(results, []interface{}{ - numFieldsForType(knownResolutionType), - int64(knownResolutionType), - int64(resolutionValue), - }...) - } else { - results = append(results, []interface{}{ - numFieldsForType(unknownResolutionType), - int64(unknownResolutionType), - int64(p.Resolution().Window), - int64(p.Resolution().Precision), - }...) - } - - retentionValue, err := policy.ValueFromRetention(p.Retention()) - if err == nil { - results = append(results, []interface{}{ - numFieldsForType(knownRetentionType), - int64(knownRetentionType), - int64(retentionValue), - }...) - } else { - results = append(results, []interface{}{ - numFieldsForType(unknownRetentionType), - int64(unknownRetentionType), - int64(p.Retention()), - }...) - } - - return results -} - -func expectedResultsForStagedPolicies(sp policy.StagedPolicies) []interface{} { - policies, _ := sp.Policies() - results := []interface{}{ - numFieldsForType(stagedPoliciesType), - sp.CutoverNanos, - sp.Tombstoned, - len(policies), - } - for _, p := range policies { - results = append(results, expectedResultsForUnaggregatedPolicy(p)...) - } - return results -} - -func expectedResultsForPoliciesList(pl policy.PoliciesList) []interface{} { - if pl.IsDefault() { - return []interface{}{ - numFieldsForType(defaultPoliciesListType), - int64(defaultPoliciesListType), - } - } - results := []interface{}{ - numFieldsForType(customPoliciesListType), - int64(customPoliciesListType), - len(pl), - } - for _, sp := range pl { - results = append(results, expectedResultsForStagedPolicies(sp)...) - } - return results -} - -func expectedResultsForUnaggregatedMetric(t *testing.T, m unaggregated.MetricUnion) []interface{} { - results := []interface{}{ - int64(unaggregatedVersion), - numFieldsForType(rootObjectType), - } - - switch m.Type { - case metric.CounterType: - results = append(results, []interface{}{ - int64(counterType), - numFieldsForType(counterType), - []byte(m.ID), - m.CounterVal, - }...) - case metric.TimerType: - results = append(results, []interface{}{ - int64(timerType), - numFieldsForType(timerType), - []byte(m.ID), - len(m.BatchTimerVal), - }...) - for _, v := range m.BatchTimerVal { - results = append(results, v) - } - case metric.GaugeType: - results = append(results, []interface{}{ - int64(gaugeType), - numFieldsForType(gaugeType), - []byte(m.ID), - m.GaugeVal, - }...) - default: - require.Fail(t, fmt.Sprintf("unrecognized metric type %v", m.Type)) - } - - return results -} - -func expectedResultsForUnaggregatedMetricWithPoliciesList( - t *testing.T, - m unaggregated.MetricUnion, - pl policy.PoliciesList, -) []interface{} { - results := []interface{}{ - int64(unaggregatedVersion), - numFieldsForType(rootObjectType), - } - - switch m.Type { - case metric.CounterType: - results = append(results, []interface{}{ - int64(counterWithPoliciesListType), - numFieldsForType(counterWithPoliciesListType), - numFieldsForType(counterType), - []byte(m.ID), - m.CounterVal, - }...) - case metric.TimerType: - results = append(results, []interface{}{ - int64(batchTimerWithPoliciesListType), - numFieldsForType(batchTimerWithPoliciesListType), - numFieldsForType(timerType), - []byte(m.ID), - len(m.BatchTimerVal), - }...) - for _, v := range m.BatchTimerVal { - results = append(results, v) - } - case metric.GaugeType: - results = append(results, []interface{}{ - int64(gaugeWithPoliciesListType), - numFieldsForType(gaugeWithPoliciesListType), - numFieldsForType(gaugeType), - []byte(m.ID), - m.GaugeVal, - }...) - default: - require.Fail(t, fmt.Sprintf("unrecognized metric type %v", m.Type)) - } - - plRes := expectedResultsForPoliciesList(pl) - results = append(results, plRes...) - - return results -} diff --git a/src/metrics/encoding/msgpack/unaggregated_iterator.go b/src/metrics/encoding/msgpack/unaggregated_iterator.go deleted file mode 100644 index e254c2de5b..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_iterator.go +++ /dev/null @@ -1,326 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "fmt" - "io" - "math" - - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - "github.com/m3db/m3/src/x/pool" -) - -const ( - defaultInitTimerValuesCapacity = 16 -) - -// unaggregatedIterator uses MessagePack to decode different types of unaggregated metrics. -// It is not thread-safe. -type unaggregatedIterator struct { - iteratorBase - - largeFloatsSize int - largeFloatsPool pool.FloatsPool - iteratorPool UnaggregatedIteratorPool - ignoreHigherVersion bool - - closed bool - metric unaggregated.MetricUnion - policiesList policy.PoliciesList - id id.RawID - timerValues []float64 - cachedPolicies [][]policy.Policy - cachedPoliciesList policy.PoliciesList -} - -// NewUnaggregatedIterator creates a new unaggregated iterator. -func NewUnaggregatedIterator(reader io.Reader, opts UnaggregatedIteratorOptions) UnaggregatedIterator { - if opts == nil { - opts = NewUnaggregatedIteratorOptions() - } - it := &unaggregatedIterator{ - iteratorBase: newBaseIterator(reader, opts.ReaderBufferSize()), - ignoreHigherVersion: opts.IgnoreHigherVersion(), - largeFloatsSize: opts.LargeFloatsSize(), - largeFloatsPool: opts.LargeFloatsPool(), - iteratorPool: opts.IteratorPool(), - timerValues: make([]float64, 0, defaultInitTimerValuesCapacity), - } - return it -} - -func (it *unaggregatedIterator) Err() error { return it.err() } - -func (it *unaggregatedIterator) Reset(reader io.Reader) { - it.closed = false - it.metric.Reset() - it.reset(reader) -} - -func (it *unaggregatedIterator) Metric() unaggregated.MetricUnion { - return it.metric -} - -func (it *unaggregatedIterator) PoliciesList() policy.PoliciesList { - return it.policiesList -} - -func (it *unaggregatedIterator) Next() bool { - if it.err() != nil || it.closed { - return false - } - - // Reset the pointers in metric union to reduce GC sweep overhead. - it.metric.BatchTimerVal = nil - it.metric.TimerValPool = nil - - return it.decodeRootObject() -} - -func (it *unaggregatedIterator) Close() { - if it.closed { - return - } - it.closed = true - it.reset(emptyReader) - it.metric.Reset() - it.policiesList = nil - it.cachedPolicies = nil - it.cachedPoliciesList = nil - if it.iteratorPool != nil { - it.iteratorPool.Put(it) - } -} - -func (it *unaggregatedIterator) decodeRootObject() bool { - version := it.decodeVersion() - if it.err() != nil { - return false - } - // If the actual version is higher than supported version, we skip - // the data for this metric and continue to the next. - if version > unaggregatedVersion { - if it.ignoreHigherVersion { - it.skip(it.decodeNumObjectFields()) - return it.Next() - } - it.setErr(fmt.Errorf("received version %d is higher than supported version %d", version, unaggregatedVersion)) - return false - } - // Otherwise we proceed to decoding normally. - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(rootObjectType) - if !ok { - return false - } - objType := it.decodeObjectType() - if it.err() != nil { - return false - } - switch objType { - case counterType, timerType, gaugeType: - it.decodeMetric(objType) - case counterWithPoliciesListType, batchTimerWithPoliciesListType, gaugeWithPoliciesListType: - it.decodeMetricWithPoliciesList(objType) - default: - it.setErr(fmt.Errorf("unrecognized object type %v", objType)) - } - it.skip(numActualFields - numExpectedFields) - - return it.err() == nil -} - -func (it *unaggregatedIterator) decodeMetric(objType objectType) { - switch objType { - case counterType: - it.decodeCounter() - case timerType: - it.decodeBatchTimer() - case gaugeType: - it.decodeGauge() - default: - it.setErr(fmt.Errorf("unrecognized metric type %v", objType)) - } -} - -func (it *unaggregatedIterator) decodeMetricWithPoliciesList(objType objectType) { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(objType) - if !ok { - return - } - switch objType { - case counterWithPoliciesListType: - it.decodeCounter() - case batchTimerWithPoliciesListType: - it.decodeBatchTimer() - case gaugeWithPoliciesListType: - it.decodeGauge() - default: - it.setErr(fmt.Errorf("unrecognized metric with policies type %v", objType)) - return - } - it.decodePoliciesList() - it.skip(numActualFields - numExpectedFields) -} - -func (it *unaggregatedIterator) decodeCounter() { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(counterType) - if !ok { - return - } - it.metric.Type = metric.CounterType - it.metric.ID = it.decodeID() - it.metric.CounterVal = it.decodeVarint() - it.skip(numActualFields - numExpectedFields) -} - -func (it *unaggregatedIterator) decodeBatchTimer() { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(timerType) - if !ok { - return - } - it.metric.Type = metric.TimerType - it.metric.ID = it.decodeID() - var ( - timerValues []float64 - poolAlloc = false - numValues = it.decodeArrayLen() - ) - if cap(it.timerValues) >= numValues { - it.timerValues = it.timerValues[:0] - timerValues = it.timerValues - } else if numValues <= it.largeFloatsSize { - newCapcity := int(math.Max(float64(numValues), float64(cap(it.timerValues)*2))) - if newCapcity > it.largeFloatsSize { - newCapcity = it.largeFloatsSize - } - it.timerValues = make([]float64, 0, newCapcity) - timerValues = it.timerValues - } else { - timerValues = it.largeFloatsPool.Get(numValues) - poolAlloc = true - } - for i := 0; i < numValues; i++ { - timerValues = append(timerValues, it.decodeFloat64()) - } - it.metric.BatchTimerVal = timerValues - if poolAlloc { - it.metric.TimerValPool = it.largeFloatsPool - } - it.skip(numActualFields - numExpectedFields) -} - -func (it *unaggregatedIterator) decodeGauge() { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(gaugeType) - if !ok { - return - } - it.metric.Type = metric.GaugeType - it.metric.ID = it.decodeID() - it.metric.GaugeVal = it.decodeFloat64() - it.skip(numActualFields - numExpectedFields) -} - -func (it *unaggregatedIterator) decodePoliciesList() { - numActualFields := it.decodeNumObjectFields() - policiesListType := it.decodeObjectType() - numExpectedFields, ok := it.checkExpectedNumFieldsForType( - policiesListType, - numActualFields, - ) - if !ok { - return - } - switch policiesListType { - case defaultPoliciesListType: - it.policiesList = policy.DefaultPoliciesList - case customPoliciesListType: - numStagedPolicies := it.decodeArrayLen() - if cap(it.cachedPoliciesList) < numStagedPolicies { - it.cachedPoliciesList = make(policy.PoliciesList, 0, numStagedPolicies) - } else { - for i := 0; i < len(it.cachedPoliciesList); i++ { - it.cachedPoliciesList[i].Reset() - } - it.cachedPoliciesList = it.cachedPoliciesList[:0] - } - if len(it.cachedPolicies) < numStagedPolicies { - it.cachedPolicies = make([][]policy.Policy, numStagedPolicies) - } - for policyIdx := 0; policyIdx < numStagedPolicies; policyIdx++ { - decodedStagedPolicies := it.decodeStagedPolicies(policyIdx) - it.cachedPoliciesList = append(it.cachedPoliciesList, decodedStagedPolicies) - } - it.policiesList = it.cachedPoliciesList - default: - it.setErr(fmt.Errorf("unrecognized policies list type: %v", policiesListType)) - } - it.skip(numActualFields - numExpectedFields) -} - -// decodeStagedPolicies decodes a staged policies using the cached policies -// to avoid memory reallocations and returns the decoded staged policies. -// If an error is encountered during decoding, it is stored in the iterator. -func (it *unaggregatedIterator) decodeStagedPolicies(policyIdx int) policy.StagedPolicies { - numExpectedFields, numActualFields, ok := it.checkNumFieldsForType(stagedPoliciesType) - if !ok { - return policy.DefaultStagedPolicies - } - cutoverNanos := it.decodeVarint() - tombstoned := it.decodeBool() - numPolicies := it.decodeArrayLen() - if cap(it.cachedPolicies[policyIdx]) < numPolicies { - it.cachedPolicies[policyIdx] = make([]policy.Policy, 0, numPolicies) - } else { - it.cachedPolicies[policyIdx] = it.cachedPolicies[policyIdx][:0] - } - for i := 0; i < numPolicies; i++ { - it.cachedPolicies[policyIdx] = append(it.cachedPolicies[policyIdx], it.decodePolicy()) - } - stagedPolicies := policy.NewStagedPolicies(cutoverNanos, tombstoned, it.cachedPolicies[policyIdx]) - it.skip(numActualFields - numExpectedFields) - return stagedPolicies -} - -func (it *unaggregatedIterator) decodeID() id.RawID { - idLen := it.decodeBytesLen() - if it.err() != nil { - return nil - } - // NB(xichen): DecodeBytesLen() returns -1 if the byte slice is nil. - if idLen == -1 { - it.id = it.id[:0] - return it.id - } - if cap(it.id) < idLen { - it.id = make([]byte, idLen) - } else { - it.id = it.id[:idLen] - } - if _, err := io.ReadFull(it.reader(), it.id); err != nil { - it.setErr(err) - return nil - } - return it.id -} diff --git a/src/metrics/encoding/msgpack/unaggregated_iterator_pool.go b/src/metrics/encoding/msgpack/unaggregated_iterator_pool.go deleted file mode 100644 index e459ea3574..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_iterator_pool.go +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import "github.com/m3db/m3/src/x/pool" - -type unaggregatedIteratorPool struct { - pool pool.ObjectPool -} - -// NewUnaggregatedIteratorPool creates a new pool for unaggregated iterators. -func NewUnaggregatedIteratorPool(opts pool.ObjectPoolOptions) UnaggregatedIteratorPool { - return &unaggregatedIteratorPool{pool: pool.NewObjectPool(opts)} -} - -func (p *unaggregatedIteratorPool) Init(alloc UnaggregatedIteratorAlloc) { - p.pool.Init(func() interface{} { - return alloc() - }) -} - -func (p *unaggregatedIteratorPool) Get() UnaggregatedIterator { - return p.pool.Get().(UnaggregatedIterator) -} - -func (p *unaggregatedIteratorPool) Put(it UnaggregatedIterator) { - p.pool.Put(it) -} diff --git a/src/metrics/encoding/msgpack/unaggregated_iterator_pool_test.go b/src/metrics/encoding/msgpack/unaggregated_iterator_pool_test.go deleted file mode 100644 index ca0ed9acad..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_iterator_pool_test.go +++ /dev/null @@ -1,54 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "testing" - - "github.com/m3db/m3/src/x/pool" - - "github.com/stretchr/testify/require" -) - -func TestUnaggregatedIteratorPool(t *testing.T) { - p := NewUnaggregatedIteratorPool(pool.NewObjectPoolOptions().SetSize(1)) - itOpts := NewUnaggregatedIteratorOptions().SetIteratorPool(p) - p.Init(func() UnaggregatedIterator { - return NewUnaggregatedIterator(nil, itOpts) - }) - - // Retrieve an iterator from the pool. - it := p.Get() - it.Reset(bytes.NewBuffer([]byte{0x1, 0x2})) - - // Closing the iterator should put it back to the pool. - it.Close() - require.True(t, it.(*unaggregatedIterator).closed) - - // Retrieve the iterator and assert it's the same iterator. - it = p.Get() - require.True(t, it.(*unaggregatedIterator).closed) - - // Reset the iterator and assert it's been reset. - it.Reset(nil) - require.False(t, it.(*unaggregatedIterator).closed) -} diff --git a/src/metrics/encoding/msgpack/unaggregated_iterator_test.go b/src/metrics/encoding/msgpack/unaggregated_iterator_test.go deleted file mode 100644 index 98f17f460e..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_iterator_test.go +++ /dev/null @@ -1,620 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bufio" - "bytes" - "errors" - "io" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/id" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -func TestUnaggregatedIteratorDecodeDefaultPoliciesList(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodePoliciesList(testDefaultStagedPoliciesList) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodePoliciesList() - require.NoError(t, it.Err()) - pl := it.PoliciesList() - require.Equal(t, testDefaultStagedPoliciesList, pl) -} - -func TestUnaggregatedIteratorDecodeSingleCustomPoliciesListWithAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodePoliciesList(testSingleCustomStagedPoliciesList) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodePoliciesList() - require.NoError(t, it.Err()) - pl := it.PoliciesList() - require.Equal(t, testSingleCustomStagedPoliciesList, pl) - require.True(t, len(it.cachedPolicies) >= len(testSingleCustomStagedPoliciesList)) - policies, _ := testSingleCustomStagedPoliciesList[0].Policies() - require.Equal(t, it.cachedPolicies[0], policies) -} - -func TestUnaggregatedIteratorDecodeSingleCustomPoliciesListNoPoliciesListAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodePoliciesList(testSingleCustomStagedPoliciesList) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.cachedPoliciesList = make(policy.PoliciesList, len(testSingleCustomStagedPoliciesList)*3) - it.decodePoliciesList() - require.NoError(t, it.Err()) - pl := it.PoliciesList() - require.Equal(t, testSingleCustomStagedPoliciesList, pl) - require.True(t, len(it.cachedPolicies) >= len(testSingleCustomStagedPoliciesList)) - policies, _ := testSingleCustomStagedPoliciesList[0].Policies() - require.Equal(t, it.cachedPolicies[0], policies) -} - -func TestUnaggregatedIteratorDecodeSingleCustomPoliciesListNoAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodePoliciesList(testSingleCustomStagedPoliciesList) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.cachedPoliciesList = make(policy.PoliciesList, len(testSingleCustomStagedPoliciesList)*3) - it.cachedPolicies = make([][]policy.Policy, len(testSingleCustomStagedPoliciesList)*3) - it.cachedPolicies[0] = make([]policy.Policy, 32) - it.decodePoliciesList() - require.NoError(t, it.Err()) - pl := it.PoliciesList() - require.Equal(t, testSingleCustomStagedPoliciesList, pl) - require.True(t, len(it.cachedPolicies) >= len(testSingleCustomStagedPoliciesList)) - policies, _ := testSingleCustomStagedPoliciesList[0].Policies() - require.Equal(t, it.cachedPolicies[0], policies) -} - -func TestUnaggregatedIteratorDecodeMultiCustomPoliciesListWithAlloc(t *testing.T) { - input := testMultiCustomStagedPoliciesList - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodePoliciesList(input) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodePoliciesList() - require.NoError(t, it.Err()) - pl := it.PoliciesList() - require.Equal(t, input, pl) - require.True(t, len(it.cachedPolicies) >= len(input)) - for i := 0; i < len(input); i++ { - policies, _ := input[i].Policies() - require.Equal(t, it.cachedPolicies[i], policies) - } -} - -func TestUnaggregatedIteratorDecodeIDDecodeBytesLenError(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodeFloat64(1.0) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - require.Equal(t, 0, len(it.decodeID())) - require.Error(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeIDNilBytes(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodeBytes(nil) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - require.Equal(t, 0, len(it.decodeID())) - require.NoError(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeIDWithAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - data := []byte("foobarbaz") - enc.encodeBytes(data) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - require.Equal(t, id.RawID(data), it.decodeID()) - require.NoError(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeIDNoAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - data := []byte("foobarbaz") - enc.encodeBytes(data) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.id = make([]byte, len(data)*3) - require.Equal(t, id.RawID(data), it.decodeID()) - require.NoError(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeIDReadError(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - data := []byte("foobarbazasierasekr") - enc.encodeBytesLen(len(data) + 1) - require.NoError(t, enc.err()) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - require.Equal(t, 0, len(it.decodeID())) - require.Error(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeIDReadSuccess(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - data := []byte("foobarbazasierasekr") - enc.encodeBytes(data) - require.NoError(t, enc.err()) - - // Intentionally buffer some data in the buffered reader. - buf := append([]byte{1}, enc.Encoder().Bytes()...) - reader := bufio.NewReaderSize(bytes.NewBuffer(buf), 16) - _, err := reader.Read(make([]byte, 1)) - require.NoError(t, err) - - it := testUnaggregatedIterator(reader).(*unaggregatedIterator) - it.id = make([]byte, len(data)/2) - require.Equal(t, id.RawID(data), it.decodeID()) - require.NoError(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeBatchTimerDecodeArrayLenError(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodeFloat64(1.0) - require.NoError(t, enc.err()) - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodeBatchTimer() - require.Error(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeBatchTimerNoValues(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - bt := unaggregated.BatchTimer{ - ID: []byte("foo"), - Values: nil, - } - enc.encodeBatchTimer(bt) - require.NoError(t, enc.err()) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodeBatchTimer() - - require.NoError(t, it.Err()) - mu := it.Metric() - require.Equal(t, metric.TimerType, mu.Type) - require.Equal(t, id.RawID("foo"), mu.ID) - require.Equal(t, 0, len(mu.BatchTimerVal)) - require.Nil(t, mu.TimerValPool) -} - -func TestUnaggregatedIteratorDecodeBatchTimerDecodeFloat64Error(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - enc.encodeBatchTimerFn = func(bt unaggregated.BatchTimer) { - enc.encodeNumObjectFields(numFieldsForType(timerType)) - enc.encodeRawID(bt.ID) - enc.encodeArrayLen(len(bt.Values)) - enc.encodeBytes([]byte("foo")) - } - require.NoError(t, enc.err()) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodeBatchTimer() - - require.Error(t, it.Err()) -} - -func TestUnaggregatedIteratorDecodeBatchTimerNoAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - bt := unaggregated.BatchTimer{ - ID: []byte("foo"), - Values: []float64{1.0, 2.0, 3.0, 4.0}, - } - enc.encodeBatchTimer(bt) - require.NoError(t, enc.err()) - - // Allocate a large enough buffer to avoid triggering an allocation. - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.timerValues = make([]float64, 1000) - it.decodeBatchTimer() - - require.NoError(t, it.Err()) - mu := it.Metric() - require.Equal(t, metric.TimerType, mu.Type) - require.Equal(t, id.RawID("foo"), mu.ID) - require.Equal(t, bt.Values, mu.BatchTimerVal) - require.Equal(t, cap(it.timerValues), cap(mu.BatchTimerVal)) - require.Nil(t, mu.TimerValPool) -} - -func TestUnaggregatedIteratorDecodeBatchTimerWithAllocNonPoolAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - bt := unaggregated.BatchTimer{ - ID: []byte("foo"), - Values: []float64{1.0, 2.0, 3.0, 4.0}, - } - enc.encodeBatchTimer(bt) - require.NoError(t, enc.err()) - - // Allocate a large enough buffer to avoid triggering an allocation. - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.decodeBatchTimer() - - require.NoError(t, it.Err()) - mu := it.Metric() - require.Equal(t, metric.TimerType, mu.Type) - require.Equal(t, id.RawID("foo"), mu.ID) - require.Equal(t, bt.Values, mu.BatchTimerVal) - require.Equal(t, cap(it.timerValues), cap(mu.BatchTimerVal)) - require.Nil(t, mu.TimerValPool) -} - -func TestUnaggregatedIteratorDecodeBatchTimerWithAllocPoolAlloc(t *testing.T) { - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - bt := unaggregated.BatchTimer{ - ID: []byte("foo"), - Values: []float64{1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0}, - } - enc.encodeBatchTimer(bt) - require.NoError(t, enc.err()) - - // Allocate a large enough buffer to avoid triggering an allocation. - it := testUnaggregatedIterator(enc.Encoder().Buffer()).(*unaggregatedIterator) - it.timerValues = nil - it.largeFloatsSize = 2 - it.decodeBatchTimer() - - require.NoError(t, it.Err()) - mu := it.Metric() - require.Equal(t, metric.TimerType, mu.Type) - require.Equal(t, id.RawID("foo"), mu.ID) - require.Equal(t, bt.Values, mu.BatchTimerVal) - require.True(t, cap(mu.BatchTimerVal) >= len(bt.Values)) - require.Nil(t, it.timerValues) - require.NotNil(t, mu.TimerValPool) - require.Equal(t, it.largeFloatsPool, mu.TimerValPool) -} - -func TestUnaggregatedIteratorDecodeNewerVersionThanSupported(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Version encoded is higher than supported version. - enc.encodeRootObjectFn = func(objType objectType) { - enc.encodeVersion(unaggregatedVersion + 1) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType)) - enc.encodeObjectType(objType) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - // Now restore the encode top-level function and encode another counter. - enc.encodeRootObjectFn = enc.encodeRootObject - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - // Check that we skipped the first counter and successfully decoded the second counter. - it := testUnaggregatedIterator(bytes.NewBuffer(enc.Encoder().Bytes())) - it.(*unaggregatedIterator).ignoreHigherVersion = true - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) - - it.Reset(bytes.NewBuffer(enc.Encoder().Bytes())) - it.(*unaggregatedIterator).ignoreHigherVersion = false - validateUnaggregatedDecodeResults(t, it, nil, errors.New("received version 2 is higher than supported version 1")) -} - -func TestUnaggregatedIteratorDecodeRootObjectMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the root object. - enc.encodeRootObjectFn = func(objType objectType) { - enc.encodeVersion(unaggregatedVersion) - enc.encodeNumObjectFields(numFieldsForType(rootObjectType) + 1) - enc.encodeObjectType(objType) - } - err := testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList) - require.NoError(t, err) - enc.encodeVarint(0) - require.NoError(t, enc.err()) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the counter. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodeCounterWithPoliciesMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the counter with policies object. - enc.encodeCounterWithPoliciesListFn = func(cp unaggregated.CounterWithPoliciesList) { - enc.encodeNumObjectFields(numFieldsForType(counterWithPoliciesListType) + 1) - enc.encodeCounterFn(cp.Counter) - enc.encodePoliciesListFn(cp.PoliciesList) - enc.encodeVarint(0) - } - err := testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList) - require.NoError(t, err) - require.NoError(t, enc.err()) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the counter. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodeCounterMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the counter object. - enc.encodeCounterFn = func(c unaggregated.Counter) { - enc.encodeNumObjectFields(numFieldsForType(counterType) + 1) - enc.encodeRawID(c.ID) - enc.encodeVarint(int64(c.Value)) - enc.encodeVarint(0) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the counter. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodeBatchTimerMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testBatchTimer, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the batch timer object. - enc.encodeBatchTimerFn = func(bt unaggregated.BatchTimer) { - enc.encodeNumObjectFields(numFieldsForType(timerType) + 1) - enc.encodeRawID(bt.ID) - enc.encodeArrayLen(len(bt.Values)) - for _, v := range bt.Values { - enc.encodeFloat64(v) - } - enc.encodeVarint(0) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the batch timer. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodeGaugeMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testGauge, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the gauge object. - enc.encodeGaugeFn = func(g unaggregated.Gauge) { - enc.encodeNumObjectFields(numFieldsForType(gaugeType) + 1) - enc.encodeRawID(g.ID) - enc.encodeFloat64(g.Value) - enc.encodeVarint(0) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the gauge. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodePolicyWithCustomResolution(t *testing.T) { - input := metricWithPoliciesList{ - metric: testGauge, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(3*time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - }, - } - enc := testUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the policy. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodePolicyWithCustomRetention(t *testing.T) { - input := metricWithPoliciesList{ - metric: testGauge, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, 289*time.Hour), aggregation.DefaultID), - }, - ), - }, - } - enc := testUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the policy. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodePolicyMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testGauge, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - }, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - baseEncoder := enc.encoderBase.(*baseEncoder) - - // Pretend we added an extra int field to the policy object. - baseEncoder.encodeStoragePolicyFn = func(p policy.StoragePolicy) { - baseEncoder.encodeNumObjectFields(numFieldsForType(storagePolicyType) + 1) - baseEncoder.encodeResolution(p.Resolution()) - baseEncoder.encodeRetention(p.Retention()) - baseEncoder.encodeVarint(0) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the policy. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodePoliciesListMoreFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testGauge, - policiesList: testSingleCustomStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the policy object. - enc.encodePoliciesListFn = func(pl policy.PoliciesList) { - enc.encodeNumObjectFields(numFieldsForType(customPoliciesListType) + 1) - enc.encodeObjectType(customPoliciesListType) - enc.encodeArrayLen(len(pl)) - for _, sp := range pl { - enc.encodeStagedPolicies(sp) - } - enc.encodeVarint(0) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we successfully decoded the policy. - validateUnaggregatedDecodeResults(t, it, []metricWithPoliciesList{input}, io.EOF) -} - -func TestUnaggregatedIteratorDecodeCounterFewerFieldsThanExpected(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - } - enc := testUnaggregatedEncoder().(*unaggregatedEncoder) - - // Pretend we added an extra int field to the counter object. - enc.encodeCounterFn = func(c unaggregated.Counter) { - enc.encodeNumObjectFields(numFieldsForType(counterType) - 1) - enc.encodeRawID(c.ID) - } - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - - // Check that we encountered an error during decoding. - validateUnaggregatedDecodeResults(t, it, nil, errors.New("number of fields mismatch: expected 2 actual 1")) -} - -func TestUnaggregatedIteratorDecodeError(t *testing.T) { - it := NewUnaggregatedIterator(nil, nil) - err := errors.New("foo") - it.(*unaggregatedIterator).setErr(err) - - require.False(t, it.Next()) - require.Equal(t, err, it.Err()) -} - -func TestUnaggregatedIteratorReset(t *testing.T) { - it := NewUnaggregatedIterator(nil, nil) - err := errors.New("foo") - it.(*unaggregatedIterator).setErr(err) - - it.Reset(nil) - require.NoError(t, it.(*unaggregatedIterator).Err()) - require.False(t, it.(*unaggregatedIterator).closed) -} - -func TestUnaggregatedIteratorClose(t *testing.T) { - it := NewUnaggregatedIterator(nil, nil) - it.Close() - require.False(t, it.Next()) - require.NoError(t, it.Err()) - require.True(t, it.(*unaggregatedIterator).closed) -} - -func TestUnaggregatedIteratorDecodeInvalidTimeUnit(t *testing.T) { - input := metricWithPoliciesList{ - metric: testCounter, - policiesList: testStagedPoliciesWithInvalidTimeUnit, - } - enc := testUnaggregatedEncoder() - require.NoError(t, testUnaggregatedEncodeMetricWithPoliciesList(enc, input.metric, input.policiesList)) - it := testUnaggregatedIterator(enc.Encoder().Buffer()) - validateUnaggregatedDecodeResults(t, it, nil, errors.New("invalid precision unknown")) -} - -func validateUnaggregatedDecodeResults( - t *testing.T, - it UnaggregatedIterator, - expectedResults []metricWithPoliciesList, - expectedErr error, -) { - var results []metricWithPoliciesList - for it.Next() { - metric, policiesList := it.Metric(), it.PoliciesList() - policiesList = toPoliciesList(policiesList) - results = append(results, metricWithPoliciesList{ - metric: metric, - policiesList: policiesList, - }) - } - require.Equal(t, expectedErr, it.Err()) - validateMetricsWithPoliciesList(t, expectedResults, results) -} diff --git a/src/metrics/encoding/msgpack/unaggregated_roundtrip_test.go b/src/metrics/encoding/msgpack/unaggregated_roundtrip_test.go deleted file mode 100644 index 0c70ee8d06..0000000000 --- a/src/metrics/encoding/msgpack/unaggregated_roundtrip_test.go +++ /dev/null @@ -1,561 +0,0 @@ -// Copyright (c) 2016 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 msgpack - -import ( - "bytes" - "fmt" - "io" - "math/rand" - "testing" - "time" - - "github.com/m3db/m3/src/metrics/aggregation" - "github.com/m3db/m3/src/metrics/metric" - "github.com/m3db/m3/src/metrics/metric/unaggregated" - "github.com/m3db/m3/src/metrics/policy" - xtime "github.com/m3db/m3/src/x/time" - - "github.com/stretchr/testify/require" -) - -var ( - testCounter = unaggregated.MetricUnion{ - Type: metric.CounterType, - ID: []byte("foo"), - CounterVal: 1234, - } - - testBatchTimer = unaggregated.MetricUnion{ - Type: metric.TimerType, - ID: []byte("foo"), - BatchTimerVal: []float64{222.22, 345.67, 901.23345}, - } - - testGauge = unaggregated.MetricUnion{ - Type: metric.GaugeType, - ID: []byte("foo"), - GaugeVal: 123.456, - } - - testDefaultStagedPoliciesList = policy.DefaultPoliciesList - - testSingleCustomStagedPoliciesList = policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(20*time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 25*24*time.Hour), aggregation.DefaultID), - }, - ), - } - - testMultiCustomStagedPoliciesList = policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(20*time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 25*24*time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Minute).UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - } - - testStagedPoliciesWithInvalidTimeUnit = policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Unit(100), time.Hour), aggregation.DefaultID), - }, - ), - } - - testInputWithAllTypesAndDefaultPoliciesList = []metricWithPoliciesList{ - { - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - }, - { - metric: testBatchTimer, - policiesList: testDefaultStagedPoliciesList, - }, - { - metric: testGauge, - policiesList: testDefaultStagedPoliciesList, - }, - } - - testInputWithAllTypesAndSingleCustomPoliciesList = []metricWithPoliciesList{ - // Retain this metric at 20 second resolution for 6 hours, - // then 1 minute for 2 days, then 10 minutes for 25 days. - { - metric: testBatchTimer, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(20*time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.ID{8}), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 25*24*time.Hour), aggregation.DefaultID), - }, - ), - }, - }, - // Retain this metric at 1 second resolution for 1 hour. - { - metric: testCounter, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - }, - }, - // Retain this metric at 10 minute resolution for 45 days. - { - metric: testGauge, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 45*24*time.Hour), aggregation.ID{100}), - }, - ), - }, - }, - } - - testInputWithAllTypesAndMultiCustomPoliciesList = []metricWithPoliciesList{ - { - metric: testBatchTimer, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(20*time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 25*24*time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Minute).UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - }, - }, - { - metric: testCounter, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Hour).UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 45*24*time.Hour), aggregation.DefaultID), - }, - ), - }, - }, - { - metric: testGauge, - policiesList: policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 45*24*time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Nanosecond).UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(5*time.Minute, xtime.Minute, 36*time.Hour), aggregation.DefaultID), - }, - ), - }, - }, - } -) - -func TestUnaggregatedEncodeDecodeCounter(t *testing.T) { - validateUnaggregatedMetricRoundtrip(t, testCounter) -} - -func TestUnaggregatedEncodeDecodeBatchTimer(t *testing.T) { - validateUnaggregatedMetricRoundtrip(t, testBatchTimer) -} - -func TestUnaggregatedEncodeDecodeGauge(t *testing.T) { - validateUnaggregatedMetricRoundtrip(t, testGauge) -} - -func TestUnaggregatedEncodeDecodeCounterWithDefaultPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, metricWithPoliciesList{ - metric: testCounter, - policiesList: testDefaultStagedPoliciesList, - }) -} - -func TestUnaggregatedEncodeDecodeBatchTimerWithDefaultPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, metricWithPoliciesList{ - metric: testBatchTimer, - policiesList: testDefaultStagedPoliciesList, - }) -} - -func TestUnaggregatedEncodeDecodeGaugeWithDefaultPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, metricWithPoliciesList{ - metric: testGauge, - policiesList: testDefaultStagedPoliciesList, - }) -} - -func TestUnaggregatedEncodeDecodeAllMetricTypes(t *testing.T) { - inputs := []unaggregated.MetricUnion{testCounter, testBatchTimer, testGauge} - validateUnaggregatedMetricRoundtrip(t, inputs...) -} - -func TestUnaggregatedEncodeDecodeAllTypesWithDefaultPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, testInputWithAllTypesAndDefaultPoliciesList...) -} - -func TestUnaggregatedEncodeDecodeAllTypesWithSingleCustomPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, testInputWithAllTypesAndSingleCustomPoliciesList...) -} - -func TestUnaggregatedEncodeDecodeAllTypesWithMultiCustomPoliciesList(t *testing.T) { - validateUnaggregatedMetricWithPoliciesListRoundtrip(t, testInputWithAllTypesAndMultiCustomPoliciesList...) -} - -func TestUnaggregatedEncodeDecodeMetricStress(t *testing.T) { - numIter := 10 - numMetrics := 10000 - allMetrics := []unaggregated.MetricUnion{testCounter, testBatchTimer, testGauge} - encoder := testUnaggregatedEncoder() - iterator := testUnaggregatedIterator(nil) - for i := 0; i < numIter; i++ { - var inputs []unaggregated.MetricUnion - for j := 0; j < numMetrics; j++ { - m := allMetrics[rand.Int63n(int64(len(allMetrics)))] - inputs = append(inputs, m) - } - validateUnaggregatedMetricRoundtripWithEncoderAndIterator(t, encoder, iterator, inputs...) - } -} - -func TestUnaggregatedEncodeDecodeMetricWithPoliciesListStress(t *testing.T) { - numIter := 10 - numMetrics := 10000 - allMetrics := []unaggregated.MetricUnion{testCounter, testBatchTimer, testGauge} - allPolicies := []policy.PoliciesList{ - testDefaultStagedPoliciesList, - policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.DefaultID), - }, - ), - }, - policy.PoliciesList{ - policy.NewStagedPolicies( - time.Now().UnixNano(), - false, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(20*time.Second, xtime.Second, 6*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(time.Minute, xtime.Minute, 2*24*time.Hour), aggregation.DefaultID), - policy.NewPolicy(policy.NewStoragePolicy(10*time.Minute, xtime.Minute, 25*24*time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Minute).UnixNano(), - true, - []policy.Policy{ - policy.NewPolicy(policy.NewStoragePolicy(time.Second, xtime.Second, time.Hour), aggregation.DefaultID), - }, - ), - policy.NewStagedPolicies( - time.Now().Add(time.Minute).UnixNano(), - false, - []policy.Policy{}, - ), - }, - } - - encoder := testUnaggregatedEncoder() - iterator := testUnaggregatedIterator(nil) - for i := 0; i < numIter; i++ { - var inputs []metricWithPoliciesList - for j := 0; j < numMetrics; j++ { - m := allMetrics[rand.Int63n(int64(len(allMetrics)))] - p := allPolicies[rand.Int63n(int64(len(allPolicies)))] - inputs = append(inputs, metricWithPoliciesList{metric: m, policiesList: p}) - } - validateUnaggregatedMetricWithPoliciesListRoundtripWithEncoderAndIterator(t, encoder, iterator, inputs...) - } -} - -type metricWithPoliciesList struct { - metric unaggregated.MetricUnion - policiesList policy.PoliciesList -} - -func testCapturingBaseEncoder(encoder encoderBase) *[]interface{} { - baseEncoder := encoder.(*baseEncoder) - - var result []interface{} - baseEncoder.encodeVarintFn = func(value int64) { - result = append(result, value) - } - baseEncoder.encodeBoolFn = func(value bool) { - result = append(result, value) - } - baseEncoder.encodeFloat64Fn = func(value float64) { - result = append(result, value) - } - baseEncoder.encodeBytesFn = func(value []byte) { - result = append(result, value) - } - baseEncoder.encodeBytesLenFn = func(value int) { - result = append(result, value) - } - baseEncoder.encodeArrayLenFn = func(value int) { - result = append(result, value) - } - - return &result -} - -func testUnaggregatedEncoder() UnaggregatedEncoder { - return NewUnaggregatedEncoder(NewBufferedEncoder()) -} - -func testUnaggregatedIterator(reader io.Reader) UnaggregatedIterator { - opts := NewUnaggregatedIteratorOptions() - return NewUnaggregatedIterator(reader, opts) -} - -func testUnaggregatedEncodeMetric(encoder UnaggregatedEncoder, m unaggregated.MetricUnion) error { - switch m.Type { - case metric.CounterType: - return encoder.EncodeCounter(m.Counter()) - case metric.TimerType: - return encoder.EncodeBatchTimer(m.BatchTimer()) - case metric.GaugeType: - return encoder.EncodeGauge(m.Gauge()) - default: - return fmt.Errorf("unrecognized metric type %v", m.Type) - } -} - -func testUnaggregatedEncodeMetricWithPoliciesList( - encoder UnaggregatedEncoder, - m unaggregated.MetricUnion, - pl policy.PoliciesList, -) error { - switch m.Type { - case metric.CounterType: - return encoder.EncodeCounterWithPoliciesList(unaggregated.CounterWithPoliciesList{ - Counter: m.Counter(), - PoliciesList: pl, - }) - case metric.TimerType: - return encoder.EncodeBatchTimerWithPoliciesList(unaggregated.BatchTimerWithPoliciesList{ - BatchTimer: m.BatchTimer(), - PoliciesList: pl, - }) - case metric.GaugeType: - return encoder.EncodeGaugeWithPoliciesList(unaggregated.GaugeWithPoliciesList{ - Gauge: m.Gauge(), - PoliciesList: pl, - }) - default: - return fmt.Errorf("unrecognized metric type %v", m.Type) - } -} - -func compareUnaggregatedMetric( - t *testing.T, - expected unaggregated.MetricUnion, - actual unaggregated.MetricUnion, -) { - require.Equal(t, expected.Type, actual.Type) - switch expected.Type { - case metric.CounterType: - require.Equal(t, expected.Counter(), actual.Counter()) - case metric.TimerType: - require.Equal(t, expected.BatchTimer(), actual.BatchTimer()) - case metric.GaugeType: - require.Equal(t, expected.Gauge(), actual.Gauge()) - default: - require.Fail(t, fmt.Sprintf("unrecognized metric type %v", expected.Type)) - } -} - -func comparedPoliciesList(t *testing.T, expected policy.PoliciesList, actual policy.PoliciesList) { - require.Equal(t, len(expected), len(actual)) - for i := 0; i < len(expected); i++ { - require.Equal(t, expected[i].CutoverNanos, actual[i].CutoverNanos) - require.Equal(t, expected[i].Tombstoned, actual[i].Tombstoned) - expectedPolicies, expectedIsDefault := expected[i].Policies() - actualPolicies, actualIsDefault := actual[i].Policies() - require.Equal(t, expectedIsDefault, actualIsDefault) - require.Equal(t, expectedPolicies, actualPolicies) - } -} - -func validateUnaggregatedMetricRoundtrip(t *testing.T, inputs ...unaggregated.MetricUnion) { - encoder := testUnaggregatedEncoder() - it := testUnaggregatedIterator(nil) - validateUnaggregatedMetricRoundtripWithEncoderAndIterator(t, encoder, it, inputs...) -} - -func validateUnaggregatedMetricRoundtripWithEncoderAndIterator( - t *testing.T, - encoder UnaggregatedEncoder, - it UnaggregatedIterator, - inputs ...unaggregated.MetricUnion, -) { - var results []unaggregated.MetricUnion - - // Encode the batch of metrics. - encoder.Reset(NewBufferedEncoder()) - for _, input := range inputs { - err := testUnaggregatedEncodeMetric(encoder, input) - require.NoError(t, err) - } - - // Decode the batch of metrics. - byteStream := bytes.NewBuffer(encoder.Encoder().Bytes()) - it.Reset(byteStream) - for it.Next() { - m := it.Metric() - results = append(results, m) - } - - // Assert the results match expectations. - require.Equal(t, io.EOF, it.Err()) - validateMetrics(t, inputs, results) -} - -func validateMetrics(t *testing.T, inputs, results []unaggregated.MetricUnion) { - require.Equal(t, len(inputs), len(results)) - for i := 0; i < len(inputs); i++ { - compareUnaggregatedMetric(t, inputs[i], results[i]) - } -} - -func validateUnaggregatedMetricWithPoliciesListRoundtrip(t *testing.T, inputs ...metricWithPoliciesList) { - encoder := testUnaggregatedEncoder() - it := testUnaggregatedIterator(nil) - validateUnaggregatedMetricWithPoliciesListRoundtripWithEncoderAndIterator(t, encoder, it, inputs...) -} - -func validateUnaggregatedMetricWithPoliciesListRoundtripWithEncoderAndIterator( - t *testing.T, - encoder UnaggregatedEncoder, - it UnaggregatedIterator, - inputs ...metricWithPoliciesList, -) { - var results []metricWithPoliciesList - - // Encode the batch of metrics. - encoder.Reset(NewBufferedEncoder()) - for _, input := range inputs { - err := testUnaggregatedEncodeMetricWithPoliciesList(encoder, input.metric, input.policiesList) - require.NoError(t, err) - } - - // Decode the batch of metrics. - byteStream := bytes.NewBuffer(encoder.Encoder().Bytes()) - it.Reset(byteStream) - for it.Next() { - m, pl := it.Metric(), it.PoliciesList() - - // Make a copy of cached policies list because it becomes invalid - // on the next Next() call. - pl = toPoliciesList(pl) - - results = append(results, metricWithPoliciesList{ - metric: m, - policiesList: pl, - }) - } - - // Assert the results match expectations. - require.Equal(t, io.EOF, it.Err()) - validateMetricsWithPoliciesList(t, inputs, results) -} - -func validateMetricsWithPoliciesList(t *testing.T, inputs, results []metricWithPoliciesList) { - require.Equal(t, len(inputs), len(results)) - for i := 0; i < len(inputs); i++ { - compareUnaggregatedMetric(t, inputs[i].metric, results[i].metric) - comparedPoliciesList(t, inputs[i].policiesList, results[i].policiesList) - } -} - -func toStagedPolicies(p policy.StagedPolicies) policy.StagedPolicies { - srcPolicies, _ := p.Policies() - destPolicies := make([]policy.Policy, len(srcPolicies)) - copy(destPolicies, srcPolicies) - return policy.NewStagedPolicies(p.CutoverNanos, p.Tombstoned, destPolicies) -} - -func toPoliciesList(pl policy.PoliciesList) policy.PoliciesList { - if pl.IsDefault() { - return policy.DefaultPoliciesList - } - policiesList := make(policy.PoliciesList, 0, len(pl)) - for i := 0; i < len(pl); i++ { - policiesList = append(policiesList, toStagedPolicies(pl[i])) - } - return policiesList -} diff --git a/src/metrics/encoding/msgpack/wire_format.md b/src/metrics/encoding/msgpack/wire_format.md deleted file mode 100644 index 7d896b849b..0000000000 --- a/src/metrics/encoding/msgpack/wire_format.md +++ /dev/null @@ -1,115 +0,0 @@ -## Wire format for unaggregated metrics - -* Message format - * Version - * Number of root object fields - * Root object type - * Root object (can be one of the following): - * CounterWithPoliciesList - * BatchTimerWithPoliciesList - * GaugeWithPoliciesList - -* CounterWithPoliciesList object - * Number of CounterWithPoliciesList fields - * Counter object - * PoliciesList object - -* BatchTimerWithPoliciesList object - * Number of BatchTimerWithPoliciesList fields - * BatchTimer object - * PoliciesList object - -* GaugeWithPoliciesList object - * Number of GaugeWithPoliciesList fields - * Gauge object - * PoliciesList object - -* Counter object - * Number of Counter fields - * Counter ID - * Counter value - -* BatchTimer object - * Number of BatchTimer fields - * BatchTimer ID - * BatchTimer values - -* Gauge object - * Number of Gauge fields - * Gauge ID - * Gauge value - -* PoliciesList object - * Number of PoliciesList fields - * PoliciesList (can be one of the following) - * DefaultPoliciesList - * PoliciesList type - * CustomPoliciesList - * PoliciesList type - * List of StagedPolicies objects - -* StagedPolicies object - * Number of StagedPolicies fields - * Cutover - * Tombstoned - * List of Policy objects - -* Policy object - * Number of Policy fields - * Resolution object - * Retention object - -* Resolution object - * Number of Resolution fields - * Resolution (can be one of the following) - * KnownResolution - * Resolution type - * ResolutionValue - * UnknownResolution - * Resolution type - * Resolution window in nanoseconds - * Resolution precision - -* Retention object - * Number of Retention fields - * Retention (can be one of the following) - * KnownRetention - * Retention type - * RetentionValue - * UnknownRetention - * Retention type - * Retention duration in nanoseconds - -## Wire format for aggregated metrics - -* Message format - * Version - * Number of root object fields - * Root object type - * Root object (can be one of the following): - * RawMetricWithPolicy - -* RawMetricWithPolicy object - * Number of RawMetricWithPolicy fields - * Raw metric object - * Policy object - -* Raw metric object - * Version - * Number of metric fields - * Metric ID - * Metric timestamp - * Metric value - -* Policy object (same format as in unaggregated metrics) - -## Schema changes - -Backward-compatible changes (e.g., adding an additional field to the end of an object) can be -deployed or rolled back separately on the client-side and the server-side. It is unnecessary to -increase the version for backward-compatible changes. - -Backward-incompatible changes (e.g., removing a field or changing a field type) must be deployed -to the server-side first then to the client-side. It is REQUIRED to increase the version for -backward-incompatible changes. If the changes are deployed to the client-side first, the server -will optionally ignore the messages with the higher version.