Optional segment load/drop management without zookeeper using http#4874
Optional segment load/drop management without zookeeper using http#4874himanshug wants to merge 9 commits intoapache:masterfrom
Conversation
50043d1 to
103d434
Compare
There was a problem hiding this comment.
Pair is a really bad class to build an API around. It is poorly defined in terms of what the LHS and RHS are. Please remove the JSON annotations here and instead create a concrete class that actually represents what you are sending across the wire.
In general, if you ever see Pair<> on any sort of public function signature or API, it was done incorrectly.
There was a problem hiding this comment.
not using Pair anymore.
There was a problem hiding this comment.
You did not document this, was that intentional?
There was a problem hiding this comment.
yes, will let it run on our servers for a while before documenting. hopefully we'll have enough confidence in this before 0.11.1 release so will get documented before that release.
There was a problem hiding this comment.
This annotation is an abomination. Just make it public
There was a problem hiding this comment.
This constructor requires creator to pass the ExecutorService which is very handy for tests to be able to deterministic. However, I don't really want any other code to use this constructor and hence added that annotation. If you still believe that its not useful then I'll remove it.
There was a problem hiding this comment.
Are there no extra objects you could log out with this to provide more information in the log?
There was a problem hiding this comment.
no, just wanted to mark start/stop in log to check whether it indeed gets started/stopped etc.
There was a problem hiding this comment.
Is this doing a cache of the load/drop requests or a cache of the segments that have already been stored? If the latter, Niketh has a PR that moves the cache to be co-located with the segment storage instead of a separate cache directory. This won't follow that logic, please explore.
There was a problem hiding this comment.
unrelated to work in this PR. its a copy paste from ZkCoordinator and this PR is keeping old things exactly same.
There was a problem hiding this comment.
This is acting like these requests succeeded, is that the correct thing to do?
There was a problem hiding this comment.
yes, this behavior is retained from existing behavior in CuratorLoadQueuePeon (copy of older LoadQueuePeon class )
There was a problem hiding this comment.
What's the difference between this and dropSegment?
There was a problem hiding this comment.
this behavior is retained from existing behavior in CuratorLoadQueuePeon (copy of older LoadQueuePeon class ). I am not sure about this but it was recently added so kept.
There was a problem hiding this comment.
Should this really be returning the set, which is likely mutable?
There was a problem hiding this comment.
this behavior is retained from existing behavior in CuratorLoadQueuePeon (copy of older LoadQueuePeon class ). but changed to return unmodifiable version.
There was a problem hiding this comment.
What's the point of this type thing? We should either leverage polymorphism or completely eliminate it. Using it sometimes and not other times gets confusing
There was a problem hiding this comment.
changed to use Polymorphism.
There was a problem hiding this comment.
I really don't understand this cancellation future stuff....
There was a problem hiding this comment.
this code is removed.
ec0b74f to
20ca85b
Compare
20ca85b to
c8d1c0a
Compare
TODOs:Test on a cluster with decent load.This patch introduces following changes.
On Historical Side:
ListenableFuture SegmentLoadDropHandler.processBatch(List<DataSegmentChangeRequest>)is added to support http endpoint for segment load/drop management.POST /druid-internal/v1/segments/changeRequestsis added in SegmentListerResource that usesListenableFuture SegmentLoadDropHandler.processBatch(List<DataSegmentChangeRequest>)to delegate batch of load/drop requests received. Here is a copy-paste of javadoc for this endpoint.druid.segmentCache.numLoadingThreadsconfiguration is revived and must be greater than the"druid.coordinator.loadqueuepeon.http.batchSizedescribed later.On Coordinator Side:
druid.coordinator.loadqueuepeon.type= http or curator, curator by defaultdruid.coordinator.loadqueuepeon.http.batchSize= number of load/drop to try and process in parallel, must be less thandruid.segmentCache.numLoadingThreadson historicaldruid.coordinator.loadqueuepeon.http.repeatDelay= delay Duration for periodic check for new load/drop request to send to historical. note this could be very large given that same code is executed whenever a new load/drop is requested to HttpLoadQueuePeon without waiting for schedule to kick in, default 1 minutedruid.coordinator.loadqueuepeon.http.hostTimeout= default 5 minutes, timeout used to be specified on new endpoint introduced for batch load/drop request.In the long run, I plan to remove CuratorLoadQueuePeon and ZkCoordinator. They are being kept now only to be backward compatible and for HttpLoadQueuePeon to prove itself in production.
Documentation for HttpLoadQueuePeon is intentionally left out.