[ 
https://issues.apache.org/jira/browse/STORM-1976?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15381220#comment-15381220
 ] 

Jungtaek Lim commented on STORM-1976:
-------------------------------------

cleanup-corrupt-topologies! is a legacy code written at very first version of 
Nimbus so it doesn't consider H/A of nimbus. It assumes that all topology codes 
should be available at Nimbus startup, but in many cases it doesn't.

Let me show some cases:

1. When we launch nimbus process from new node. The node never has topology 
code in local dir since this is the first time Nimbus process is launched. So 
while startup of Nimbus, Nimbus calls cleanup-corrupt-topologies!, and all 
topologies goes away from ZK.
(This is easy to reproduce and occurred during valid use case so IMO this is 
'Blocker'.)

2. At a glance, Local BlobStore does full replications to all nimbuses which is 
same as before.
But when submitting topology, leader Nimbus starts to make assignment when 
topology.min.replication.count is achieved or it waits longer than 
topology.max.replication.wait.time.sec. So there's still a gap between topology 
assignment and full replications. If one or more follower nimbuses are crashed 
before replicating topology code at that time and restarted, 
cleanup-corrupt-topologies! removes that topology from ZK.

Btw, I didn't look deeply at the source code, but full replication seems to be 
started immediately when topology is submitted. I set 
nimbus.code.sync.freq.secs to 600 to give a delay, but full replication is 
started while topology code is uploading to leader nimbus. If then this gap may 
be small in LAN.

N1 (leader)
{code}
2016-07-17 07:51:39.128 o.a.s.d.nimbus [INFO] Received topology submission for 
production-topology2 with conf {"topology.max.task.parallelism" nil, 
"topology.submitter.principal" "", "topology.acker.executors" nil, 
"topology.eventlogger.executors" 0, "topology.debug" true, 
"storm.zookeeper.superACL" nil, "topology.users" (), "topology.submitter.user" 
"storm", "topology.kryo.register" nil, "topology.kryo.decorators" (), 
"storm.id" "production-topology2-1-1468741899", "topology.name" 
"production-topology2"}
...
2016-07-17 07:51:39.301 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormjar.jar/<node1>:6627-1
2016-07-17 07:51:39.346 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormconf.ser/<node1>:6627-1
2016-07-17 07:51:39.389 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormcode.ser/<node1>:6627-1
...
2016-07-17 07:51:39.400 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormjar.jar with id 
5b4103f0-d4cf-4ec9-9191-e2a0e06ad97c
...
2016-07-17 07:51:39.403 o.a.s.d.nimbus [INFO] desired replication count 1 
achieved, current-replication-count for conf key = 1, current-replication-count 
for code key = 1, current-replication-count for jar key = 1
...
2016-07-17 07:51:39.415 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormjar.jar with id 
854cabdd-080f-4c6f-91fe-81950490e67d
2016-07-17 07:51:39.436 o.a.s.d.nimbus [INFO] Activating production-topology2: 
production-topology2-1-1468741899
2016-07-17 07:51:40.578 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormjar.jar with id 
50809e38-61b2-4533-9942-0aa28cb327c0
2016-07-17 07:51:41.058 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormjar.jar with id 
83949b7a-ac55-484a-a4f2-5361a90ee518
2016-07-17 07:51:41.836 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormcode.ser with id 
c6c80b3c-99da-4557-9af5-7f49d6eaccc9
2016-07-17 07:51:41.906 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormconf.ser with id 
051d7ced-7f8f-4170-88a2-86a01fe49e28
2016-07-17 07:51:42.703 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormcode.ser with id 
dd7bf0ab-6b06-46e5-9b44-f4d956812a0b
2016-07-17 07:51:42.768 o.a.s.d.nimbus [INFO] Created download session for 
production-topology2-1-1468741899-stormconf.ser with id 
cea017dc-c574-4891-a00a-3febd6bd001a
{code}

N2 (non-leader)
{code}
2016-07-17 07:51:40.526 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormjar.jar/<node2>:6627-0
2016-07-17 07:51:41.806 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormjar.jar/<node2>:6627-1
2016-07-17 07:51:41.891 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormcode.ser/<node2>:6627-0
2016-07-17 07:51:41.956 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormconf.ser/<node2>:6627-0
{code}

N3 (non-leader)
{code}
2016-07-17 07:51:40.999 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormjar.jar/<node3>:6627-0
2016-07-17 07:51:42.665 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormjar.jar/<node3>:6627-1
2016-07-17 07:51:42.735 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormcode.ser/<node3>:6627-0
2016-07-17 07:51:42.795 o.a.s.cluster [INFO] 
setup-path/blobstore/production-topology2-1-1468741899-stormconf.ser/<node3>:6627-0
...
{code}

> Storm Nimbus H/A has issue on cleaning corrupted topologies
> -----------------------------------------------------------
>
>                 Key: STORM-1976
>                 URL: https://issues.apache.org/jira/browse/STORM-1976
>             Project: Apache Storm
>          Issue Type: Bug
>          Components: storm-core
>    Affects Versions: 1.0.0, 1.0.1
>            Reporter: Raghav Kumar Gautam
>            Assignee: Jungtaek Lim
>            Priority: Blocker
>
> In the following scenario storm-ha runs into issues:
> 1. Kill a non-leader nimbus
> 2. Submit a topology
> 3. Bring up the non-leader nimbus
> After step-3 expectation is that the non-leader nimbus will download topology 
> jar. Instead it cleans up the topology.
> {code}
> 2016-07-12 07:11:09.511 o.a.s.c.zookeeper-state-factory [WARN] Received event 
> ::none: with disconnected Reader Zookeeper.
> 2016-07-12 07:11:09.587 o.a.s.zookeeper [INFO] Queued up for leader lock.
> 2016-07-12 07:11:09.608 o.a.s.d.nimbus [INFO] Corrupt topology 
> JoinedNonLeaderNimbusTriesToDownloadTopologyCode-2-1468307239 has state on 
> zookeeper but doesn't have a local dir on Nimbus. Cleaning up...
> 2016-07-12 07:11:09.932 o.a.h.m.s.s.StormTimelineMetricsReporter [INFO] 
> Preparing Storm Metrics Reporter
> 2016-07-12 07:11:09.946 o.a.s.d.m.MetricsUtils [INFO] Using statistics 
> reporter 
> plugin:org.apache.storm.daemon.metrics.reporters.JmxPreparableReporter
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to