github.com/m3db/m3@v1.5.0/src/aggregator/integration/resend_stress_test.go (about) 1 // +build integration 2 3 // Copyright (c) 2018 Uber Technologies, Inc. 4 // 5 // Permission is hereby granted, free of charge, to any person obtaining a copy 6 // of this software and associated documentation files (the "Software"), to deal 7 // in the Software without restriction, including without limitation the rights 8 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 9 // copies of the Software, and to permit persons to whom the Software is 10 // furnished to do so, subject to the following conditions: 11 // 12 // The above copyright notice and this permission notice shall be included in 13 // all copies or substantial portions of the Software. 14 // 15 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 16 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 17 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 18 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 19 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 20 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 21 // THE SOFTWARE. 22 23 package integration 24 25 import ( 26 "math/rand" 27 "sync" 28 "testing" 29 "time" 30 31 "github.com/stretchr/testify/require" 32 33 "github.com/m3db/m3/src/aggregator/aggregator" 34 aggclient "github.com/m3db/m3/src/aggregator/client" 35 "github.com/m3db/m3/src/cluster/placement" 36 maggregation "github.com/m3db/m3/src/metrics/aggregation" 37 "github.com/m3db/m3/src/metrics/metadata" 38 "github.com/m3db/m3/src/metrics/metric" 39 metricid "github.com/m3db/m3/src/metrics/metric/id" 40 "github.com/m3db/m3/src/metrics/metric/unaggregated" 41 "github.com/m3db/m3/src/metrics/pipeline" 42 "github.com/m3db/m3/src/metrics/pipeline/applied" 43 "github.com/m3db/m3/src/metrics/policy" 44 "github.com/m3db/m3/src/metrics/transformation" 45 "github.com/m3db/m3/src/x/clock" 46 "github.com/m3db/m3/src/x/instrument" 47 xtest "github.com/m3db/m3/src/x/test" 48 xtime "github.com/m3db/m3/src/x/time" 49 ) 50 51 //nolint 52 func TestResendAggregatedValueStress(t *testing.T) { 53 if testing.Short() { 54 t.SkipNow() 55 } 56 57 aggregatorClientType, err := getAggregatorClientTypeFromEnv() 58 require.NoError(t, err) 59 60 serverOpts := newTestServerOptions(t) 61 62 // Placement setup. 63 var ( 64 numTotalShards = 1024 65 placementKey = "/placement" 66 ) 67 serverSetup := struct { 68 rawTCPAddr string 69 httpAddr string 70 m3MsgAddr string 71 instanceConfig placementInstanceConfig 72 }{ 73 rawTCPAddr: "localhost:6000", 74 httpAddr: "localhost:16000", 75 m3MsgAddr: "localhost:26000", 76 instanceConfig: placementInstanceConfig{ 77 instanceID: "localhost:6000", 78 shardSetID: 1, 79 shardStartInclusive: 0, 80 shardEndExclusive: 1024, 81 }, 82 } 83 84 serverSetup.instanceConfig.instanceID = serverSetup.rawTCPAddr 85 if aggregatorClientType == aggclient.M3MsgAggregatorClient { 86 serverSetup.instanceConfig.instanceID = serverSetup.m3MsgAddr 87 } 88 89 instance := serverSetup.instanceConfig.newPlacementInstance() 90 initPlacement := newPlacement(numTotalShards, []placement.Instance{instance}) 91 setPlacement(t, placementKey, serverOpts.ClusterClient(), initPlacement) 92 serverOpts = setupTopic(t, serverOpts, initPlacement) 93 require.NoError(t, err) 94 95 // Election cluster setup. 96 electionCluster := newTestCluster(t) 97 98 // Sharding function maps all metrics to shard 0 except for the rollup metric, 99 // which gets mapped to the last shard. 100 pipelineRollupID := "pipelineRollup" 101 shardFn := func(id []byte, numShards uint32) uint32 { 102 if pipelineRollupID == string(id) { 103 return numShards - 1 104 } 105 return 0 106 } 107 108 // Admin client connection options setup. 109 connectionOpts := aggclient.NewConnectionOptions(). 110 SetInitReconnectThreshold(1). 111 SetMaxReconnectThreshold(1). 112 SetMaxReconnectDuration(2 * time.Second). 113 SetWriteTimeout(time.Second) 114 115 // Create servers. 116 instrumentOpts := instrument.NewOptions() 117 logger := xtest.NewLogger(t) 118 instrumentOpts = instrumentOpts.SetLogger(logger) 119 serverOpts = serverOpts. 120 SetBufferForPastTimedMetric(time.Second * 5). 121 SetMaxAllowedForwardingDelayFn(func(resolution time.Duration, numForwardedTimes int) time.Duration { 122 return resolution 123 }). 124 // allow testing entry Closing 125 SetEntryTTL(time.Second). 126 SetInstrumentOptions(instrumentOpts). 127 SetElectionCluster(electionCluster). 128 SetHTTPAddr(serverSetup.httpAddr). 129 SetInstanceID(serverSetup.instanceConfig.instanceID). 130 SetTopicName(defaultTopicName). 131 SetRawTCPAddr(serverSetup.rawTCPAddr). 132 SetM3MsgAddr(serverSetup.m3MsgAddr). 133 SetShardFn(shardFn). 134 SetShardSetID(serverSetup.instanceConfig.shardSetID). 135 SetClientConnectionOptions(connectionOpts). 136 SetDiscardNaNAggregatedValues(false) 137 server := newTestServerSetup(t, serverOpts) 138 139 // Start the servers. 140 log := xtest.NewLogger(t) 141 require.NoError(t, server.startServer()) 142 log.Sugar().Info("server is now up") 143 144 // Waiting for server to be leader. 145 err = server.waitUntilLeader() 146 require.NoError(t, err) 147 148 var ( 149 storagePolicies = policy.StoragePolicies{ 150 policy.NewStoragePolicy(5*time.Millisecond, xtime.Second, time.Hour), 151 } 152 ) 153 154 stagedMetadatas := metadata.StagedMetadatas{ 155 { 156 CutoverNanos: 0, 157 Tombstoned: false, 158 Metadata: metadata.Metadata{ 159 Pipelines: []metadata.PipelineMetadata{ 160 { 161 AggregationID: maggregation.DefaultID, 162 ResendEnabled: true, 163 StoragePolicies: storagePolicies, 164 Pipeline: applied.NewPipeline([]applied.OpUnion{ 165 { 166 Type: pipeline.TransformationOpType, 167 Transformation: pipeline.TransformationOp{Type: transformation.Increase}, 168 }, 169 { 170 Type: pipeline.RollupOpType, 171 Rollup: applied.RollupOp{ 172 ID: []byte(pipelineRollupID), 173 AggregationID: maggregation.MustCompressTypes(maggregation.Sum), 174 }, 175 }, 176 { 177 Type: pipeline.TransformationOpType, 178 Transformation: pipeline.TransformationOp{Type: transformation.Reset}, 179 }, 180 }), 181 }, 182 }, 183 }, 184 }, 185 } 186 187 start := time.Now().Truncate(time.Millisecond * 5) 188 log.Sugar().Infof("Start time: %v", start) 189 c1 := server.newClient(t) 190 c2 := server.newClient(t) 191 stop1 := make(chan struct{}, 1) 192 stop2 := make(chan struct{}, 1) 193 resolution := storagePolicies[0].Resolution().Window 194 var wg sync.WaitGroup 195 wg.Add(3) 196 go func() { 197 writeMetrics(t, c1, stop1, "foo", start, resolution, stagedMetadatas) 198 wg.Done() 199 }() 200 go func() { 201 writeMetrics(t, c2, stop2, "bar", start, resolution, stagedMetadatas) 202 wg.Done() 203 }() 204 go func() { 205 defer wg.Done() 206 ticker := time.NewTicker(time.Millisecond) 207 expectedTs := start.Add(time.Millisecond * 5) 208 zero := false 209 rollupMetricID := metricid.ChunkedID{ 210 Prefix: aggregator.NewOptions(clock.NewOptions()).FullGaugePrefix(), 211 Data: []byte(pipelineRollupID), 212 }.String() 213 round := 0 214 for range ticker.C { 215 if zero { 216 if server.removeIf(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0], 0) { 217 zero = false 218 expectedTs = expectedTs.Add(time.Millisecond * 2).Add(time.Microsecond * 500) 219 round++ 220 } 221 } else { 222 if server.removeIf(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0], 10) { 223 zero = true 224 expectedTs = expectedTs.Add(time.Millisecond * 2).Add(time.Microsecond * 500) 225 } 226 } 227 if time.Now().Sub(expectedTs) > time.Second*5 { 228 actualVal := server.value(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0]) 229 log.Sugar().Fatalf("failed for: ts:=%v value=%v zero=%v, round=%v", 230 expectedTs, actualVal, zero, round) 231 } 232 233 if round > 2000 { 234 stop1 <- struct{}{} 235 stop2 <- struct{}{} 236 ticker.Stop() 237 return 238 } 239 } 240 }() 241 242 wg.Wait() 243 log.Sugar().Infof("done. Ran for %v\n", time.Since(start)) 244 245 // give time for the aggregations to Close. 246 time.Sleep(time.Second * 5) 247 248 require.NoError(t, c1.close()) 249 require.NoError(t, c2.close()) 250 251 electionCluster.Close() 252 253 // Stop the server. 254 require.NoError(t, server.stopServer()) 255 log.Sugar().Info("server is now down") 256 } 257 258 func writeMetrics( 259 t *testing.T, 260 c *client, 261 stop chan struct{}, 262 metricID string, 263 start time.Time, 264 resolution time.Duration, 265 stagedMetadatas metadata.StagedMetadatas) { 266 require.NoError(t, c.connect()) 267 value := 0.0 268 ts := start.Add(time.Millisecond * -1) 269 ticker := time.NewTicker(resolution) 270 delay := time.Millisecond * 60 271 rnd := rand.New(rand.NewSource(10)) //nolint:gosec 272 var wg sync.WaitGroup 273 for range ticker.C { 274 select { 275 case <-stop: 276 wg.Wait() 277 return 278 default: 279 } 280 ts = ts.Add(resolution) 281 value += 5.0 282 wg.Add(1) 283 go func(ts time.Time, value float64) { 284 defer wg.Done() 285 if rnd.Intn(3) == 0 { 286 time.Sleep(delay) 287 } 288 m := unaggregated.MetricUnion{ 289 Type: metric.GaugeType, 290 ID: metricid.RawID(metricID), 291 ClientTimeNanos: xtime.ToUnixNano(ts), 292 GaugeVal: value, 293 } 294 require.NoError(t, c.writeUntimedMetricWithMetadatas(m, stagedMetadatas)) 295 require.NoError(t, c.flush()) 296 }(ts, value) 297 } 298 }