[
https://issues.apache.org/jira/browse/STORM-1190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14998861#comment-14998861
]
ASF GitHub Bot commented on STORM-1190:
---------------------------------------
Github user revans2 commented on the pull request:
https://github.com/apache/storm/pull/870#issuecomment-155475891
I did some micro-benchmark experiments.
```
public class Test extends Thread {
final long _expectedEnd;
final long _sleepTime;
public Test(long ee, long st) {
_expectedEnd = ee;
_sleepTime = st;
}
public void run() {
try {
while (System.currentTimeMillis() < _expectedEnd) {
Thread.sleep(_sleepTime);
}
} catch (Exception e) {
throw new RuntimeException(e);
}
}
public static void main(String [] args) throws Exception {
long sleepTime = 1;
if (args.length > 0) {
sleepTime = Long.valueOf(args[0]);
}
long totalTimeSec = 100;
if (args.length > 1) {
totalTimeSec = Long.valueOf(args[1]);
}
int totalThreads = 10;
if (args.length > 2) {
totalThreads = Integer.valueOf(args[2]);
}
long totalTimeMs = totalTimeSec * 1000;
long expectedEnd = System.currentTimeMillis() + totalTimeMs;
int ret = -1;
try {
Test [] tests = new Test[totalThreads];
for (int i = 0; i < totalThreads; i++) {
tests[i] = new Test(expectedEnd, sleepTime);
tests[i].start();
}
for (int i = 0; i < totalThreads; i++) {
tests[i].join();
}
ret = 0;
} finally {
System.exit(ret);
}
}
}
```
can make my laptop become unresponsive at 300 threads and a 1 ms sleep time.
```
import java.util.concurrent.*;
import java.util.concurrent.atomic.*;
import java.util.Random;
public class Test extends Thread {
public static final ScheduledThreadPoolExecutor _timer = new
ScheduledThreadPoolExecutor(0);
public static final AtomicLong _error = new AtomicLong(0);
private long _expected;
private final long _sleepTime;
private final Random _rand = new Random();
public Test(long sleepTime) {
_sleepTime = sleepTime;
_expected = System.currentTimeMillis() + sleepTime;
}
public void run() {
long now = System.currentTimeMillis();
long err = Math.abs(now - _expected);
_error.addAndGet(err);
if (_rand.nextInt(100) >= 99) {
try {
Thread.sleep(10);
} catch (Exception e) {
//ignored
}
}
_expected = System.currentTimeMillis() + _sleepTime;
}
public static void main(String [] args) throws Exception {
long sleepTime = 1;
if (args.length > 0) {
sleepTime = Long.valueOf(args[0]);
}
long totalTimeSec = 100;
if (args.length > 1) {
totalTimeSec = Long.valueOf(args[1]);
}
int totalThreads = 10;
if (args.length > 2) {
totalThreads = Integer.valueOf(args[2]);
}
long totalTimeMs = totalTimeSec * 1000;
int ret = -1;
try {
ScheduledFuture<?> [] tests = new ScheduledFuture<?>[totalThreads];
for (int i = 0; i < totalThreads; i++) {
Test t = new Test(sleepTime);
//TODO real one will need locking
_timer.setCorePoolSize(_timer.getCorePoolSize() + 1);
tests[i] = _timer.scheduleWithFixedDelay(t, sleepTime, sleepTime,
TimeUnit.MILLISECONDS);
}
Thread.sleep(totalTimeMs);
for (int i = 0; i < totalThreads; i++) {
tests[i].cancel(true);
}
System.out.println("AVG Error
"+(((double)_error.get())/totalThreads/totalTimeSec));
ret = 0;
} finally {
System.exit(ret);
}
}
}
```
uses almost no CPU and does more or less the same thing, but with more
jitter, and a measurement of how much error there is. I tried various
combinations of thins to improve the performance. Now I think we need to write
a custom SchedulerExecutorService, that can increase and decrease the number of
threads based off of how much load there is. It can purposely run multiple
tasks together, even if they would be spread out with a more accurate clock
waking them up.
> System load spikes in recent snapshot
> -------------------------------------
>
> Key: STORM-1190
> URL: https://issues.apache.org/jira/browse/STORM-1190
> Project: Apache Storm
> Issue Type: Bug
> Components: storm-core
> Affects Versions: 0.11.0
> Environment: 10x (CoreOS stable (766.4.0) / k8s 1.0.1 / docker
> running on Azure VMs)
> Reporter: Michael Schonfeld
> Priority: Critical
> Attachments: Screenshot 2015-11-08 22.17.57.png, Screenshot
> 2015-11-08 22.18.06.png
>
>
> We've been running Storm's snapshots on our production cluster for a little
> while now (that back pressure support really helped us), and we've noticed a
> sudden spike in system load when going from
> commit@ba1250993d10ffc523c9f5464371fbeb406d216f to the current latest
> commit@c12e28c829fcfabc0a3a775fb9714968b7e3e349. Both versions were running
> the exact same topologies, and there was no significant change in workload.
> Not exactly sure how to even begin to debug this, so we ended up just rolling
> back. Thoughts?
> Stats screenshots attached
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)