Skip to content

Issue tracker [STORM-3362] Solved: eventHubSpout uses a blocking receiver in nextTuple()#2981

Merged
asfgit merged 1 commit intoapache:masterfrom
CaperAi:fixing_blocking_call_in_EventHubSpout
Mar 29, 2019
Merged

Issue tracker [STORM-3362] Solved: eventHubSpout uses a blocking receiver in nextTuple()#2981
asfgit merged 1 commit intoapache:masterfrom
CaperAi:fixing_blocking_call_in_EventHubSpout

Conversation

@ysyyork
Copy link
Contributor

@ysyyork ysyyork commented Mar 26, 2019

No description provided.

@srdo
Copy link
Contributor

srdo commented Mar 26, 2019

Thanks for the contribution.

The change looks fine, but I think we should let people configure the timeout. Could we add it to https://github.com/apache/storm/blob/8d4432233c8776247da520b5935b7eebfc267ba5/external/storm-eventhubs/src/main/java/org/apache/storm/eventhubs/spout/EventHubSpoutConfig.java ?

@ysyyork ysyyork force-pushed the fixing_blocking_call_in_EventHubSpout branch from 8d44322 to b779c00 Compare March 26, 2019 14:38
@ysyyork
Copy link
Contributor Author

ysyyork commented Mar 26, 2019

Good suggestion. How about this? @srdo

@srdo
Copy link
Contributor

srdo commented Mar 26, 2019

LGTM, thanks. Please update the commit message to contain the JIRA issue (see the other PRs for example).

Will merge after the 24 hour waiting period, assuming tests pass :)

@ysyyork ysyyork force-pushed the fixing_blocking_call_in_EventHubSpout branch from b779c00 to 7419fd1 Compare March 26, 2019 14:43
@ysyyork
Copy link
Contributor Author

ysyyork commented Mar 26, 2019

Sure, done

@srdo
Copy link
Contributor

srdo commented Mar 26, 2019

Great, +1 pending tests.

@srdo
Copy link
Contributor

srdo commented Mar 26, 2019

Test failure is unrelated.

@ysyyork
Copy link
Contributor Author

ysyyork commented Mar 26, 2019

yeah, i didn't get it either. do I need to do anything?

@srdo
Copy link
Contributor

srdo commented Mar 26, 2019

Not for this PR, no.

@ysyyork
Copy link
Contributor Author

ysyyork commented Mar 26, 2019

Alright thx

Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The code itself looks good. My 2 cents default value may need to be reconsidered.

// disabling filter
private String connectionString;
private String topologyName;
private int receiverTimeoutInMillis = 10; // default
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is 10ms a realistic value for timeout? Since it's the default value and end users don't touch unless they face timeout error, the value would need to cover most of cases. We know it should be short to let Spout handles ack/fail fast, but 10ms sounds too small to reach remote-end.

Copy link
Contributor Author

@ysyyork ysyyork Mar 28, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you are right. I think this depends on how big is the payload in event hub. eventhub has a 1 MB limit for packet size. If we considering worst case scenario, then the bandwidth requirements are 1MB/0.01sec = 100M/sec which seems a really high requirements. based on this, let's tune this to like 100mills so the bandwidth requirements become 10M/sec. do you think this is reasonable? let me know. My current application, the payload is pretty small. only like several bytes so it doesn't matter.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not familiar with the EH client library, so I don't know if it fetches data in a background thread, but assuming it only fetches data when polled, latency to EH will be a concern. With a default of 100ms, you will only be able to fetch data at all if the latency is less than 100ms.

The Kafka spout uses 200ms by default, but Kafka can be run "near" the spout (in network hops terms, e.g. in the same LAN), while EH is a hosted service. It might be good to set it conservatively, e.g. to 500-1000ms, and let people fine tune it manually.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sounds good. let me set it to 1 sec then to be safe

@ysyyork ysyyork force-pushed the fixing_blocking_call_in_EventHubSpout branch from 7419fd1 to ec0a99a Compare March 28, 2019 15:37
Copy link
Contributor

@HeartSaVioR HeartSaVioR left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@asfgit asfgit merged commit ec0a99a into apache:master Mar 29, 2019
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants