[GitHub] storm issue #1679: STORM-2087: storm-kafka-client - tuples not always being ...

2016-11-18 Thread harshach
Github user harshach commented on the issue:

https://github.com/apache/storm/pull/1679
  
Thanks for the patch @jfenc91 and for being patient with reviews.
Merged into master. I would like to merge this into 1.x-branch as well. 
There are few issues on SingleTopicKafkaSpoutTest.java as its using Java8 
syntax. Can you please open another PR for 1.x-branch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm pull request #1679: STORM-2087: storm-kafka-client - tuples not always...

2016-11-18 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/storm/pull/1679


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


RE: NullPointerException on startup

2016-11-18 Thread Cuneo, Nicholas
I’m having trouble finding that version, is it located in a separate repository?

Thanks,
Nick

From: P. Taylor Goetz [mailto:ptgo...@gmail.com]
Sent: Friday, November 18, 2016 12:40 PM
To: dev@storm.apache.org; Cuneo, Nicholas
Subject: Re: NullPointerException on startup

(Moving thread to dev@ to discuss versions not yet released)

Nick,

I suspect you might be hitting a bug in 1.0.2 that might be fixed in one of the 
next releases.

Can you try setting the version of storm-kafka-client to 1.1.0-SNAPSHOT (leave 
other storm components at 1.0.2)?

If that fixes the issue, let us know on 
dev@storm.apache.org so we can back port the fix.

-Taylor

On Nov 18, 2016, at 2:10 PM, Cuneo, Nicholas 
> wrote:

Hello,

We are working with storm 1.0.2 and using Kafka client to subscribe to kafka 
topics to retrieve data.  Randomly when our topologies are starting up we 
receive a null pointer exception which is killing the topology.  We can’t seem 
to identify why this happens, or what measures we can take to prevent it.  I’m 
considering raising a ticket with storm to check for null to prevent the 
topology from crashing.

2016-11-17 23:11:05.366 o.a.s.util [ERROR] Async loop died!
java.lang.RuntimeException: java.lang.NullPointerException
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:464)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:430)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatch(DisruptorQueue.java:420) 
~[storm-core-1.0.2.jar:1.0.2]
at org.apache.storm.disruptor$consume_batch.invoke(disruptor.clj:69) 
~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$fn__7990$fn__8005$fn__8036.invoke(executor.clj:628)
 ~[storm-core-1.0.2.jar:1.0.2]
at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484) 
[storm-core-1.0.2.jar:1.0.2]
at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
at java.lang.Thread.run(Thread.java:745) [?:1.8.0_91]
Caused by: java.lang.NullPointerException
at org.apache.storm.kafka.spout.KafkaSpout.ack(KafkaSpout.java:316) 
~[stormjar.jar:?]
at org.apache.storm.daemon.executor$ack_spout_msg.invoke(executor.clj:448) 
~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$fn__7990$tuple_action_fn__7996.invoke(executor.clj:536)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
 ~[storm-core-1.0.2.jar:1.0.2]
... 7 more
2016-11-17 23:11:05.379 o.a.s.d.executor [ERROR]
java.lang.RuntimeException: java.lang.NullPointerException
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:464)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:430)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatch(DisruptorQueue.java:420) 
~[storm-core-1.0.2.jar:1.0.2]
at org.apache.storm.disruptor$consume_batch.invoke(disruptor.clj:69) 
~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$fn__7990$fn__8005$fn__8036.invoke(executor.clj:628)
 ~[storm-core-1.0.2.jar:1.0.2]
at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484) 
[storm-core-1.0.2.jar:1.0.2]
at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
at java.lang.Thread.run(Thread.java:745) [?:1.8.0_91]
Caused by: java.lang.NullPointerException
at org.apache.storm.kafka.spout.KafkaSpout.ack(KafkaSpout.java:316) 
~[stormjar.jar:?]
at org.apache.storm.daemon.executor$ack_spout_msg.invoke(executor.clj:448) 
~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$fn__7990$tuple_action_fn__7996.invoke(executor.clj:536)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
 ~[storm-core-1.0.2.jar:1.0.2]
at 
org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
 ~[storm-core-1.0.2.jar:1.0.2]
... 7 more
2016-11-17 23:11:05.473 o.a.s.util [ERROR] Halting process: ("Worker died")
java.lang.RuntimeException: ("Worker died")
at org.apache.storm.util$exit_process_BANG_.doInvoke(util.clj:341) 
[storm-core-1.0.2.jar:1.0.2]
at clojure.lang.RestFn.invoke(RestFn.java:423) [clojure-1.7.0.jar:?]
at org.apache.storm.daemon.worker$fn__8663$fn__8664.invoke(worker.clj:765) 
[storm-core-1.0.2.jar:1.0.2]
at 

[GitHub] storm pull request #1786: STORM-1281: LocalCluster, testing4j and testing.cl...

2016-11-18 Thread revans2
Github user revans2 commented on a diff in the pull request:

https://github.com/apache/storm/pull/1786#discussion_r88749944
  
--- Diff: storm-core/test/clj/org/apache/storm/clojure_test.clj ---
@@ -1,159 +0,0 @@
-;; Licensed to the Apache Software Foundation (ASF) under one
--- End diff --

There is a copy of this in storm-clojure already so no need to two of them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


Re: NullPointerException on startup

2016-11-18 Thread P. Taylor Goetz
(Moving thread to dev@ to discuss versions not yet released)

Nick,

I suspect you might be hitting a bug in 1.0.2 that might be fixed in one of the 
next releases.

Can you try setting the version of storm-kafka-client to 1.1.0-SNAPSHOT (leave 
other storm components at 1.0.2)?

If that fixes the issue, let us know on dev@storm.apache.org 
 so we can back port the fix.

-Taylor

> On Nov 18, 2016, at 2:10 PM, Cuneo, Nicholas  wrote:
> 
> Hello,
>  
> We are working with storm 1.0.2 and using Kafka client to subscribe to kafka 
> topics to retrieve data.  Randomly when our topologies are starting up we 
> receive a null pointer exception which is killing the topology.  We can’t 
> seem to identify why this happens, or what measures we can take to prevent 
> it.  I’m considering raising a ticket with storm to check for null to prevent 
> the topology from crashing.
>  
> 2016-11-17 23:11:05.366 o.a.s.util [ERROR] Async loop died!
> java.lang.RuntimeException: java.lang.NullPointerException
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:464)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:430)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatch(DisruptorQueue.java:420) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at org.apache.storm.disruptor$consume_batch.invoke(disruptor.clj:69) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$fn__7990$fn__8005$fn__8036.invoke(executor.clj:628)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484) 
> [storm-core-1.0.2.jar:1.0.2]
> at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> at java.lang.Thread.run(Thread.java:745) [?:1.8.0_91]
> Caused by: java.lang.NullPointerException
> at org.apache.storm.kafka.spout.KafkaSpout.ack(KafkaSpout.java:316) 
> ~[stormjar.jar:?]
> at 
> org.apache.storm.daemon.executor$ack_spout_msg.invoke(executor.clj:448) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$fn__7990$tuple_action_fn__7996.invoke(executor.clj:536)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
>  ~[storm-core-1.0.2.jar:1.0.2]
> ... 7 more
> 2016-11-17 23:11:05.379 o.a.s.d.executor [ERROR] 
> java.lang.RuntimeException: java.lang.NullPointerException
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:464)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchWhenAvailable(DisruptorQueue.java:430)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatch(DisruptorQueue.java:420) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at org.apache.storm.disruptor$consume_batch.invoke(disruptor.clj:69) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$fn__7990$fn__8005$fn__8036.invoke(executor.clj:628)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at org.apache.storm.util$async_loop$fn__624.invoke(util.clj:484) 
> [storm-core-1.0.2.jar:1.0.2]
> at clojure.lang.AFn.run(AFn.java:22) [clojure-1.7.0.jar:?]
> at java.lang.Thread.run(Thread.java:745) [?:1.8.0_91]
> Caused by: java.lang.NullPointerException
> at org.apache.storm.kafka.spout.KafkaSpout.ack(KafkaSpout.java:316) 
> ~[stormjar.jar:?]
> at 
> org.apache.storm.daemon.executor$ack_spout_msg.invoke(executor.clj:448) 
> ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$fn__7990$tuple_action_fn__7996.invoke(executor.clj:536)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$mk_task_receiver$fn__7979.invoke(executor.clj:464)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.disruptor$clojure_handler$reify__7492.onEvent(disruptor.clj:40)
>  ~[storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.utils.DisruptorQueue.consumeBatchToCursor(DisruptorQueue.java:451)
>  ~[storm-core-1.0.2.jar:1.0.2]
> ... 7 more
> 2016-11-17 23:11:05.473 o.a.s.util [ERROR] Halting process: ("Worker died")
> java.lang.RuntimeException: ("Worker died")
> at org.apache.storm.util$exit_process_BANG_.doInvoke(util.clj:341) 
> [storm-core-1.0.2.jar:1.0.2]
> at clojure.lang.RestFn.invoke(RestFn.java:423) [clojure-1.7.0.jar:?]
> at 
> org.apache.storm.daemon.worker$fn__8663$fn__8664.invoke(worker.clj:765) 
> [storm-core-1.0.2.jar:1.0.2]
> at 
> org.apache.storm.daemon.executor$mk_executor_data$fn__7875$fn__7876.invoke(executor.clj:274)
>  

[GitHub] storm pull request #1786: STORM-1281: LocalCluster, testing4j and testing.cl...

2016-11-18 Thread revans2
GitHub user revans2 opened a pull request:

https://github.com/apache/storm/pull/1786

STORM-1281: LocalCluster, testing4j and testing.clj to java

This is based off of #1744, but I wanted to make sure that my changes were 
publicly available.

Now Testing.java replaces testing4j.clj and depends on an improved 
LocalCluster.java that replaces LocalCluster.clj.

testing.clj now depends on Testing.java and has been moved under 
storm-clojure.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/revans2/incubator-storm STORM-1281

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/storm/pull/1786.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1786


commit 55a94f4a32be667a12d68e2775c55add6c0d5f03
Author: Robert (Bobby) Evans 
Date:   2016-10-24T20:59:36Z

STORM-1276: line for line translation of nimbus to java

commit 2bb7b1f5aa8338b030e1c03e088b79b8b107866a
Author: Robert (Bobby) Evans 
Date:   2016-10-24T21:13:09Z

STORM-1276: missed a TODO in the test code

commit 564ef8308e9ae7efe947cda722ff24ea40f3ad31
Author: Robert (Bobby) Evans 
Date:   2016-10-24T21:26:49Z

Removed unused converter functions

commit 6965b56d3d6c1bc9aefd1b7179b50c86e9316a85
Author: Robert (Bobby) Evans 
Date:   2016-10-24T22:04:09Z

A bit more refactoring

commit 39148edf07f2028c2edcfecc53ea9bfac525988f
Author: Robert (Bobby) Evans 
Date:   2016-11-16T18:25:18Z

Let worker handle empty credentials

commit 2aa9549f9a9ac106f652ca8b6afaaff467cfeee1
Author: Robert (Bobby) Evans 
Date:   2016-10-24T20:59:36Z

STORM-1276: line for line translation of nimbus to java

commit da171591580e259c3a264b8889b1940154ffac93
Author: Robert (Bobby) Evans 
Date:   2016-11-05T15:33:47Z

STORM-1281: LocalCluster, testing4j and testing.clj to java

commit 083f764306d0d48291110f7e0a0697a6a86c1635
Author: Robert (Bobby) Evans 
Date:   2016-11-18T13:18:00Z

Fix some race conditions in nimbus and make test plan compiler share a 
local cluster (runs faster)




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1739: STORM-1443 [Storm SQL] Support customizing parallelism in...

2016-11-18 Thread vesense
Github user vesense commented on the issue:

https://github.com/apache/storm/pull/1739
  
@HeartSaVioR 
>There might be some ways to pass partition count to upstream, and easy way 
to do might be adding method to DataSourcesProvider. What I'm considering is 
that we're now only thinking about partition count, but Calcite supports table 
statistics which contains estimated row count (not available for streaming 
env., partitioning attributes, etc.). Is passing partition count exhaustive? 
I'm not sure.

Perhaps I can give you some suggestions when I have a deeper understanding 
in Calcite. But now I think we can address STORM-2147 using a simple way before 
we get a better one.

>As this is on top of STORM-1446 which needs understanding of Calcite, 
learning Calcite is more important for you to work on further works for Storm 
SQL, especially Storm SQL lacks reviewers to go forward. So if you haven't had 
time, please take your time to get it.

Thanks for your advice, yes, learning Calcite is in my plan.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1742: STORM-2170 [Storm SQL] Add built-in socket datasource to ...

2016-11-18 Thread vesense
Github user vesense commented on the issue:

https://github.com/apache/storm/pull/1742
  
OK. nice :smile: 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1742: STORM-2170 [Storm SQL] Add built-in socket datasource to ...

2016-11-18 Thread HeartSaVioR
Github user HeartSaVioR commented on the issue:

https://github.com/apache/storm/pull/1742
  
@vesense Yes I plan to update #1777 after merging this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] storm issue #1742: STORM-2170 [Storm SQL] Add built-in socket datasource to ...

2016-11-18 Thread vesense
Github user vesense commented on the issue:

https://github.com/apache/storm/pull/1742
  
@HeartSaVioR It would be better to add some `how to use` document.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---