-
Notifications
You must be signed in to change notification settings - Fork 4.1k
/
LocalCluster.java
1306 lines (1158 loc) · 49.2 KB
/
LocalCluster.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
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.storm;
import java.lang.reflect.Method;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.UnaryOperator;
import net.minidev.json.JSONValue;
import net.minidev.json.parser.ParseException;
import org.apache.storm.blobstore.BlobStore;
import org.apache.storm.cluster.ClusterStateContext;
import org.apache.storm.cluster.ClusterUtils;
import org.apache.storm.cluster.DaemonType;
import org.apache.storm.cluster.IStateStorage;
import org.apache.storm.cluster.IStormClusterState;
import org.apache.storm.daemon.Acker;
import org.apache.storm.daemon.DaemonCommon;
import org.apache.storm.daemon.Shutdownable;
import org.apache.storm.daemon.StormCommon;
import org.apache.storm.daemon.nimbus.Nimbus;
import org.apache.storm.daemon.nimbus.Nimbus.StandaloneINimbus;
import org.apache.storm.daemon.nimbus.TopoCache;
import org.apache.storm.daemon.supervisor.ReadClusterState;
import org.apache.storm.daemon.supervisor.StandaloneSupervisor;
import org.apache.storm.daemon.supervisor.Supervisor;
import org.apache.storm.executor.LocalExecutor;
import org.apache.storm.generated.AlreadyAliveException;
import org.apache.storm.generated.AuthorizationException;
import org.apache.storm.generated.BeginDownloadResult;
import org.apache.storm.generated.ClusterSummary;
import org.apache.storm.generated.ComponentPageInfo;
import org.apache.storm.generated.Credentials;
import org.apache.storm.generated.GetInfoOptions;
import org.apache.storm.generated.InvalidTopologyException;
import org.apache.storm.generated.KeyAlreadyExistsException;
import org.apache.storm.generated.KeyNotFoundException;
import org.apache.storm.generated.KillOptions;
import org.apache.storm.generated.ListBlobsResult;
import org.apache.storm.generated.LogConfig;
import org.apache.storm.generated.Nimbus.Iface;
import org.apache.storm.generated.Nimbus.Processor;
import org.apache.storm.generated.NimbusSummary;
import org.apache.storm.generated.NotAliveException;
import org.apache.storm.generated.OwnerResourceSummary;
import org.apache.storm.generated.ProfileAction;
import org.apache.storm.generated.ProfileRequest;
import org.apache.storm.generated.ReadableBlobMeta;
import org.apache.storm.generated.RebalanceOptions;
import org.apache.storm.generated.SettableBlobMeta;
import org.apache.storm.generated.StormTopology;
import org.apache.storm.generated.SubmitOptions;
import org.apache.storm.generated.SupervisorAssignments;
import org.apache.storm.generated.SupervisorPageInfo;
import org.apache.storm.generated.SupervisorWorkerHeartbeat;
import org.apache.storm.generated.SupervisorWorkerHeartbeats;
import org.apache.storm.generated.TopologyHistoryInfo;
import org.apache.storm.generated.TopologyInfo;
import org.apache.storm.generated.TopologyPageInfo;
import org.apache.storm.generated.TopologySummary;
import org.apache.storm.generated.WorkerMetrics;
import org.apache.storm.messaging.IContext;
import org.apache.storm.messaging.local.Context;
import org.apache.storm.metric.StormMetricsRegistry;
import org.apache.storm.nimbus.ILeaderElector;
import org.apache.storm.scheduler.INimbus;
import org.apache.storm.scheduler.ISupervisor;
import org.apache.storm.security.auth.IGroupMappingServiceProvider;
import org.apache.storm.security.auth.ThriftConnectionType;
import org.apache.storm.security.auth.ThriftServer;
import org.apache.storm.shade.org.apache.zookeeper.server.ServerConfig;
import org.apache.storm.task.IBolt;
import org.apache.storm.testing.InProcessZookeeper;
import org.apache.storm.testing.NonRichBoltTracker;
import org.apache.storm.testing.TmpPath;
import org.apache.storm.testing.TrackedTopology;
import org.apache.storm.thrift.TException;
import org.apache.storm.utils.ConfigUtils;
import org.apache.storm.utils.DRPCClient;
import org.apache.storm.utils.NimbusClient;
import org.apache.storm.utils.ObjectReader;
import org.apache.storm.utils.RegisteredGlobalState;
import org.apache.storm.utils.StormCommonInstaller;
import org.apache.storm.utils.Time;
import org.apache.storm.utils.Time.SimulatedTime;
import org.apache.storm.utils.Utils;
import org.apache.storm.utils.WrappedAuthorizationException;
import org.apache.storm.utils.WrappedKeyNotFoundException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A stand alone storm cluster that runs inside a single process. It is intended to be used for testing. Both internal testing for Apache
* Storm itself and for people building storm topologies.
* <p>
* LocalCluster is an AutoCloseable so if you are using it in tests you can use a try block to be sure it is shut down.
* </p>
* try (LocalCluster cluster = new LocalCluster()) { // Do some tests } // The cluster has been shut down.
*/
public class LocalCluster implements ILocalClusterTrackedTopologyAware, Iface {
public static final KillOptions KILL_NOW = new KillOptions();
private static final Logger LOG = LoggerFactory.getLogger(LocalCluster.class);
private static final long DEFAULT_ZK_PORT = 2181;
static {
KILL_NOW.set_wait_secs(0);
}
private final Nimbus nimbus;
//This is very private and does not need to be exposed
private int portCounter;
private final Map<String, Object> daemonConf;
private final List<Supervisor> supervisors;
private final IStateStorage state;
private final IStormClusterState clusterState;
private final String stormHomeBackup;
private final List<TmpPath> tmpDirs;
private final InProcessZookeeper zookeeper;
private final IContext sharedContext;
private final ThriftServer thriftServer;
private final String trackId;
private final StormCommonInstaller commonInstaller;
private final SimulatedTime time;
private final NimbusClient.LocalOverride nimbusOverride;
private final StormMetricsRegistry metricRegistry;
/**
* Create a default LocalCluster.
*
* @throws Exception on any error
*/
public LocalCluster() throws Exception {
this(new Builder().withDaemonConf(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true));
}
/**
* Create a LocalCluster that connects to an existing Zookeeper instance.
*
* @param zkHost the host for ZK
* @param zkPort the port for ZK
* @throws Exception on any error
*/
public LocalCluster(String zkHost, Long zkPort) throws Exception {
this(new Builder().withDaemonConf(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true)
.withDaemonConf(Config.STORM_ZOOKEEPER_SERVERS, Arrays.asList(zkHost))
.withDaemonConf(Config.STORM_ZOOKEEPER_PORT, zkPort));
}
@SuppressWarnings("deprecation")
private LocalCluster(Builder builder) throws Exception {
if (builder.simulateTime) {
time = new SimulatedTime();
} else {
time = null;
}
boolean success = false;
try {
this.trackId = builder.trackId;
if (trackId != null) {
ConcurrentHashMap<String, AtomicInteger> metrics = new ConcurrentHashMap<>();
metrics.put("spout-emitted", new AtomicInteger(0));
metrics.put("transferred", new AtomicInteger(0));
metrics.put("processed", new AtomicInteger(0));
this.commonInstaller = new StormCommonInstaller(new TrackedStormCommon(this.trackId));
LOG.warn("Adding tracked metrics for ID {}", this.trackId);
RegisteredGlobalState.setState(this.trackId, metrics);
LocalExecutor.setTrackId(this.trackId);
} else {
this.commonInstaller = null;
}
this.tmpDirs = new ArrayList<>();
this.supervisors = new ArrayList<>();
TmpPath nimbusTmp = new TmpPath();
this.tmpDirs.add(nimbusTmp);
stormHomeBackup = System.getProperty(ConfigUtils.STORM_HOME);
TmpPath stormHome = new TmpPath();
if (!stormHome.getFile().mkdirs()) {
throw new IllegalStateException("Failed to create storm.home directory " + stormHome.getPath());
}
this.tmpDirs.add(stormHome);
System.setProperty(ConfigUtils.STORM_HOME, stormHome.getPath());
Map<String, Object> conf = ConfigUtils.readStormConfig();
conf.put(Config.TOPOLOGY_SKIP_MISSING_KRYO_REGISTRATIONS, true);
conf.put(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, false);
conf.put(Config.TOPOLOGY_TRIDENT_BATCH_EMIT_INTERVAL_MILLIS, 50);
conf.put(Config.STORM_CLUSTER_MODE, "local");
conf.put(Config.BLOBSTORE_DIR, nimbusTmp.getPath());
conf.put(Config.TOPOLOGY_MIN_REPLICATION_COUNT, 1);
InProcessZookeeper zookeeper = null;
if (!builder.daemonConf.containsKey(Config.STORM_ZOOKEEPER_SERVERS)) {
zookeeper = new InProcessZookeeper();
conf.put(Config.STORM_ZOOKEEPER_PORT, zookeeper.getPort());
conf.put(Config.STORM_ZOOKEEPER_SERVERS, Arrays.asList("localhost"));
}
this.zookeeper = zookeeper;
conf.putAll(builder.daemonConf);
this.daemonConf = new HashMap<>(conf);
this.metricRegistry = new StormMetricsRegistry();
this.portCounter = builder.supervisorSlotPortMin;
ClusterStateContext cs = new ClusterStateContext(DaemonType.NIMBUS, daemonConf);
this.state = ClusterUtils.mkStateStorage(this.daemonConf, null, cs);
if (builder.clusterState == null) {
clusterState = ClusterUtils.mkStormClusterState(this.daemonConf, cs);
} else {
this.clusterState = builder.clusterState;
}
if (!Time.isSimulating()) {
//Ensure Nimbus assigns topologies as quickly as possible
conf.put(DaemonConfig.NIMBUS_MONITOR_FREQ_SECS, 1);
}
//Set it for nimbus only
conf.put(Config.STORM_LOCAL_DIR, nimbusTmp.getPath());
Nimbus nimbus = new Nimbus(conf, builder.inimbus == null ? new StandaloneINimbus() : builder.inimbus,
this.getClusterState(), null, builder.store, builder.topoCache, builder.leaderElector,
builder.groupMapper, metricRegistry);
if (builder.nimbusWrapper != null) {
nimbus = builder.nimbusWrapper.apply(nimbus);
}
this.nimbus = nimbus;
this.nimbus.launchServer();
if (!this.nimbus.awaitLeadership(Testing.TEST_TIMEOUT_MS, TimeUnit.MILLISECONDS)) {
//Ensure Nimbus has leadership, otherwise topology submission will fail.
throw new RuntimeException("LocalCluster Nimbus failed to gain leadership.");
}
IContext context = null;
if (!ObjectReader.getBoolean(this.daemonConf.get(Config.STORM_LOCAL_MODE_ZMQ), false)) {
context = new Context();
context.prepare(this.daemonConf);
}
this.sharedContext = context;
this.thriftServer = builder.nimbusDaemon ? startNimbusDaemon(this.daemonConf, this.nimbus) : null;
for (int i = 0; i < builder.supervisors; i++) {
addSupervisor(builder.portsPerSupervisor, null, null);
}
//Wait for a leader to be elected (or topology submission can be rejected)
try {
long timeoutAfter = System.currentTimeMillis() + 10_000;
while (!hasLeader()) {
if (timeoutAfter > System.currentTimeMillis()) {
throw new IllegalStateException("Timed out waiting for nimbus to become the leader");
}
Thread.sleep(1);
}
} catch (Exception e) {
//Ignore any exceptions we might be doing a test for authentication
}
if (thriftServer == null) {
//We don't want to override the client if there is a thrift server up and running, or we would not test any
// Of the actual thrift code
this.nimbusOverride = new NimbusClient.LocalOverride(this);
} else {
this.nimbusOverride = null;
}
success = true;
metricRegistry.startMetricsReporters(daemonConf);
} finally {
if (!success) {
close();
}
}
}
private static ThriftServer startNimbusDaemon(Map<String, Object> conf, Nimbus nimbus) {
ThriftServer ret = new ThriftServer(conf, new Processor<>(nimbus), ThriftConnectionType.NIMBUS);
LOG.info("Starting Nimbus server...");
new Thread(() -> ret.serve()).start();
return ret;
}
private static boolean areAllWorkersWaiting() {
boolean ret = true;
for (Shutdownable s : ProcessSimulator.getAllProcessHandles()) {
if (s instanceof DaemonCommon) {
ret = ret && ((DaemonCommon) s).isWaiting();
}
}
return ret;
}
/**
* Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls. NOTE local mode override happens by default now
* unless netty is turned on for the local cluster.
*
* @param c the callable to run in this mode
* @param ttlSec the number of seconds to let the cluster run after c has completed
* @return the result of calling C
*
* @throws Exception on any Exception.
*/
public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec) throws Exception {
return withLocalModeOverride(c, ttlSec, null);
}
/**
* Run c with a local mode cluster overriding the NimbusClient and DRPCClient calls. NOTE local mode override happens by default now
* unless netty is turned on for the local cluster.
*
* @param c the callable to run in this mode
* @param ttlSec the number of seconds to let the cluster run after c has completed
* @param daemonConf configs to set for the daemon processes.
* @return the result of calling C
*
* @throws Exception on any Exception.
*/
@SuppressWarnings("checkstyle:VariableDeclarationUsageDistance")
public static <T> T withLocalModeOverride(Callable<T> c, long ttlSec, Map<String, Object> daemonConf) throws Exception {
LOG.info("\n\n\t\tSTARTING LOCAL MODE CLUSTER\n\n");
Builder builder = new Builder();
if (daemonConf != null) {
builder.withDaemonConf(daemonConf);
}
try (LocalCluster local = builder.build();
LocalDRPC drpc = new LocalDRPC(local.metricRegistry);
DRPCClient.LocalOverride drpcOverride = new DRPCClient.LocalOverride(drpc)) {
T ret = c.call();
LOG.info("\n\n\t\tRUNNING LOCAL CLUSTER for {} seconds.\n\n", ttlSec);
Thread.sleep(ttlSec * 1000);
LOG.info("\n\n\t\tSTOPPING LOCAL MODE CLUSTER\n\n");
return ret;
}
}
/**
* Main entry point to running in local mode.
* @param args arguments to be run in local mode
* @throws Exception on any error when running.
*/
public static void main(final String[] args) throws Exception {
if (args.length < 1) {
throw new IllegalArgumentException("No class was specified to run");
}
long ttl = 20;
String ttlString = System.getProperty("storm.local.sleeptime", "20");
try {
ttl = Long.valueOf(ttlString);
} catch (NumberFormatException e) {
LOG.warn("could not parse the sleep time defaulting to {} seconds", ttl);
}
Map<String, Object> daemonConf = new HashMap<>();
String zkOverride = System.getProperty("storm.local.zookeeper");
if (zkOverride != null) {
LOG.info("Using ZooKeeper at '{}' instead of in-process one.", zkOverride);
long zkPort = DEFAULT_ZK_PORT;
String zkHost = null;
if (zkOverride.contains(":")) {
String[] hostPort = zkOverride.split(":");
zkHost = hostPort[0];
zkPort = hostPort.length > 1 ? Long.parseLong(hostPort[1]) : DEFAULT_ZK_PORT;
} else {
zkHost = zkOverride;
}
daemonConf.put(Config.TOPOLOGY_ENABLE_MESSAGE_TIMEOUTS, true);
daemonConf.put(Config.STORM_ZOOKEEPER_SERVERS, Arrays.asList(zkHost));
daemonConf.put(Config.STORM_ZOOKEEPER_PORT, zkPort);
}
withLocalModeOverride(() -> {
String klass = args[0];
String[] newArgs = Arrays.copyOfRange(args, 1, args.length);
Class<?> c = Class.forName(klass);
Method main = c.getDeclaredMethod("main", String[].class);
LOG.info("\n\n\t\tRUNNING {} with args {}\n\n", main, Arrays.toString(newArgs));
main.invoke(null, (Object) newArgs);
return (Void) null;
}, ttl, daemonConf);
//Sometimes external things used with testing don't shut down all the way
System.exit(0);
}
/**
* Checks if Nimbuses have elected a leader.
*
* @return true if there is a leader else false.
*/
private boolean hasLeader() throws AuthorizationException, TException {
ClusterSummary summary = getNimbus().getClusterInfo();
if (summary.is_set_nimbuses()) {
for (NimbusSummary sum : summary.get_nimbuses()) {
if (sum.is_isLeader()) {
return true;
}
}
}
return false;
}
/**
* Reference to nimbus.
* @return Nimbus itself so you can interact with it directly, if needed.
*/
public Nimbus getNimbus() {
return nimbus;
}
/**
* Reference to metrics registry.
* @return The metrics registry for the local cluster.
*/
public StormMetricsRegistry getMetricRegistry() {
return metricRegistry;
}
/**
* Get daemon configuration.
* @return the base config for the daemons
*/
public Map<String, Object> getDaemonConf() {
return new HashMap<>(daemonConf);
}
@Override
public LocalTopology submitTopology(String topologyName, Map<String, Object> conf, StormTopology topology)
throws TException {
if (!Utils.isValidConf(conf)) {
throw new IllegalArgumentException("Topology conf is not json-serializable");
}
getNimbus().submitTopology(topologyName, null, JSONValue.toJSONString(conf), Utils.addVersions(topology));
ISubmitterHook hook = (ISubmitterHook) Utils.getConfiguredClass(conf, Config.STORM_TOPOLOGY_SUBMISSION_NOTIFIER_PLUGIN);
if (hook != null) {
TopologyInfo topologyInfo = Utils.getTopologyInfo(topologyName, null, conf);
try {
hook.notify(topologyInfo, conf, topology);
} catch (IllegalAccessException e) {
throw new RuntimeException(e);
}
}
return new LocalTopology(topologyName, topology);
}
@Override
public LocalTopology submitTopology(String topologyName, Map<String, Object> conf, TrackedTopology topology)
throws TException {
return submitTopology(topologyName, conf, topology.getTopology());
}
@Override
public void submitTopology(String name, String uploadedJarLocation, String jsonConf, StormTopology topology)
throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException {
try {
@SuppressWarnings("unchecked")
Map<String, Object> conf = (Map<String, Object>) JSONValue.parseWithException(jsonConf);
submitTopology(name, conf, topology);
} catch (ParseException e) {
throw new RuntimeException(e);
}
}
@Override
public LocalTopology submitTopologyWithOpts(String topologyName, Map<String, Object> conf, StormTopology topology,
SubmitOptions submitOpts)
throws TException {
if (!Utils.isValidConf(conf)) {
throw new IllegalArgumentException("Topology conf is not json-serializable");
}
getNimbus().submitTopologyWithOpts(topologyName, null, JSONValue.toJSONString(conf), Utils.addVersions(topology), submitOpts);
return new LocalTopology(topologyName, topology);
}
@Override
public LocalTopology submitTopologyWithOpts(String topologyName, Map<String, Object> conf, TrackedTopology topology,
SubmitOptions submitOpts)
throws TException {
return submitTopologyWithOpts(topologyName, conf, topology.getTopology(), submitOpts);
}
@Override
public void submitTopologyWithOpts(String name, String uploadedJarLocation, String jsonConf, StormTopology topology,
SubmitOptions options)
throws AlreadyAliveException, InvalidTopologyException, AuthorizationException, TException {
try {
@SuppressWarnings("unchecked")
Map<String, Object> conf = (Map<String, Object>) JSONValue.parseWithException(jsonConf);
submitTopologyWithOpts(name, conf, topology, options);
} catch (ParseException e) {
throw new RuntimeException(e);
}
}
@Override
public void uploadNewCredentials(String topologyName, Credentials creds) throws TException {
getNimbus().uploadNewCredentials(topologyName, creds);
}
@Override
public void killTopology(String topologyName) throws TException {
getNimbus().killTopology(topologyName);
}
@Override
public void killTopologyWithOpts(String name, KillOptions options) throws TException {
getNimbus().killTopologyWithOpts(name, options);
}
@Override
public void activate(String topologyName) throws TException {
getNimbus().activate(topologyName);
}
@Override
public void deactivate(String topologyName) throws TException {
getNimbus().deactivate(topologyName);
}
@Override
public void rebalance(String name, RebalanceOptions options) throws TException {
getNimbus().rebalance(name, options);
}
@Override
public void shutdown() {
try {
close();
} catch (Exception e) {
throw new RuntimeException(e);
}
}
@Override
public String getTopologyConf(String id) throws TException {
return getNimbus().getTopologyConf(id);
}
@Override
public StormTopology getTopology(String id) throws TException {
return getNimbus().getTopology(id);
}
@Override
public ClusterSummary getClusterInfo() throws TException {
return getNimbus().getClusterInfo();
}
@Override
public List<TopologySummary> getTopologySummaries() throws TException {
return getNimbus().getTopologySummaries();
}
@Override
public TopologySummary getTopologySummaryByName(String name) throws TException {
return getNimbus().getTopologySummaryByName(name);
}
@Override
public TopologySummary getTopologySummary(String id) throws TException {
return getNimbus().getTopologySummary(id);
}
@Override
public TopologyInfo getTopologyInfo(String id) throws TException {
return getNimbus().getTopologyInfo(id);
}
@Override
public TopologyInfo getTopologyInfoByName(String name) throws TException {
return getNimbus().getTopologyInfoByName(name);
}
@Override
public TopologyInfo getTopologyInfoWithOpts(String id, GetInfoOptions options) throws TException {
return nimbus.getTopologyInfoWithOpts(id, options);
}
@Override
public TopologyInfo getTopologyInfoByNameWithOpts(String name, GetInfoOptions options) throws TException {
return nimbus.getTopologyInfoByNameWithOpts(name, options);
}
public int getThriftServerPort() {
return thriftServer.getPort();
}
@Override
public boolean isRemoteBlobExists(String blobKey) throws AuthorizationException, TException {
throw new KeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public synchronized void close() throws Exception {
if (metricRegistry != null) {
metricRegistry.stopMetricsReporters();
}
if (nimbusOverride != null) {
nimbusOverride.close();
}
if (nimbus != null) {
nimbus.shutdown();
}
if (thriftServer != null) {
LOG.info("shutting down thrift server");
try {
thriftServer.stop();
} catch (Exception e) {
LOG.info("failed to stop thrift", e);
}
}
if (state != null) {
state.close();
}
if (getClusterState() != null) {
getClusterState().disconnect();
}
for (Supervisor s : supervisors) {
s.shutdownAllWorkers(null, ReadClusterState.THREAD_DUMP_ON_ERROR);
s.close();
}
ProcessSimulator.killAllProcesses();
if (zookeeper != null) {
LOG.info("Shutting down in process zookeeper");
zookeeper.close();
LOG.info("Done shutting down in process zookeeper");
}
for (TmpPath p : tmpDirs) {
p.close();
}
if (this.trackId != null) {
LOG.warn("Clearing tracked metrics for ID {}", this.trackId);
LocalExecutor.clearTrackId();
RegisteredGlobalState.clearState(this.trackId);
}
if (this.commonInstaller != null) {
this.commonInstaller.close();
}
if (time != null) {
time.close();
}
if (stormHomeBackup != null) {
System.setProperty(ConfigUtils.STORM_HOME, stormHomeBackup);
} else {
System.clearProperty(ConfigUtils.STORM_HOME);
}
}
/**
* Get a specific Supervisor. This is intended mostly for internal testing.
*
* @param id the id of the supervisor
*/
public synchronized Supervisor getSupervisor(String id) {
for (Supervisor s : supervisors) {
if (id.equals(s.getId())) {
return s;
}
}
return null;
}
/**
* Kill a specific supervisor. This is intended mostly for internal testing.
*
* @param id the id of the supervisor
*/
public synchronized void killSupervisor(String id) {
for (Iterator<Supervisor> it = supervisors.iterator(); it.hasNext(); ) {
Supervisor s = it.next();
if (id.equals(s.getId())) {
it.remove();
s.close();
//tmpDir will be handled separately
return;
}
}
}
/**
* Add another supervisor to the topology. This is intended mostly for internal testing.
*/
public Supervisor addSupervisor() throws Exception {
return addSupervisor(null, null, null);
}
/**
* Add another supervisor to the topology. This is intended mostly for internal testing.
*
* @param ports the number of ports/slots the supervisor should have
*/
public Supervisor addSupervisor(Number ports) throws Exception {
return addSupervisor(ports, null, null);
}
/**
* Add another supervisor to the topology. This is intended mostly for internal testing.
*
* @param ports the number of ports/slots the supervisor should have
* @param id the id of the new supervisor, so you can find it later.
*/
public Supervisor addSupervisor(Number ports, String id) throws Exception {
return addSupervisor(ports, null, id);
}
/**
* Add another supervisor to the topology. This is intended mostly for internal testing.
*
* @param ports the number of ports/slots the supervisor should have
* @param conf any config values that should be added/over written in the daemon conf of the cluster.
* @param id the id of the new supervisor, so you can find it later.
*/
public synchronized Supervisor addSupervisor(Number ports, Map<String, Object> conf, String id) throws Exception {
if (ports == null) {
ports = 2;
}
TmpPath tmpDir = new TmpPath();
tmpDirs.add(tmpDir);
List<Integer> portNumbers = new ArrayList<>(ports.intValue());
for (int i = 0; i < ports.intValue(); i++) {
portNumbers.add(portCounter++);
}
Map<String, Object> superConf = new HashMap<>(daemonConf);
if (conf != null) {
superConf.putAll(conf);
}
superConf.put(Config.STORM_LOCAL_DIR, tmpDir.getPath());
superConf.put(DaemonConfig.SUPERVISOR_SLOTS_PORTS, portNumbers);
if (!Time.isSimulating()) {
//Monitor for assignment changes as often as possible, so e.g. shutdown happens as fast as possible.
superConf.put(DaemonConfig.SUPERVISOR_MONITOR_FREQUENCY_SECS, 1);
}
final String superId = id == null ? Utils.uuid() : id;
ISupervisor isuper = new StandaloneSupervisor() {
@Override
public String generateSupervisorId() {
return superId;
}
};
if (!ConfigUtils.isLocalMode(superConf)) {
throw new IllegalArgumentException("Cannot start server in distrubuted mode!");
}
Supervisor s = new Supervisor(superConf, sharedContext, isuper, metricRegistry);
s.launch();
s.setLocalNimbus(this.nimbus);
this.nimbus.addSupervisor(s);
supervisors.add(s);
return s;
}
private boolean areAllSupervisorsWaiting() {
boolean ret = true;
for (Supervisor s : supervisors) {
ret = ret && s.isWaiting();
}
return ret;
}
/**
* Wait for the cluster to be idle. This is intended to be used with Simulated time and is for internal testing.
* Note that this does not wait for spout or bolt executors to be idle.
*
* @throws InterruptedException if interrupted while waiting.
* @throws AssertionError if the cluster did not come to an idle point with a timeout.
*/
public void waitForIdle() throws InterruptedException {
waitForIdle(Testing.TEST_TIMEOUT_MS);
}
/**
* Wait for the cluster to be idle. This is intended to be used with Simulated time and is for internal testing.
* Note that this does not wait for spout or bolt executors to be idle.
*
* @param timeoutMs the number of ms to wait before throwing an error.
* @throws InterruptedException if interrupted while waiting.
* @throws AssertionError if the cluster did not come to an idle point with a timeout.
*/
public void waitForIdle(long timeoutMs) throws InterruptedException {
Random rand = ThreadLocalRandom.current();
//wait until all workers, supervisors, and nimbus is waiting
final long endTime = System.currentTimeMillis() + timeoutMs;
while (!(nimbus.isWaiting()
&& areAllSupervisorsWaiting()
&& areAllWorkersWaiting())) {
if (System.currentTimeMillis() >= endTime) {
LOG.info("Cluster was not idle in {} ms", timeoutMs);
LOG.info(Utils.threadDump());
throw new AssertionError("Test timed out (" + timeoutMs + "ms) cluster not idle");
}
Thread.sleep(rand.nextInt(20));
}
}
@Override
public void advanceClusterTime(int secs) throws InterruptedException {
advanceClusterTime(secs, 1);
}
@Override
public void advanceClusterTime(int secs, int incSecs) throws InterruptedException {
waitForIdle();
for (int amountLeft = secs; amountLeft > 0; amountLeft -= incSecs) {
int diff = Math.min(incSecs, amountLeft);
Time.advanceTimeSecs(diff);
waitForIdle();
}
}
@Override
public IStormClusterState getClusterState() {
return clusterState;
}
@Override
public String getTrackedId() {
return trackId;
}
//Nimbus Compatibility
@Override
public void setLogConfig(String name, LogConfig config) throws TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public LogConfig getLogConfig(String name) throws TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public void debug(String name, String component, boolean enable, double samplingPercentage)
throws NotAliveException, AuthorizationException, TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public void setWorkerProfiler(String id, ProfileRequest profileRequest) throws TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public List<ProfileRequest> getComponentPendingProfileActions(String id, String componentId, ProfileAction action)
throws TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public String beginCreateBlob(String key, SettableBlobMeta meta)
throws AuthorizationException, KeyAlreadyExistsException, TException {
throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public String beginUpdateBlob(String key) throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void uploadBlobChunk(String session, ByteBuffer chunk) throws AuthorizationException, TException {
throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void finishBlobUpload(String session) throws AuthorizationException, TException {
throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void cancelBlobUpload(String session) throws AuthorizationException, TException {
throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public ReadableBlobMeta getBlobMeta(String key) throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void setBlobMeta(String key, SettableBlobMeta meta)
throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public BeginDownloadResult beginBlobDownload(String key)
throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public ByteBuffer downloadBlobChunk(String session) throws AuthorizationException, TException {
throw new RuntimeException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void deleteBlob(String key) throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public ListBlobsResult listBlobs(String session) throws TException {
//Blobs are not supported in local mode. Return nothing
ListBlobsResult ret = new ListBlobsResult();
ret.set_keys(new ArrayList<>());
return ret;
}
@Override
public int getBlobReplication(String key) throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public int updateBlobReplication(String key, int replication)
throws AuthorizationException, KeyNotFoundException, TException {
throw new WrappedKeyNotFoundException("BLOBS NOT SUPPORTED IN LOCAL MODE");
}
@Override
public void createStateInZookeeper(String key) throws TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public String beginFileUpload() throws AuthorizationException, TException {
//Just ignore these for now. We are going to throw it away anyways
return Utils.uuid();
}
@Override
public void uploadChunk(String location, ByteBuffer chunk) throws AuthorizationException, TException {
//Just throw it away in local mode
}
@Override
public void finishFileUpload(String location) throws AuthorizationException, TException {
//Just throw it away in local mode
}
@Override
public ByteBuffer downloadChunk(String id) throws AuthorizationException, TException {
throw new WrappedAuthorizationException("FILE DOWNLOAD NOT SUPPORTED IN LOCAL MODE");
}
@Override
public String getNimbusConf() throws AuthorizationException, TException {
// TODO Auto-generated method stub
throw new RuntimeException("NOT IMPLEMENTED YET");
}
@Override
public NimbusSummary getLeader() throws AuthorizationException, TException {
return nimbus.getLeader();
}
@Override
public boolean isTopologyNameAllowed(String name) throws AuthorizationException, TException {