As far as I know, most of Storm will run on Java 11, except for some
specific modules that have dependencies incompatible with Java 11.
See https://issues.apache.org/jira/browse/STORM-2798
When I left off working on that ticket, we were blocked by the dependencies
(e.g. Cassandra) not supporting
s. I then goit the same UI error, and
> checked my logs.
>
> On Mon, Nov 18, 2019 at 11:51 AM Stig Rohde Døssing <
> stigdoess...@gmail.com> wrote:
>
>> Please elaborate on what you mean by "starting everything up again", e.g.
>> by listing the commands
Please elaborate on what you mean by "starting everything up again", e.g.
by listing the commands you're running. The error you're getting is simply
saying that the topology "test-14-1573863430" is no longer running, which
could be for all kinds of reasons.
Den man. 18. nov. 2019 kl. 20.31 skrev B
No, not for a while now. See
https://github.com/apache/storm/blob/v2.1.0/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/FirstPollOffsetStrategy.java#L30.
The spout only ignores the committed offset once you redeploy the topology.
As your worker is just crashing, the topology
Whoops, I think the user mailing list fell off the reply list. Sorry,
didn't mean to mail you directly.
I haven't heard of this before, but people may have encountered it without
mentioning it. I am not aware of a workaround. You're right that it would
be good to get this fixed. https://issues.apa
Hi Zac,
Storm is supposed to only put the contents of lib-worker on the classpath
of your topology (and a couple of other directories like extlib), which
does not contain Guava. When you are checking whether your jar contains
Guava, are you doing this by extracting the topology jar, or just by
loo
Sorry, maybe what I posted was a little confusing. The linked file is the
wrapper Storm has around the real Disruptor queue. You can open the Storm
project and take a look at that class and where it is used, and get answers
to your questions.
Den fre. 6. sep. 2019 kl. 17.01 skrev Nithin Uppalapati
I'm not aware of any documentation on it. The Disruptor implementation was
replaced as of Storm 2.0.0 with JCTools queues. You can find documentation
on that here https://issues.apache.org/jira/browse/STORM-2306
If you want to read the code around the Disruptor queue,
https://github.com/apache/sto
Take a look at
https://github.com/apache/storm/blob/7e98f2916a2d6e6087fa09ad8bb7dca45ba3f436/storm-server/src/main/java/org/apache/storm/zookeeper/LeaderElectorImp.java,
this is the Nimbus leader election code.
Den tor. 5. sep. 2019 kl. 20.32 skrev Zainal Arifin (BLOOMBERG/ 731 LEX) <
zari...@bloo
It doesn't support Java 11. Storm 2.0.0 has had work done to support Java
11, but the Cassandra and Hadoop parts are blocked by those projects
lacking support so far. I think the rest of Storm 2.0.0 is likely to work.
I don't believe you need to follow any special steps to migrate to Java 11.
Den
Regarding cloud providers offering hosted Storm clusters, that's really up
to the cloud providers to figure out. I know at least Azure used to provide
hosted Storm clusters under the HDInsight label. They may still do.
Den tor. 29. aug. 2019 kl. 20.28 skrev Ethan Li :
> Sorry I don’t have knowled
se if we don't have a resources jar or a classpath url, we
> aren't really extracting anything from a jar.
>
> Another thing, what's your opinion on how I should test this?
> *LocallyCachedTopologyBlob* do not have unit tests.
>
> Diogo.
>
> On Tue, Aug 27, 20
Hi Diogo,
Thanks for your thorough explanation. I think you are right, and this is a
bug. We'd be happy to see a PR to fix this.
I think a decent way to handle this could be adding an extra else clause to
https://github.com/apache/storm/blob/2ba95bbd1c911d4fc6363b1c4b9c4c6d86ac9aae/storm-server/s
As far as I know you can't use Storm without Zookeeper. Pacemaker is just
for receiving heartbeats, Storm still needs to store other information in
Zookeeper, e.g. which topologies are deployed.
Den fre. 23. aug. 2019 kl. 10.57 skrev Igor A. :
> Ethan,
>
> >Pacemaker is an additional daemon that
7;re
reprocessing old messages.
> that explains why only first few were getting flushed in time window
Great, happy you found the cause.
Den tor. 1. aug. 2019 kl. 14.04 skrev Sandeep Singh <
tosandeepsi...@gmail.com>:
> Thank you very much for the response. Please see my comments inlin
Regarding why you need the 5th tuple, it is happening because you are using
timestamp fields. The windowing code will receive the first 4 tuples and
add them to the same window. Until it receives the 5th tuple, there is no
way to tell whether the window is "done", as we might receive more tuples
th
Here are a couple of links to the source that might help you.
The system property is coming from
https://github.com/apache/storm/blob/eaed3cbf3610bbefb9466f2c40db1e0e2a20ca9c/storm-server/src/main/java/org/apache/storm/daemon/supervisor/BasicContainer.java#L574
It is being calculated by
https://g
Sorry about the formatting on these, it looked fine in Gmail.
Den ons. 24. jul. 2019 kl. 09.24 skrev Stig Rohde Døssing :
> [CVEID]:CVE-2018-11779[PRODUCT]:Apache Storm[VERSION]:Apache Storm 1.1.0 to
> 1.2.2[PROBLEMTYPE]:CWE-502: Deserialization of Untrusted Data[DESCRIPTION]:In
> Apa
[CVEID]:CVE-2018-11779[PRODUCT]:Apache Storm[VERSION]:Apache Storm
1.1.0 to 1.2.2[PROBLEMTYPE]:CWE-502: Deserialization of Untrusted
Data[DESCRIPTION]:In Apache Storm versions 1.1.0 to 1.2.2,
when the user is using the storm-kafka-client or
storm-kafka modules,
it is pos
[CVEID]:CVE-2018-1320[PRODUCT]:Apache Storm[VERSION]:Apache Storm
0.9.1-incubating to 1.2.2[PROBLEMTYPE]:CWE-20: Input
Validation[DESCRIPTION]:Apache Storm versions 0.9.1-incubating to
1.2.2
use Thrift library versions vulnerable to CVE-2018-1320.
Mitigation: Upgrade to Apache Storm
versions 0.9.1-incubating to 1.2.2, it
is possible to read files off the
host's file system that were not intended to be
accessible via these endpoints.
Mitigation: Upgrade to Apache Storm 1.2.3 or later.
Credit: Stig Rohde Døssing for discovery and fix
blind alley, but I am really
>> confused is 1.2.3 or 2.0.0 suggested for a new production? Comparing with
>> apache kafka, it’s much easier to choose because it has only one version
>> line, see https://kafka.apache.org/downloads#2.3.0 which means it’s easy
>> to choose a su
Hi,
Most of these version lines aren't active. The currently maintained lines
are 1.2.x and 2.0.0.
Regarding the difference between 1.x and 2.x, please see
https://storm.apache.org/2019/05/30/storm200-released.html. You can also go
look at the release notes here
https://www.apache.org/dist/storm/
The Apache Storm community is pleased to announce the release of Apache
Storm version 1.2.3.
Storm is a distributed, fault-tolerant, and high-performance realtime
computation system that provides strong guarantees on the processing of
data. You can read more about Storm on the project website:
h
. 17.23 skrev Stig Rohde Døssing <
stigdoess...@gmail.com>:
> This is just a loose idea but I think you can solve all of these very
> easily without the windowing bolts if you add a data store.
>
> 1: Have a bolt that produces an alarm immediately if it sees a tuple where
> the
This is just a loose idea but I think you can solve all of these very
easily without the windowing bolts if you add a data store.
1: Have a bolt that produces an alarm immediately if it sees a tuple where
the condition is met.
2: When your bolt sees the condition is true, it writes into the data s
luesSerializer
>
>
> What do you think about creating wrapper around Kryo*Serializers and
> Kryo*Deserializers with ThreadLocal to get rid of possible future kryo
> threading issues?
>
>
> Best regards,
>
> Michal
>
>
> Wiadomość oryginalna -
I don't think we've changed Kryo stuff much since 1.0.0. Last version
upgrade was https://github.com/apache/storm/pull/1255.
Maybe it could be a threading issue in the Netty messaging implementation
in
https://github.com/apache/storm/tree/master/storm-client/src/jvm/org/apache/storm/messaging/nett
a.common.serialization.StringSerializer")
> // Setup deserialization to fields : (String key, String json
> value) => (String Key, Unpacked object from json)
> // .setRecordTranslator(new
> ByTopicRecordTranslator<>((TargetInterface Serializable)
I don't see anything wrong with the code you posted. Could you post the
full AntConfigurableTopology code? It's hard to tell from that snippet what
your topology setup looks like.
Den tor. 6. jun. 2019 kl. 12.33 skrev aurelien violette <
aurelie...@webgroup-limited.com>:
> Hello,
>
> I was sucess
Hi Nicole,
No, this is the right place.
Your question is a little unclear to me since it is hard to tell whether
you're talking about the supervisor Storm daemon, or the supervisord tool
for running services. I'm going to assume you are asking why storm-nimbus
is in the BACKOFF state, and whether
The "execute" method will never be called concurrently on the same bolt
instance. In your topology, you will have some number of bolt C copies
running (the task count for bolt C). Each copy will only receive one tuple
in "execute" at a time. Any other tuples received while "execute" is
running will
No, I think you are right. It doesn't look like the 2.0.0 artifacts are
present in Maven yet.
CC'ing to the dev list.
Den man. 3. jun. 2019 kl. 17.30 skrev Re'em Bensimhon :
> Hey everybody
>
> I'm trying to test out the new Storm release (very exciting :-) but I'm
> not managing to find the v2.
There was a recent PR to partially address this at
https://github.com/apache/storm/pull/3016, it probably doesn't fix your
exact case, but fixing your case would probably require doing something
similar.
The reason this is broken is that storm-kafka-monitor is a workaround to us
not having generic
Thanks for running the release Taylor.
Most likely users will want the storm-client jar, rather than the
storm-core jar.
Den tor. 30. maj 2019 kl. 21.56 skrev P. Taylor Goetz :
> The Apache Storm community is pleased to announce the release of Apache
> Storm version 2.0.0.
>
> Storm is a distrib
Hi Pavithra,
The wiring looks fine to me. Most likely the issue is in the bolt code.
Could you post the spout_1/bolt_1/bolt_2 code as well (at least the bit
related to emitting)?
Den ons. 29. maj 2019 kl. 21.11 skrev pavidhanaj :
>
> Hi,
>
> I need to use the bolt which has to consume the data e
1 - m1 is not sent to your bolts unless you manually include it in the
tuple. If you do, you can do whatever you want with it in the bolts. Storm
doesn't look at m1 at all, beyond receiving it from your spout
implementation, and passing it back to the spout on fail or ack. Instead,
Storm generates
Hi Pradeep,
Before you start implementing your own, you may want to look at
https://github.com/Parsely/streamparse. Those guys can probably answer your
questions as well.
Den fre. 17. maj 2019 kl. 22.04 skrev Pradeep Badiger <
pradeepbadi...@fico.com>:
> Hi,
>
>
>
> I was looking at multilang fe
I may be completely off base with this, but I think metrics reporters are
configured in the cluster configuration, and not on a per-topology basis.
Note how the documentation you linked sets the config in storm.yaml, rather
than in the topology configuration.
Could you try putting the metrics conf
>From your image, you have 3 spout tasks, but only 2 Kafka partitions. This
should mean that one of your executors isn't doing anything. I think this
is why you're seeing this log line where offsetManagers is empty.
2019-05-10T00:13:50.176+05:30 KafkaSpout [TRACE] No offsets to commit.
KafkaSpout{
I would just try it. If it's still an issue after the upgrade, we can look
at it.
Den fre. 10. maj 2019 kl. 22.29 skrev Mitchell Rathbun (BLOOMBERG/ 731 LEX)
:
> We frequently get ClosedSelectorException from our KafkaSpout
> implementation on topology shutdown. We are using storm-kafka-client
>
Ah, sorry, got off on the wrong track due to the linked issue, which is
talking about worker JVM exit codes.
Den man. 6. maj 2019 kl. 19.34 skrev Mitchell Rathbun (BLOOMBERG/ 731 LEX) <
mrathb...@bloomberg.net>:
> Sorry if my initial question was misleading. The "Storm kill" command
> returned 14
I would assume that what actually happened is that most of your workers
don't manage to finish shutting down the worker gracefully, and so exit
with code 20 due to the 1 second time limit imposed by the shutdown hook.
One of your workers happened to run the entire shutdown sequence within the
1 sec
No, I think that's what you have to do. Killing the topology will have the
effect you want. There isn't currently a
"kill-topologies-running-on-this-supervisor-and-then-kill-supervisor"
command.
Den tor. 2. maj 2019 kl. 19.04 skrev Mitchell Rathbun (BLOOMBERG/ 731 LEX) <
mrathb...@bloomberg.net>:
I'm a little confused by what you're saying. When you say all the
topologies were killed, are you talking about the worker JVMs, or are you
talking about the topologies being removed from the cluster?
Could you explain exactly what you want to have happen?
Den tor. 2. maj 2019 kl. 18.13 skrev Mit
g the output to a file through a '> output.file' at the
> end of the command line; I assume this was causing some kind of problem
> with starting or managing the local tasks. I had done the first runs
> without the pipe, but apparently missed the output. I appreciate your help,
Hi David,
I wouldn't worry about that log. It's not an error as far as I can tell.
The log is at INFO level as well.
Storm-starter works for me. I get the same logs as you, but after "4987
[main] INFO o.a.s.d.nimbus - Activating test: test-1-1556744624", I get a
bunch more log from Storm running
I don't believe the supervisor needs to be running to run kill_workers. You
can kill the supervisor first, then run kill_workers to get rid of the
workers, do the restart and reboot the supervisor.
Den ons. 1. maj 2019 kl. 22.09 skrev Mitchell Rathbun (BLOOMBERG/ 731 LEX) <
mrathb...@bloomberg.net
I believe kill_workers is for cleaning up workers if e.g. you want to shut
down a supervisor node, or if you have an unstable machine you want to take
out of the cluster. The command was introduced because simply killing the
supervisor process would leave the workers alive.
If you want to kill the
; **@***.**.**.110~>jps -l
>
> 33536 org.apache.storm.daemon.supervisor.Supervisor
>
>
>
> Thanks & Regards,
> Julian.
>
> --
>
>
> On Mon, 15 Apr 2019 15:00:56 +0530 *Stig Rohde Døssing
> >* wrote
>
>
It would be good to know which commands those Java processes are running.
Could you try getting the output of "jps -lvm " for the Java processes
that have the ports open?
Den man. 15. apr. 2019 kl. 07.21 skrev Thomas Julian :
> Hi Roshan,
>
> Thank you for your reply. What I understand from your
I don't think you can get the worker logs in the console directly. The
issue is that your container will be running both the supervisor, and a
number of workers. The workers are started by the supervisor, and aren't
attached to the console you use to start the supervisor. Even if you
configure a co
You could probably also just tail the worker log files into the console you
are using to run the supervisor, but it'll most likely get messy to read.
Den tor. 4. apr. 2019 kl. 20.04 skrev Stig Rohde Døssing <
stigdoess...@gmail.com>:
> I don't think you can get the worker
I believe this is handled by the Logviewer daemon. If you run "storm
logviewer" on your worker machines, the Logviewer daemon will periodically
delete old topology directories.
Take a look at
https://github.com/apache/storm/blob/9680928a0c6e26a8a2d3e5ceb9a632579546353d/storm-webapp/src/main/java/o
own reporter were not visible.
>
> On Thu, Apr 4, 2019 at 12:39 PM Stig Rohde Døssing
> wrote:
>
>> There might be an issue with the API for StateFactory here. You need a
>> TopologyContext to use the new metrics API, but makeState doesn't take one.
>> Others can c
19 kl. 08.58 skrev Stig Rohde Døssing <
stigdoess...@gmail.com>:
> As far as I can tell, the JMX reporting is only hooked up to the metrics
> v2 API. You're using metrics v1. Could you try to register your metric with
> the new metrics system? You can find documentation at
> https
As far as I can tell, the JMX reporting is only hooked up to the metrics v2
API. You're using metrics v1. Could you try to register your metric with
the new metrics system? You can find documentation at
https://storm.apache.org/releases/2.0.0-SNAPSHOT/metrics_v2.html.
Den ons. 3. apr. 2019 kl. 15.
Supplementing the other great answers here, Storm checks max spout pending
before it invokes nextTuple on the spout. The spout executor calls
nextTuple in a loop. If the spout task has more pending tuples (i.e.
emitted but not acked or failed or timed out) than max spout pending, the
executor won't
You might find
https://storm.apache.org/releases/2.0.0-SNAPSHOT/Understanding-the-parallelism-of-a-Storm-topology.html
informational.
The executors invoke nextTuple. As you have configured your topology, you
will have 3 tasks (unless you set it explicitly, you get the same number of
tasks as execu
Storm does this automatically. Take a look at
https://stackoverflow.com/a/51384243/8845188.
The default is a flat wait that just sleeps if nothing is emitted when
nextTuple is called.
If you want to implement your own wait strategy (e.g. a progressive one),
you can implement
https://github.com/ap
Why are you trying to set up your metrics from your scheduler class? The
scheduler runs as part of Nimbus, presumably you would want to set up
metrics in your worker JVMs. Why not do the metrics setup in your bolts'
prepare methods, rather than the scheduler, as in this example
https://github.com/
I think it's likely 1.2.3 and 2.0.0 will be released at nearly the same
time.
Den tir. 12. mar. 2019 kl. 08.35 skrev Steven James <
steven.ja...@eumetsat.int>:
> Thanks for the response Stig is this 1.2.3 RC?
>
>
>
> Cheers,
>
>
>
> Steve.
>
>
>
Hi Steve,
I would hope for a new RC for both within the next week or two, assuming no
other blockers pop up. If we find no major issues with the RC, release will
follow a few days after the vote.
Den man. 11. mar. 2019 kl. 15.13 skrev Steven James <
steven.ja...@eumetsat.int>:
> Good afternoon,
Also some tuples may fail during topology startup due to timeout, if the
workers happen to not start at roughly the same time.
Den tor. 7. mar. 2019 kl. 07.24 skrev Ravi Sharma :
> Hi Kai,
> Seems like tuple timeout errors(no failed tuples in bolts but spout
> reports failure), Whats the value fo
The workers are dying for some reason. Check the worker log, it will likely
say why the worker died. The worker may also have died due to an out of
memory exception.
Den tir. 5. mar. 2019 kl. 07.54 skrev Jayant Sharma <
sharmajayan...@gmail.com>:
> Hi,
>
> We are getting errors such as
>
> Worke
1. No, the worker heartbeats to Nimbus in a thread independent of the
executors.
https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/daemon/worker/Worker.java#L187
2. This will most likely not work. As you can see at
https://github.com/apache/storm/blob/901f4aa81ccb95
Hi,
Yes, these versions will likely be released soon. The recommended version
is (IMO) the latest release, since that will be the version that has had
the most bugfixes put into it.
Den søn. 17. feb. 2019 kl. 12.07 skrev Lieberman, Yuval <
yuval.lieber...@2bprecisehealth.com>:
> Hi,
>
>
>
> I am
I'm not sure if it's the only issue, but you might be affected by
https://issues.apache.org/jira/browse/STORM-3102.
Den man. 7. jan. 2019 kl. 09.37 skrev Bernard Quizon <
bernard.qui...@stellarloyalty.com>:
> Hi Roshan,
>
> Just to narrow it down .. can u get rid of the bolt and see what numbers
nt/src/main/java/org/apache/storm/kafka/spout/KafkaSpout.java#L465-L473.>,
>> putting that back in 1.x-branch, I did not got this exception so far, I
>> will run it few more times to be sure and let you know.
>>
>>
>>
>>
>> Best Regards
>>
>>
l this
> be released?
>
> Thanks for your help :)
>
>
>
>
>
> Best Regards
>
> Saurabh Kumar Mimani
>
>
>
>
> On Mon, Dec 10, 2018 at 1:12 AM Stig Rohde Døssing
> wrote:
>
>> I'm assuming you applied the fix on top of 1.2.1 or somethi
59be>.
>
>
>
> Best Regards
>
> Saurabh Kumar Mimani
>
>
>
>
> On Sun, Dec 9, 2018 at 3:19 AM Stig Rohde Døssing
> wrote:
>
>> I believe I have a fix, your logs were helpful. Please try out the
>> changes in https://github.com/apache/storm/pull/2923/fi
mimani/56dd31db34e4356b25c796d78261f7b8> are
> logs of Kafka Spout, when I was able to reproduce it in cluster mode with
> my topology, in case these helps.
>
>
>
> Best Regards
>
> Saurabh Kumar Mimani
>
>
>
>
> On Wed, Dec 5, 2018 at 11:33 PM Stig Rohde Døssi
It looks like that module only exists in master (2.0.0) and not any of the
released 1.x lines. You can either wait for 2.0.0 to release, or check out
the source from https://github.com/apache/storm and build it yourself. See
https://github.com/apache/storm/blob/master/DEVELOPER.md#building for how
Hi Saurabh,
The tuple emitted by the spout will only be acked once all branches of the
tuple tree have been acked, i.e. all 100 tuples are acked.
The error you're seeing was added as part of
https://issues.apache.org/jira/browse/STORM-2666 to try to avoid having
that bug pop up again. Could you t
Your first link points to storm-kafka, while your second link points to
storm-kafka-client. Storm-kafka-client is intended as a replacement for
storm-kafka, but the two codebases are unrelated. The metric is missing
because no one has implemented metrics for the new Trident spout yet.
There's an im
See
https://github.com/apache/storm/blob/21bb1388414d373572779289edc785c7e5aa52aa/storm-client/src/jvm/org/apache/storm/daemon/GrouperFactory.java#L174
and
https://github.com/apache/storm/blob/21bb1388414d373572779289edc785c7e5aa52aa/storm-client/src/jvm/org/apache/storm/utils/TupleUtils.java#L38
.
Also you should be aware of
https://issues.apache.org/jira/browse/STORM-3280 if you intend to use
Trident based windowing.
Den fre. 23. nov. 2018 kl. 12.10 skrev Stig Rohde Døssing <
stigdoess...@gmail.com>:
> I don't believe you can. The Trident windowing cod
I don't believe you can. The Trident windowing code seems to use
System.currentTimeMillis without allowing you to set a timestamp extractor.
If you want to add this functionality, you should look at
https://github.com/apache/storm/blob/21bb1388414d373572779289edc785c7e5aa52aa/storm-client/src/jvm/o
Are you sure topology.worker.childopts are actually being applied to the
logwriter JVM? Keep in mind that the entire worker launch command is being
passed through as command line parameters to the logwriter main method, so
topology.worker.childopts will appear in the command line for the
logwriter,
> nowhere to be found in the fields.
>
>
> On Mon, Sep 17, 2018 at 10:45 PM Stig Rohde Døssing <
> stigdoess...@gmail.com> wrote:
>
>> The storm-kafka-client spout is a complete rewrite. The new spout stores
>> offsets in Kafka instead of Zookeeper, so you don
will try to submit PRs for current storm and 1.1.x and 1.2.x.
>
> I see 1.1.x-branch on git. Which one is for 1.2.x. Is it 1.x-branch?
>
>
> Best regards,
>
> Michal
>
> W dniu 20.09.2018 o 16:30, Stig Rohde Døssing pisze:
>
> I don't think there are any open iss
I don't think there are any open issues/PRs to do it, but I don't see why
we couldn't upgrade. Feel free to raise an issue at
https://issues.apache.org/jira, and if you want to contribute the patch,
please submit a PR to https://github.com/apache/storm/pulls. You can find a
guide on how to develop
The storm-kafka-client spout is a complete rewrite. The new spout stores
offsets in Kafka instead of Zookeeper, so you don't need to set any
Zookeeper configuration. You will need to migrate your committed offsets
using
https://github.com/apache/storm/tree/master/external/storm-kafka-migration
thou
ant to be sure it has no outstanding impact.
>
> 2018년 8월 18일 (토) 오후 9:37, Stig Rohde Døssing 님이
> 작성:
>
>> Hi,
>>
>> The metrics v2 system contains a Ganglia reporter based on Dropwizard
>> Metrics' metrics-ganglia module. One of the dependencies of this modul
Just want to add a quick note as well that you should use the latest
storm-kafka-client. The intent is that you add a dependency both to
storm-kafka-client, and org.apache.kafka:kafka-clients in your project.
This will let you control the Kafka client version. See
https://github.com/apache/storm/bl
Happy to hear it :) If you run into trouble, we're tracking
incompatibilities here https://issues.apache.org/jira/browse/STORM-2798.
You might hit issues with HDFS, I don't think Hadoop supports Java 9+ yet
https://issues.apache.org/jira/browse/HADOOP-11123.
2018-08-16 16:34 GMT+02:00 Alexandre Ve
5:32 GMT+02:00 Jungtaek Lim :
>
>> IMHO I'd take this as a voice for release 1.2.3. We have addressed 21
>> issues so far for Storm 1.2.3, so worth to call it for new maintenance
>> release.
>>
>> https://issues.apache.org/jira/projects/STORM/versions/123
Before you start turning other knobs, you should be aware of
https://issues.apache.org/jira/browse/STORM-3102, which is a performance
penalty when using Kafka 0.11 or higher. It should be fixed in the latest
code, but hasn't been released yet.
2018-08-07 16:53 GMT+02:00 Nithin Uppalapati (BLOOMBER
Also forgot to mention, in Trident the offsetCommitPeriodMs and
maxUncommittedOffsets don't do anything. Trident spouts "commit" into
Storm's Zookeeper rather than Kafka, so the mentioned settings don't do
anything.
2018-07-30 19:58 GMT+02:00 Shaik Asifullah :
> Hi,
> I am using Apache Storm Trid
Hi,
I haven't really looked at optimizing storm-kafka-client for Trident, but
I'd suggest you start by making a topology that only contains a spout and
no bolts and see what that'll push through. That'll let you figure out
whether the bottleneck is in your processing or in the spout, or in the
par
RM/versions/12343340
>
> -Jungtaek Lim (HeartSaVioR)
>
> 2018년 7월 29일 (일) 오후 7:42, Stig Rohde Døssing 님이 작성:
>
>> I'm not aware of a workaround, but why not check out 1.x-branch and build
>> storm-kafka-client yourself? You shouldn't need to upgrade the Storm
>
I'm not aware of a workaround, but why not check out 1.x-branch and build
storm-kafka-client yourself? You shouldn't need to upgrade the Storm
cluster to a snapshot to fix this, just replace the storm-kafka-client jar.
2018-07-26 17:28 GMT+02:00 Mitchell Rathbun (BLOOMBERG/ 731 LEX) <
mrathb...@bl
You might be affected by this issue
https://issues.apache.org/jira/browse/STORM-3102.
2018-06-26 8:33 GMT+02:00 zz :
> hi
> I am a new storm user, i have met a performance problem,please give me
> some suggestion.
>
> dev environment:
> server os:centos 7.5
> zookeeper:3.4.10(one node)
> kafka:2.
64.invoke(executor.clj:468)
> ~[storm-core-1.2.1.jar:1.2.1]
> at
> org.apache.storm.disruptor$clojure_handler$reify__4475.onEvent(disruptor.clj:41)
> ~[storm-core-1.2.1.jar:1.2.1]
> at
> org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.jav
No, haven't seen it before.
I took a look at NamedTopicFilter, and I think what's happening is that the
partitionsFor call here
https://github.com/apache/storm/blob/v1.2.1/external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/NamedTopicFilter.java#L57
is returning null. Did you co
w Consumer handle failure of Kafka brokers? If I connect to
> one broker on 9092 and that brokers dies, how will it know to connect to
> the other on 9093? Is it because it knows that consumer group belongs to
> two brokers?
>
> Thanks
> Pavel
>
> On Sun, May 20, 2018 at 10:5
The storm-kafka-client module is intended as the replacement for the
storm-kafka module. The storm-kafka module uses a Kafka client that is now
deprecated (
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/consumer/SimpleConsumer.scala)
under the hood. Storm-kafka-client uses
ht
Hi,
That's not possible as far as I know. You can route tuples based on the
tuple content, but you can't restructure the topology while it is running.
Consider elaborating on what you're trying to accomplish, that way it's
easier to suggest a solution.
2018-05-18 12:47 GMT+02:00 Abdul Samad :
>
ript (storm.ps1)
>> to interact with Storm instead of the storm.cmd script.
>> If not, see if you can reproduce the issue with the example topology I
>> linked above. This would let us rule out an issue with the topology code.
>>
>> 2018-05-08 19:11 GMT+02:00 Satyam Sing
e
>
> On 08-May-2018, at 10:26 PM, Stig Rohde Døssing wrote:
>
> Thanks. I'll try to reproduce it. What kind of spout/topology is it (e.g.
> "regular", Trident, multilang)?
>
> 2018-05-08 18:51 GMT+02:00 Satyam Singh :
>
>> Hi Stig,
>>
>> I am
1 - 100 of 205 matches
Mail list logo