forked from apache/cassandra
/
StressAction.java
393 lines (340 loc) · 14.6 KB
/
StressAction.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.cassandra.stress;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import org.apache.cassandra.stress.operations.OpDistribution;
import org.apache.cassandra.stress.operations.OpDistributionFactory;
import org.apache.cassandra.stress.settings.ConnectionAPI;
import org.apache.cassandra.stress.settings.SettingsCommand;
import org.apache.cassandra.stress.settings.StressSettings;
import org.apache.cassandra.stress.util.JavaDriverClient;
import org.apache.cassandra.stress.util.ThriftClient;
import org.apache.cassandra.transport.SimpleClient;
import com.google.common.util.concurrent.RateLimiter;
import com.google.common.util.concurrent.Uninterruptibles;
public class StressAction implements Runnable
{
private final StressSettings settings;
private final PrintStream output;
public StressAction(StressSettings settings, PrintStream out)
{
this.settings = settings;
output = out;
}
public void run()
{
// creating keyspace and column families
settings.maybeCreateKeyspaces();
output.println("Sleeping 2s...");
Uninterruptibles.sleepUninterruptibly(2, TimeUnit.SECONDS);
if (!settings.command.noWarmup)
warmup(settings.command.getFactory(settings));
if (settings.command.truncate == SettingsCommand.TruncateWhen.ONCE)
settings.command.truncateTables(settings);
// TODO : move this to a new queue wrapper that gates progress based on a poisson (or configurable) distribution
RateLimiter rateLimiter = null;
if (settings.rate.opRateTargetPerSecond > 0)
rateLimiter = RateLimiter.create(settings.rate.opRateTargetPerSecond);
boolean success;
if (settings.rate.minThreads > 0)
success = runMulti(settings.rate.auto, rateLimiter);
else
success = null != run(settings.command.getFactory(settings), settings.rate.threadCount, settings.command.count,
settings.command.duration, rateLimiter, settings.command.durationUnits, output, false);
if (success)
output.println("END");
else
output.println("FAILURE");
settings.disconnect();
if (!success)
throw new RuntimeException("Failed to execute stress action");
}
// type provided separately to support recursive call for mixed command with each command type it is performing
private void warmup(OpDistributionFactory operations)
{
PrintStream warmupOutput = new PrintStream(new OutputStream() { @Override public void write(int b) throws IOException { } } );
// do 25% of iterations as warmup but no more than 50k (by default hotspot compiles methods after 10k invocations)
int iterations = (settings.command.count > 0
? Math.min(50000, (int)(settings.command.count * 0.25))
: 50000) * settings.node.nodes.size();
int threads = 100;
if (settings.rate.maxThreads > 0)
threads = Math.min(threads, settings.rate.maxThreads);
if (settings.rate.threadCount > 0)
threads = Math.min(threads, settings.rate.threadCount);
for (OpDistributionFactory single : operations.each())
{
// we need to warm up all the nodes in the cluster ideally, but we may not be the only stress instance;
// so warm up all the nodes we're speaking to only.
output.println(String.format("Warming up %s with %d iterations...", single.desc(), iterations));
boolean success = null != run(single, threads, iterations, 0, null, null, warmupOutput, true);
if (!success)
throw new RuntimeException("Failed to execute warmup");
}
}
// TODO : permit varying more than just thread count
// TODO : vary thread count based on percentage improvement of previous increment, not by fixed amounts
private boolean runMulti(boolean auto, RateLimiter rateLimiter)
{
if (settings.command.targetUncertainty >= 0)
output.println("WARNING: uncertainty mode (err<) results in uneven workload between thread runs, so should be used for high level analysis only");
int prevThreadCount = -1;
int threadCount = settings.rate.minThreads;
List<StressMetrics> results = new ArrayList<>();
List<String> runIds = new ArrayList<>();
do
{
output.println("");
output.println(String.format("Running with %d threadCount", threadCount));
if (settings.command.truncate == SettingsCommand.TruncateWhen.ALWAYS)
settings.command.truncateTables(settings);
StressMetrics result = run(settings.command.getFactory(settings), threadCount, settings.command.count,
settings.command.duration, rateLimiter, settings.command.durationUnits, output, false);
if (result == null)
return false;
results.add(result);
if (prevThreadCount > 0)
System.out.println(String.format("Improvement over %d threadCount: %.0f%%",
prevThreadCount, 100 * averageImprovement(results, 1)));
runIds.add(threadCount + " threadCount");
prevThreadCount = threadCount;
if (threadCount < 16)
threadCount *= 2;
else
threadCount *= 1.5;
if (!results.isEmpty() && threadCount > settings.rate.maxThreads)
break;
if (settings.command.type.updates)
{
// pause an arbitrary period of time to let the commit log flush, etc. shouldn't make much difference
// as we only increase load, never decrease it
output.println("Sleeping for 15s");
try
{
Thread.sleep(15 * 1000);
} catch (InterruptedException e)
{
return false;
}
}
// run until we have not improved throughput significantly for previous three runs
} while (!auto || (hasAverageImprovement(results, 3, 0) && hasAverageImprovement(results, 5, settings.command.targetUncertainty)));
// summarise all results
StressMetrics.summarise(runIds, results, output, settings.samples.historyCount);
return true;
}
private boolean hasAverageImprovement(List<StressMetrics> results, int count, double minImprovement)
{
return results.size() < count + 1 || averageImprovement(results, count) >= minImprovement;
}
private double averageImprovement(List<StressMetrics> results, int count)
{
double improvement = 0;
for (int i = results.size() - count ; i < results.size() ; i++)
{
double prev = results.get(i - 1).getTiming().getHistory().opRate();
double cur = results.get(i).getTiming().getHistory().opRate();
improvement += (cur - prev) / prev;
}
return improvement / count;
}
private StressMetrics run(OpDistributionFactory operations,
int threadCount,
long opCount,
long duration,
RateLimiter rateLimiter,
TimeUnit durationUnits,
PrintStream output,
boolean isWarmup)
{
output.println(String.format("Running %s with %d threads %s",
operations.desc(),
threadCount,
durationUnits != null ? duration + " " + durationUnits.toString().toLowerCase()
: opCount > 0 ? "for " + opCount + " iteration"
: "until stderr of mean < " + settings.command.targetUncertainty));
final WorkManager workManager;
if (opCount < 0)
workManager = new WorkManager.ContinuousWorkManager();
else
workManager = new WorkManager.FixedWorkManager(opCount);
final StressMetrics metrics = new StressMetrics(output, settings.log.intervalMillis, settings);
final CountDownLatch done = new CountDownLatch(threadCount);
final Consumer[] consumers = new Consumer[threadCount];
for (int i = 0; i < threadCount; i++)
{
consumers[i] = new Consumer(operations.get(metrics.getTiming(), isWarmup),
done, workManager, metrics, rateLimiter);
}
// starting worker threadCount
for (int i = 0; i < threadCount; i++)
consumers[i].start();
metrics.start();
if (durationUnits != null)
{
Uninterruptibles.sleepUninterruptibly(duration, durationUnits);
workManager.stop();
}
else if (opCount <= 0)
{
try
{
metrics.waitUntilConverges(settings.command.targetUncertainty,
settings.command.minimumUncertaintyMeasurements,
settings.command.maximumUncertaintyMeasurements);
} catch (InterruptedException e) { }
workManager.stop();
}
try
{
done.await();
metrics.stop();
}
catch (InterruptedException e) {}
if (metrics.wasCancelled())
return null;
metrics.summarise();
boolean success = true;
for (Consumer consumer : consumers)
success &= consumer.success;
if (!success)
return null;
return metrics;
}
private static class StreamOfOperations {
private final OpDistribution operations;
private final RateLimiter rateLimiter;
private final WorkManager workManager;
public StreamOfOperations(OpDistribution operations, RateLimiter rateLimiter, WorkManager workManager)
{
this.operations = operations;
this.rateLimiter = rateLimiter;
this.workManager = workManager;
}
Operation nextOp()
{
Operation op = operations.next();
final int partitionCount = op.ready(workManager);
if (partitionCount == 0)
return null;
if (rateLimiter != null)
rateLimiter.acquire(partitionCount);
return op;
}
void close()
{
operations.closeTimers();
}
void abort()
{
workManager.stop();
}
}
private class Consumer extends Thread
{
private final StreamOfOperations opStream;
private final StressMetrics metrics;
private volatile boolean success = true;
private final CountDownLatch done;
public Consumer(OpDistribution operations,
CountDownLatch done,
WorkManager workManager,
StressMetrics metrics,
RateLimiter rateLimiter)
{
this.done = done;
this.metrics = metrics;
this.opStream = new StreamOfOperations(operations, rateLimiter, workManager);
}
public void run()
{
try
{
SimpleClient sclient = null;
ThriftClient tclient = null;
JavaDriverClient jclient = null;
final ConnectionAPI clientType = settings.mode.api;
switch (clientType)
{
case JAVA_DRIVER_NATIVE:
jclient = settings.getJavaDriverClient();
break;
case SIMPLE_NATIVE:
sclient = settings.getSimpleNativeClient();
break;
case THRIFT:
case THRIFT_SMART:
tclient = settings.getThriftClient();
break;
default:
throw new IllegalStateException();
}
while (true)
{
// Assumption: All ops are thread local, operations are never shared across threads.
Operation op = opStream.nextOp();
if (op == null)
break;
try
{
switch (clientType)
{
case JAVA_DRIVER_NATIVE:
op.run(jclient);
break;
case SIMPLE_NATIVE:
op.run(sclient);
break;
case THRIFT:
case THRIFT_SMART:
default:
op.run(tclient);
}
}
catch (Exception e)
{
if (output == null)
System.err.println(e.getMessage());
else
e.printStackTrace(output);
success = false;
opStream.abort();
metrics.cancel();
return;
}
}
}
catch (Exception e)
{
System.err.println(e.getMessage());
success = false;
}
finally
{
done.countDown();
opStream.close();
}
}
}
}