Skip to content

Optional long-polling based segment announcement via HTTP instead of Zookeeper#3902

Merged
himanshug merged 8 commits intoapache:masterfrom
himanshug:seg_assignment
May 17, 2017
Merged

Optional long-polling based segment announcement via HTTP instead of Zookeeper#3902
himanshug merged 8 commits intoapache:masterfrom
himanshug:seg_assignment

Conversation

@himanshug
Copy link
Copy Markdown
Contributor

@himanshug himanshug commented Feb 2, 2017

This PR introduces following configurations.

//to be used at coordinators and brokers to use an implementation of ServerInventoryView that
//syncs segments list over HTTP instead of zookeeper
druid.announcer.type=http

//to be used at historicals and realtime tasks to stop pushing segment updates to zookeeper
//false by default
druid.announcer.skipSegmentAnnouncementOnZk=true

I'm keeping those undocumented for now. We will use them on our big internal clusters first before asking users to use.

changes -
All nodes, that serve segments e.g. historicals and realtime indexing tasks, provide a "/druid-internal/v1/segments" HTTP endpoint that can incrementally provide the list of segments being served by that node, this uses Async IO so jetty threads are not held waiting. To power this endpoint, following method is added to DataSegmentAnnouncer interface.

  /**
   * Returns Future that lists the segment load/drop requests since given counter.
   */
  public ListenableFuture<SegmentChangeRequestsSnapshot> getSegmentChangesSince(SegmentChangeRequestHistory.Counter counter);

On coordinator/brokers side, HttpServerInventoryView class is introduced which is equivalent to BatchServerInventoryView but syncs the segment inventory using the mentioned HTTP endpoint. Server discovery
is still done via zookeeper. ServerInventoryView is made an interface and its old contents are moved to AbstractCuratorServerInventoryView .

@kaijianding
Copy link
Copy Markdown
Contributor

kaijianding commented Feb 3, 2017

If I understand correctly, you update segments by calling httpClient.go(url) for each server in the servers list infinitely inside a thread pool.
To my knowledge, the httpClient.go() use a connection pool which actually using ImmediateCreationResourceHolder. This resourceHolder holds an objectList whose size is configed by broker.numConnections. httpClient.go takes first connection from the head of objectList and give it back and add the connection to the end of objectList when httpClient.go() finished.
In this way all connections hold by this resourceHolder are active(or hot?), whether this is a kind of waste if the druid cluster is a not a busy cluster?

If the cluster is not a busy cluster, can a connection is in disconnected state(disconnected by historical/realtime node when over the idleTimeOut) when attempt to use this connection to get segment list? In this case a new connection is created(renew), will it cause performance issue if lots of connections need to be renew? Can always get a disconnected connection for the server if the servers list is large and numConnections is too big?

@himanshug himanshug force-pushed the seg_assignment branch 2 times, most recently from bd2f8de to 373feff Compare February 3, 2017 17:43
@himanshug
Copy link
Copy Markdown
Contributor Author

@kaijianding thanks for taking a look. you are right about the connection usage and yes coordinator/broker would be using one connection from HttpClient per historical/realtime all the time.
On a non-busy cluster or in the worst case coordinator/broker would have to create one connection per historical/realtime every 4 mins ( default config in HttpServerInventoryViewConfig ). But, They will be created sequentially and not in parallel.
However, it turns out that this problem can be very easily fixed by changing the get/giveBack behavior of ResourcePool.ImmediateCreationResourceHolder to LIFO instead of current FIFO. It can be done by replacing https://github.com/metamx/http-client/blob/master/src/main/java/com/metamx/http/client/pool/ResourcePool.java#L237 by objectList.addFirst(object). I will do a PR to fix that.

btw, I have also made few updates in SegmentListerResource to make it async and not hold any jetty threads while in wait.

@nishantmonu51 nishantmonu51 self-assigned this Feb 5, 2017
@drcrallen
Copy link
Copy Markdown
Contributor

Related #2368

@himanshug
Copy link
Copy Markdown
Contributor Author

himanshug commented Feb 8, 2017

removing the discovery of historical/realtime nodes from zookeeper is gonna relate with #2312 (as described in #2312 (comment) ) . I am looking into that and that would be done in a separate PR.

@himanshug himanshug added this to the 0.10.1 milestone Feb 22, 2017
@weijietong
Copy link
Copy Markdown

@himanshug according to your idea, through looping asking segments from historical/realtime nodes background ,how to avoid holding stale segment infos on coordinator/brokers side if not looped in time while a query occurred ?

@himanshug
Copy link
Copy Markdown
Contributor Author

@weijietong brokers/coordinators would not have stale segment info because they are running inifinite (no wait) loop to get latest segment information from historical/realtime nodes.... historical/realtime nodes "hold" the request till there is new information to provide or timeout provided in the request is reached.

@pjain1
Copy link
Copy Markdown
Member

pjain1 commented Mar 10, 2017

@himanshug a general comment - it would be good to put some comments on important classes, variables and methods. For example, a high level design comment on HttpServerInventoryView, some comments on SegmentChangeRequestHistory, usage of hash in Counter, reason of passing waitingFutures in SettableFuture etc. This would make it easier to understand the code and help the person changing the code not make mistakes.

I have gone through the code at high level and it looks good me to so far.

@himanshug himanshug force-pushed the seg_assignment branch 2 times, most recently from 3962f92 to 824631a Compare April 11, 2017 15:45
@himanshug
Copy link
Copy Markdown
Contributor Author

@pjain1 added some docs for clarification

@gianm
Copy link
Copy Markdown
Contributor

gianm commented Apr 11, 2017

@himanshug, I know I said on the dev sync today that I thought this should remain undocumented, but I changed my mind. I'm still ok with it only being reviewed by people from your organization, but I think it should be documented as experimental. That way other folks could help try it out too.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

It feels weird to me that this is taking a CuratorFramework. I think it would be better for this to depend on something closer to what it actually needs (i.e. something that allows for discovering nodes in the cluster). The implementation for that could remain essentially the same as what exists here, but I think it would be nice to not clutter this class with that code.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

refactored server notifications in separate class .

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

"segment callbacks" reads to me as if it should be called on every single segment. The actual callback being called here seems like it shouldn't be called quite that often (maybe once?). Is something named weird or am I mistaken about what's happening?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

yes , it is for individual segments and once at inventory initialization. this naming and behavior is retained from CuratorBasedServerInventoryView ( called ServerInventoryView earlier)

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why does this block on the fetch?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

not needed anymore , it was needed in an older version of code ... removed now

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

What if holder is null?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added check

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why are we guarding a final?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added comment in code, this is to ensure consistent segment state stored in DruidServer and counter managed.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This might be pre-mature optimization, but I worry about how this data structure is being used. One option would be to view the list as a revolving buffer and essentially maintain an index of the oldest "valid" item. You can replace and increment that on add and start your search from there wrapping back around.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

added a CircularBuffer impl instead of using list as a circular buffer.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Please include the actual values that came in here, just the message as it exists won't really help much with debugging if it actually gets thrown.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

updated

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

This is an optimization, maybe not so important, but I think you could actually compute the index into the array with some maths and the counters.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

not really and also as you said not so important.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Why are we removing the isAnnounced check?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

updated BatchDataSegmentAnnouncer.announce(..) to always do the check so that it gets done in other places as well.

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

If you do what I wrote in a previous comment and separate the lookup management into a different implementation of DataSegmentAnnouncer, then have that concrete type injected here instead of the interface (if you mark it @Nullable Guice should be fine even if it's not bound because the http thing isn't being used)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this now directly uses BatchDataSegmentAnnouncer

@himanshug
Copy link
Copy Markdown
Contributor Author

@cheddar I also introduced DataSegmentServerAnnouncer and removed server announcements from DataSegmentAnnouncer implementation. Now only node who really serve segments will announce themselves in inventory and not random peon task processes.

@himanshug
Copy link
Copy Markdown
Contributor Author

@gianm I still think we should keep it undocumented for now and not confuse regular users with a feature which is just to optimize things and haven't been used at scale yet. We will try and document it in next release probably.

@cheddar
Copy link
Copy Markdown
Contributor

cheddar commented May 12, 2017

👍

@himanshug himanshug merged commit daa8ef8 into apache:master May 17, 2017
@himanshug himanshug deleted the seg_assignment branch December 29, 2017 17:35
seoeun25 added a commit to seoeun25/incubator-druid that referenced this pull request Jan 10, 2020
    * Optional segment announcement via HTTP (apache#3902)

    * BatchServerInventoryView is created twice (apache#3244)
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.

8 participants