github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/aggregator/integration/resend_stress_test.go (about) 1 //go: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/m3db/m3/src/aggregator/aggregator" 32 aggclient "github.com/m3db/m3/src/aggregator/client" 33 "github.com/m3db/m3/src/cluster/placement" 34 maggregation "github.com/m3db/m3/src/metrics/aggregation" 35 "github.com/m3db/m3/src/metrics/metadata" 36 "github.com/m3db/m3/src/metrics/metric" 37 metricid "github.com/m3db/m3/src/metrics/metric/id" 38 "github.com/m3db/m3/src/metrics/metric/unaggregated" 39 "github.com/m3db/m3/src/metrics/pipeline" 40 "github.com/m3db/m3/src/metrics/pipeline/applied" 41 "github.com/m3db/m3/src/metrics/policy" 42 "github.com/m3db/m3/src/metrics/transformation" 43 "github.com/m3db/m3/src/x/clock" 44 "github.com/m3db/m3/src/x/instrument" 45 xtest "github.com/m3db/m3/src/x/test" 46 xtime "github.com/m3db/m3/src/x/time" 47 48 "github.com/stretchr/testify/require" 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 shardFn := func(id []byte, numShards uint32) uint32 { 101 if pipelineRollupID == string(id) { 102 return numShards - 1 103 } 104 return 0 105 } 106 107 // Admin client connection options setup. 108 connectionOpts := aggclient.NewConnectionOptions(). 109 SetInitReconnectThreshold(1). 110 SetMaxReconnectThreshold(1). 111 SetMaxReconnectDuration(2 * time.Second). 112 SetWriteTimeout(time.Second) 113 114 // Create servers. 115 instrumentOpts := instrument.NewOptions() 116 logger := xtest.NewLogger(t) 117 instrumentOpts = instrumentOpts.SetLogger(logger) 118 serverOpts = serverOpts. 119 SetBufferForPastTimedMetric(time.Second * 5). 120 SetMaxAllowedForwardingDelayFn(func(resolution time.Duration, numForwardedTimes int) time.Duration { 121 return resolution 122 }). 123 // allow testing entry Closing 124 SetEntryTTL(time.Second). 125 SetInstrumentOptions(instrumentOpts). 126 SetElectionCluster(electionCluster). 127 SetHTTPAddr(serverSetup.httpAddr). 128 SetInstanceID(serverSetup.instanceConfig.instanceID). 129 SetTopicName(defaultTopicName). 130 SetRawTCPAddr(serverSetup.rawTCPAddr). 131 SetM3MsgAddr(serverSetup.m3MsgAddr). 132 SetShardFn(shardFn). 133 SetShardSetID(serverSetup.instanceConfig.shardSetID). 134 SetClientConnectionOptions(connectionOpts). 135 SetDiscardNaNAggregatedValues(false) 136 server := newTestServerSetup(t, serverOpts) 137 138 // Start the servers. 139 log := xtest.NewLogger(t) 140 require.NoError(t, server.startServer()) 141 log.Sugar().Info("server is now up") 142 143 // Waiting for server to be leader. 144 err = server.waitUntilLeader() 145 require.NoError(t, err) 146 147 var ( 148 storagePolicies = policy.StoragePolicies{ 149 policy.NewStoragePolicy(5*time.Millisecond, xtime.Second, time.Hour), 150 } 151 ) 152 153 stagedMetadatas := metadata.StagedMetadatas{ 154 { 155 CutoverNanos: 0, 156 Tombstoned: false, 157 Metadata: metadata.Metadata{ 158 Pipelines: []metadata.PipelineMetadata{ 159 { 160 AggregationID: maggregation.DefaultID, 161 ResendEnabled: true, 162 StoragePolicies: storagePolicies, 163 Pipeline: applied.NewPipeline([]applied.OpUnion{ 164 { 165 Type: pipeline.TransformationOpType, 166 Transformation: pipeline.TransformationOp{Type: transformation.Increase}, 167 }, 168 { 169 Type: pipeline.RollupOpType, 170 Rollup: applied.RollupOp{ 171 ID: []byte(pipelineRollupID), 172 AggregationID: maggregation.MustCompressTypes(maggregation.Sum), 173 }, 174 }, 175 { 176 Type: pipeline.TransformationOpType, 177 Transformation: pipeline.TransformationOp{Type: transformation.Reset}, 178 }, 179 }), 180 }, 181 }, 182 }, 183 }, 184 } 185 186 start := time.Now().Truncate(time.Millisecond * 5) 187 log.Sugar().Infof("Start time: %v", start) 188 c1 := server.newClient(t) 189 c2 := server.newClient(t) 190 stop1 := make(chan struct{}, 1) 191 stop2 := make(chan struct{}, 1) 192 resolution := storagePolicies[0].Resolution().Window 193 var wg sync.WaitGroup 194 wg.Add(3) 195 go func() { 196 writeMetrics(t, c1, stop1, "foo", start, resolution, stagedMetadatas) 197 wg.Done() 198 }() 199 go func() { 200 writeMetrics(t, c2, stop2, "bar", start, resolution, stagedMetadatas) 201 wg.Done() 202 }() 203 go func() { 204 defer wg.Done() 205 ticker := time.NewTicker(time.Millisecond) 206 expectedTs := start.Add(time.Millisecond * 5) 207 zero := false 208 rollupMetricID := metricid.ChunkedID{ 209 Prefix: aggregator.NewOptions(clock.NewOptions()).FullGaugePrefix(), 210 Data: []byte(pipelineRollupID), 211 }.String() 212 round := 0 213 for range ticker.C { 214 if zero { 215 if server.removeIf(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0], 0) { 216 zero = false 217 expectedTs = expectedTs.Add(time.Millisecond * 2).Add(time.Microsecond * 500) 218 round++ 219 } 220 } else { 221 if server.removeIf(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0], 10) { 222 zero = true 223 expectedTs = expectedTs.Add(time.Millisecond * 2).Add(time.Microsecond * 500) 224 } 225 } 226 if time.Now().Sub(expectedTs) > time.Second*5 { 227 actualVal := server.value(expectedTs.UnixNano(), rollupMetricID, storagePolicies[0]) 228 log.Sugar().Fatalf("failed for: ts:=%v value=%v zero=%v, round=%v", 229 expectedTs, actualVal, zero, round) 230 } 231 232 if round > 2000 { 233 stop1 <- struct{}{} 234 stop2 <- struct{}{} 235 ticker.Stop() 236 return 237 } 238 } 239 }() 240 241 wg.Wait() 242 log.Sugar().Infof("done. Ran for %v\n", time.Since(start)) 243 244 // give time for the aggregations to Close. 245 time.Sleep(time.Second * 5) 246 247 require.NoError(t, c1.close()) 248 require.NoError(t, c2.close()) 249 250 electionCluster.Close() 251 252 // Stop the server. 253 require.NoError(t, server.stopServer()) 254 log.Sugar().Info("server is now down") 255 } 256 257 func writeMetrics( 258 t *testing.T, 259 c *client, 260 stop chan struct{}, 261 metricID string, 262 start time.Time, 263 resolution time.Duration, 264 stagedMetadatas metadata.StagedMetadatas) { 265 require.NoError(t, c.connect()) 266 value := 0.0 267 ts := start.Add(time.Millisecond * -1) 268 ticker := time.NewTicker(resolution) 269 delay := time.Millisecond * 60 270 rnd := rand.New(rand.NewSource(10)) //nolint:gosec 271 var wg sync.WaitGroup 272 for range ticker.C { 273 select { 274 case <-stop: 275 wg.Wait() 276 return 277 default: 278 } 279 ts = ts.Add(resolution) 280 value += 5.0 281 wg.Add(1) 282 go func(ts time.Time, value float64) { 283 defer wg.Done() 284 if rnd.Intn(3) == 0 { 285 time.Sleep(delay) 286 } 287 m := unaggregated.MetricUnion{ 288 Type: metric.GaugeType, 289 ID: metricid.RawID(metricID), 290 ClientTimeNanos: xtime.ToUnixNano(ts), 291 GaugeVal: value, 292 } 293 require.NoError(t, c.writeUntimedMetricWithMetadatas(m, stagedMetadatas)) 294 require.NoError(t, c.flush()) 295 }(ts, value) 296 } 297 }