[GitHub] storm pull request #1760: Add topology stream-awareness to storm-redis

2016-11-03 Thread mo-getter
Github user mo-getter commented on a diff in the pull request:

https://github.com/apache/storm/pull/1760#discussion_r86495472
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
 ---
@@ -45,18 +48,41 @@
  */
 public class RedisFilterBolt extends AbstractRedisBolt {
 private final RedisFilterMapper filterMapper;
+private final StreamMapper streamMapper;
 private final RedisDataTypeDescription.RedisDataType dataType;
 private final String additionalKey;
 
 /**
- * Constructor for single Redis environment (JedisPool)
+ * Constructor for single Redis environment (JedisPool).
+ * Tuples will be emitted to Storm's default streamId.
  * @param config configuration for initializing JedisPool
  * @param filterMapper mapper containing which datatype, query key 
that Bolt uses
  */
 public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper 
filterMapper) {
--- End diff --

Thanks for taking a look at this. I see your point, and I agree it makes 
the most sense for both to be done in the same interface. TL;DR is at the 
bottom ;-)

The same problem also applies to RedisLookupMapper, which also defines 
`declareOutputFields` separately. I just came across 
[STORM-1953](https://issues.apache.org/jira/browse/STORM-1953).

About your first option, would it make more sense for a FilterMapper to be 
a StreamMapper, rather than a StreamMapper be a FilterMapper? I'm afraid that 
making StreamMapper a FilterMapper would introduce too much ambiguity in the 
lookup and filter bolts (if the constructors accepted both), since we'd have to 
rely on only the docs to define which object's `declareOutputFields` would be 
called. It also would make STORM-1953 worse. Or did you mean the bolts only 
accept FilterMapper, and have something like this in `execute`:
```
if (filterMapper instanceof StreamMapper) {
String streamId = ((StreamMapper) filterMapper).getStreamId(input, 
value);
collector.emit(streamId, input, value);
} else {
collector.emit(input, value);
}
```
Either way, if you combine them, one downside is that the provided 
convenience StreamMapper implementations would have to be sacrificed. Making 
them abstract probably wouldn't be worth it for something like just specifying 
the stream.

In case you want to see what having FilterMapper and LookupMapper also 
extend StreamMapper looks like, I implemented that in a branch 
[here](https://github.com/apache/storm/compare/master...mo-getter:stream-mapper).
 The flexibility to dynamically choose a stream is there, but the problem is 
that the trident-related classes also use LookupMapper, and have no need to 
declare a streamId, yet users will have to implement this method in their 
LookupMappers. Just returning null is one [not so good] option here, and is 
also an option when using LookupMapper for bolts (in which case, the existing 
behavior of emitting to the default stream is maintained).

**TL;DR:** I can't think of a great solution for what you mentioned, while 
maintaining user-friendliness of the API, without totally redoing the Mapper 
interfaces, i.e. 
[STORM-1953](https://issues.apache.org/jira/browse/STORM-1953). On the other 
hand, the above commit does maintain full backward compatibility and is 
probably most convenient for users!


---
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 #1470: STORM-1886 Extend KeyValueState iface with delete

2016-11-03 Thread arunmahadevan
Github user arunmahadevan commented on the issue:

https://github.com/apache/storm/pull/1470
  
@kosii can you up-merge and push your local repo changes so that we can 
merge 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.
---


Re: How does the control flow in a Trident Topology work?

2016-11-03 Thread Manu Zhang
Hi Li,

Here is an old Chinese article on Trident
 that explains the procedure
with some images. I think you read Chinese, right ?
The article is actually one in a series which may help you more to
understand Trident.

Cheers,
Manu Zhang

On Thu, Nov 3, 2016 at 10:55 PM Bobby Evans 
wrote:

> It is very confusing and i don't completely understand it all myself, but
> for the most part what happens, as far as I understand it, is that the
> master batch coordinator sends out begin batch messages to all of the spout
> coordinators and all the state instances saying batch X is about to begin.
> When it gets an ack back (saying everyone saw it and are ready for it) it
> will then send out a message saying emit batch X,  (all tuples within that
> batch are tied to this emit as the are processed).  If that comes back as
> success then it will send out a final message to commit batch X. (but only
> if batch X-1 has already been committed).  If any of these fail it will ask
> the spouts to replay the batch.
> - Bobby
>
> On Wednesday, November 2, 2016, 5:38:29 PM CDT, Li Wang <
> wangli1...@gmail.com> wrote:Hi guys,
>
> I am trying to understand the implementation of Trident. Through reading
> the code in TridentTopolgyBuilder.java, I understand that some Coordinator
> components, such as MasterBatchCoordinator and TridentSpoutCoordinator, are
> added to a user defined topology in
> TridentTopologyBuilder.createTopology(). I try to understand the control
> flow of those coordinators, but is seems to be very difficult to get the
> sense just from source code. Is there any document giving a high level of
> the control flow of the coordinator components in a Trident Topology?
>
> Any help is highly appreciated. Thanks!
>
> Sincerely,
> Li Wang


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

2016-11-03 Thread hmcl
Github user hmcl commented on the issue:

https://github.com/apache/storm/pull/1679
  
@revans2 apologies for the delay... started it and then something came in 
the way... will finish by tomorrow for sure. Will do my best to get this merged 
in asap...


---
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 #1753: STORM-2180: KafkaSpout ack() should handle re-bala...

2016-11-03 Thread knusbaum
Github user knusbaum closed the pull request at:

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


---
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 #1753: STORM-2180: KafkaSpout ack() should handle re-balance - 1...

2016-11-03 Thread knusbaum
Github user knusbaum commented on the issue:

https://github.com/apache/storm/pull/1753
  
Closing for #1696 per request.


---
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 #1752: STORM-2180: KafkaSpout ack() should handle re-bala...

2016-11-03 Thread knusbaum
Github user knusbaum closed the pull request at:

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


---
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 #1752: STORM-2180: KafkaSpout ack() should handle re-balance

2016-11-03 Thread knusbaum
Github user knusbaum commented on the issue:

https://github.com/apache/storm/pull/1752
  
Fine by me.


---
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 #1761: STORM-2184: Don't wakeup KafkaConsumer on shutdown, spout...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1761
  
+1 The discussion on STORM-2184 makes since that wakeup seems to not be 
needed.


---
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 #1721: Update storm kafka docs.

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

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


---
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 #1721: Update storm kafka docs.

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1721
  
@aandis thanks for the fix.  I merged it into master.


---
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 #1721: Update storm kafka docs.

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1721
  
+1


---
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 #1752: STORM-2180: KafkaSpout ack() should handle re-balance

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1752
  
Looking at the code #1696 is a much more complete solutions.  @knusbaum 
could you take a look at it and let us know what you think.


---
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 #1761: STORM-2184: Don't wakeup KafkaConsumer on shutdown...

2016-11-03 Thread srdo
GitHub user srdo opened a pull request:

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

STORM-2184: Don't wakeup KafkaConsumer on shutdown, spout methods are…

... not called by multiple threads at a time

See https://issues.apache.org/jira/browse/STORM-2184



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

$ git pull https://github.com/srdo/storm STORM-2184

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

https://github.com/apache/storm/pull/1761.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 #1761


commit ac4f0404590eda08308c7698f1ada9963563f6a6
Author: Stig Rohde Døssing 
Date:   2016-11-02T17:36:44Z

STORM-2184: Don't wakeup KafkaConsumer on shutdown, spout methods are not 
called by multiple threads at a time




---
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 #1735: STORM-203 Adding paths to default java library pat...

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

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


---
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 #1696: STORM-2104: More graceful handling of acked/failed tuples...

2016-11-03 Thread srdo
Github user srdo commented on the issue:

https://github.com/apache/storm/pull/1696
  
@hmcl @revans2 @knusbaum Could you guys take a look at this at some point? 
I noticed that two new PRs (https://github.com/apache/storm/pull/1752, 
https://github.com/apache/storm/pull/1753) were opened to solve the same 
problem, and this one has been kind of radio silent for a while.


---
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 #1735: STORM-203 Adding paths to default java library path

2016-11-03 Thread knusbaum
Github user knusbaum commented on the issue:

https://github.com/apache/storm/pull/1735
  
+1


---
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 #1760: Add topology stream-awareness to storm-redis

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

https://github.com/apache/storm/pull/1760#discussion_r86378199
  
--- Diff: 
external/storm-redis/src/main/java/org/apache/storm/redis/bolt/RedisFilterBolt.java
 ---
@@ -45,18 +48,41 @@
  */
 public class RedisFilterBolt extends AbstractRedisBolt {
 private final RedisFilterMapper filterMapper;
+private final StreamMapper streamMapper;
 private final RedisDataTypeDescription.RedisDataType dataType;
 private final String additionalKey;
 
 /**
- * Constructor for single Redis environment (JedisPool)
+ * Constructor for single Redis environment (JedisPool).
+ * Tuples will be emitted to Storm's default streamId.
  * @param config configuration for initializing JedisPool
  * @param filterMapper mapper containing which datatype, query key 
that Bolt uses
  */
 public RedisFilterBolt(JedisPoolConfig config, RedisFilterMapper 
filterMapper) {
--- End diff --

This now creates a lot of coupling between the filter mapper and the stream 
mapper.  Simply because the Filter Mapper is the one that declares the output 
fields.

```
public void declareOutputFields(OutputFieldsDeclarer declarer) {
filterMapper.declareOutputFields(declarer);
}
```

So either we need to embrace the coupling and have StreamMapper also be a 
FilterMapper.  (which would require some documentation) or we find a way to 
fake out FilterMapper and have it declare multiple outputs for what the 
StreamMapper wants. 

I prefer the first one, because it seems like it would be more flexible.


---
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 #1735: STORM-203 Adding paths to default java library path

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1735
  
@knusbaum are you OK with the change?

I am +1


---
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 #1755: error log for blobstore was missing a space between strin...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1755
  
@erikdw 

Thanks, I merged this into master.


---
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 #1679: STORM-2087: storm-kafka-client - tuples not always being ...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1679
  
@knusbaum I noticed that you opened up #1753 and #1752.  Would you mind 
taking a look at this and see if there are going to be any conflicts?


---
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 #1755: error log for blobstore was missing a space betwee...

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

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


---
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 #1755: error log for blobstore was missing a space between strin...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1755
  
+1


---
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 #1759: STORM-2185: Storm Supervisor doesn't delete directories p...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1759
  
As a note to whoever merges this in we should merge this to 1.0.x too.


---
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 #1758: STORM-2185: Storm Supervisor doesn't delete directories p...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1758
  
+1 I see that the race seems to be that if a worker is shut down after 
creating the directory, but before running chown on it, then we hit this 
situation.


---
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 #1679: STORM-2087: storm-kafka-client - tuples not always being ...

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1679
  
@hmcl it has been a week, any hope of finishing the review?


---
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: How does the control flow in a Trident Topology work?

2016-11-03 Thread Bobby Evans
It is very confusing and i don't completely understand it all myself, but for 
the most part what happens, as far as I understand it, is that the master batch 
coordinator sends out begin batch messages to all of the spout coordinators and 
all the state instances saying batch X is about to begin.  When it gets an ack 
back (saying everyone saw it and are ready for it) it will then send out a 
message saying emit batch X,  (all tuples within that batch are tied to this 
emit as the are processed).  If that comes back as success then it will send 
out a final message to commit batch X. (but only if batch X-1 has already been 
committed).  If any of these fail it will ask the spouts to replay the batch.
- Bobby

On Wednesday, November 2, 2016, 5:38:29 PM CDT, Li Wang  
wrote:Hi guys,

I am trying to understand the implementation of Trident. Through reading the 
code in TridentTopolgyBuilder.java, I understand that some Coordinator 
components, such as MasterBatchCoordinator and TridentSpoutCoordinator, are 
added to a user defined topology in TridentTopologyBuilder.createTopology(). I 
try to understand the control flow of those coordinators, but is seems to be 
very difficult to get the sense just from source code. Is there any document 
giving a high level of the control flow of the coordinator components in a 
Trident Topology?

Any help is highly appreciated. Thanks!

Sincerely,
Li Wang

[GitHub] storm issue #1748: [STORM-2103][SQL] Introduce new sql external module: stor...

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

https://github.com/apache/storm/pull/1748
  
@vesense 
I manually tested that it works. Thanks for the great work!
Btw, I'm seeing that only one fields are used for storing the value, but I 
guess some users occasionally would want to store the document as it is (not 
serialized to one field). I'm OK if there's specific reason to do it (for 
example, supporting schema matters) but if it's not, I feel "storing as it is" 
is more natural and what users expected without any configurations.


---
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: [DISCUSS] Drop features about GROUP BY, JOIN from Storm SQL Trident mode

2016-11-03 Thread Jungtaek Lim
This seems to be no other opinions, so I would go on dropping "group by"
and "join" for Storm SQL Trident mode until we're ready to handle windowing
on SQL semantic.

- Jungtaek Lim (HeartSaVioR)

2016년 10월 22일 (토) 오전 12:23, Jungtaek Lim 님이 작성:

> Yes there seems many things which already supports CQL (for example
> InfluxDB), and what I mean is that there's no Streaming SQL or CQL
> standard, in point of "SQL semantic" view.
>
> We can define LINQ style "API" and include aggregation and join with
> enough discussions (if my understanding is right, that's what Structured
> Streaming is, and Flink Table API is also going ahead), but we don't even
> have higher-level API and it will be going to be duplicated work (and it
> should, if higher-level API is well defined).
>
> As I linked earlier, Calcite project proposes its streaming sql semantic,
> which defines new keywords, and new concepts, and so on.[1
> ] The thing is it's not
> implemented yet, since Calcite is having small community, and most of
> contributors are from SQL on Hadoop, not streaming area. Only a small
> contributions are done from us and Flink side.
>
> Storm SQL has some remaining works even we don't address aggregation for
> now, so it would be not easy to jump on Calcite side and discuss or
> persuade or even help implementing. One of Flink committer initiated
> discussion regarding "defining Streaming SQL semantics" [2
> ]
> but it seems that not many devs. are interested. Still seems to be an early
> stage for all.
>
> So working on our own and late participating is also a valid way we can
> choose, or participating Flink's discussion now is also a valid way. Storm
> SQL has limited contributors so IMHO we need to prioritize and concentrate.
> Unless we have more contributors coming in Storm SQL, former way looks more
> realistic.
>
> - Jungtaek Lim (HeartSaVioR)
>
> [1] https://calcite.apache.org/docs/stream.html
> [2]
> http://mail-archives.apache.org/mod_mbox/flink-dev/201610.mbox/%3ccaadrtt2t397e_jpnjm6zh-ysn8i0oouno8bnxsotvflmwh5...@mail.gmail.com%3E
>
> 2016년 10월 21일 (금) 오후 10:50, Bobby Evans 님이
> 작성:
>
> I am not currently very involved with the storm SQL so take my comments
> worth a grain of salt.  I am +1 on waiting to define groupby and join until
> we have a solid base that we can build this on.
> But I do want to contradict a bit about what there being no standard for
> streaming SQL.  Technically that is true, but in general the streaming SQL
> solutions I have seen restrict the supported queries to either have no
> aggregations at all (pure element by element operations) or rely on the
> OVER clause, and more specifically time based windows.
> The issue is that the output of a streaming operation is not a table, it
> is a protocol that includes updates.  This is where BEAM really shines
> because it exposes all of that ugly underbelly and lets users have complete
> control over that. The API is rather complex and I think a bit ugly because
> of that.  I would suggest that we define what we want it to look like, and
> let that drive the underlying implementation.  If we are feeling ambitions
> we can get together with Flink, Spark, and anyone else who might be
> interested and see if we can come to an understanding on what extensions to
> SQL we should put in to really make streaming work properly.
> - Bobby - Bobby
>
> On Friday, October 21, 2016 2:18 AM, Jungtaek Lim 
> wrote:
>
>
>  Hi devs,
>
> Sorry to send multiple mails at once, I had been resolving issues
> sequentially, and now stopped a bit and retrospect about the direction of
> Storm SQL.
>
> I'd like to propose destructive actions, dropping features about GROUP BY
> and JOIN from Storm SQL which are fortunately not released yet.
>
> The reason of dropping features is simple: This borrows Trident semantic
> (within micro-batch, or stateful), and not making sense of true "streaming"
> semantic.
>
> Spark and Flink interpret "streaming" aggregation and join as windowed
> operators. Since there's no SQL standard for streaming (even no de-facto),
> they are adding the feature to its API (Structured Streaming for Spark, and
> Table API for Flink), and don't address them to SQL side yet.
>
> I was eager to add more features on Storm SQL to make progress (even Bobby
> pointed out similarly), but after worked on these things, I change my mind
> that letting users not confusing is more important than adding features.
>
> Btw, Storm SQL "temporary" relies on Trident since we don't have
> higher-level API on core and we don't want to build topology from ground
> up. AFAIK, choosing Trident is not for living with micro-batch, and IMHO it
> should run on per-tuple streaming manner instead of micro-batch.
> Integrating streams API to Storm SQL could be great internal project for
> POC of s

[GitHub] storm issue #1756: STORM-1278: Port org.apache.storm.daemon.worker to java

2016-11-03 Thread revans2
Github user revans2 commented on the issue:

https://github.com/apache/storm/pull/1756
  
Looks great I am +1 even without the last comment I made (That code can go 
away when it is translated to java).

But since I also contributed some of the test code changes I really would 
like someone else to have a look too.


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to...

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

https://github.com/apache/storm/pull/1756#discussion_r86344854
  
--- Diff: storm-core/src/clj/org/apache/storm/testing.clj ---
@@ -667,23 +665,18 @@
   (.put "spout-emitted" (AtomicInteger. 0))
   (.put "transferred" (AtomicInteger. 0))
   (.put "processed" (AtomicInteger. 0
+  (LocalExecutor/setTrackId id#)
   (with-var-roots
--- End diff --

We don't need with-var-roots any more, we could just call with 
simulated-time-local-cluster without it


---
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 #1756: STORM-1278: Port org.apache.storm.daemon.worker to java

2016-11-03 Thread abhishekagarwal87
Github user abhishekagarwal87 commented on the issue:

https://github.com/apache/storm/pull/1756
  
The PR is ready now.


---
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.
---