Nick, I think it's worth noting that Kafka is not a real-time system, and most of these use cases (and TBH any that I can imagine) for prioritized topics are real-time use cases.
For example, you wouldn't want to pilot a jet (a canonical real-time system) based on Kafka events, as there is no guarantee that a message will be received, stored, or processed at any particular rate or by any specific deadline. Adding priorities does not change that. It would be incredibly cool to support real-time processing in Kafka, but I think this would require coordination between producers, brokers, and consumers, as is the case with exactly-once delivery, and would likely require huge architectural changes. If we did something like that, priorities would be moot, as a producer could say here's a message, make sure it's processed by this deadline, etc. I'm certainly not saying there are no good real-time use cases (there are many!), but I don't think they are a valid argument in support of prioritized topics. Ryanne On Thu, Jan 17, 2019, 12:35 PM Subhash Sriram <subhash.sri...@gmail.com wrote: > Use case: we process documents from a variety of sources. We want to > process some of these sources in a priority order, but we don’t want to > necessarily finish all the higher priority sources before going to lower > priority because the volume of higher priority sources can be extremely > high. > > We have solved this problem right now with multiple topics and we consume > from them in priority order, pausing every N records, to see if there are > any records from a lower priority source. > > That being said, our company also has a use case similar to what Jeff > described. I think priority topics would be a very valuable feature! > > Thanks, > Subhash > > On Thu, Jan 17, 2019 at 12:34 PM Jeff Widman <j...@jeffwidman.com> wrote: > > > Use case: > > I work for a company that ingests events that come from both real-time > > sources (which spike during the day) and historical log data. > > > > We want the real-time data processed in minutes, and the historical log > > data processed within hours. The consumer's business logic is the same. > > > > Our current plan is to have two topics, and two downstream consumer > groups. > > We plan to have the "hot" consumer group of the real-time data > provisioned > > at the 90th percentile of inbound message rate. And the "cold" log data > at > > the 60th percentile because it's okay if it takes longer to absorb spikes > > in cold data. > > > > Priority topics could *potentially* solve this. > > > > However, one problem we've hit with a similar priority queuing system > built > > using a different tech stack was that if there was even a handful of > > messages in the priority queues, those would keep the consumer just busy > > enough that the cold data would never be processed. > > > > The underlying root cause of the problem was two-fold: > > 1) the API only returned messages from a single queue at a time, so even > if > > the consumer requested 1,000 messages, the scheduler would see a message > in > > the hot queue and immediately return it. By the time the consumer > processed > > that and requested another batch of messages, one more message had > trickled > > into the hot queue. Versus if the API made sure to return a full batch, > > first by filling up the hot queue and then from the cold queue, we could > > still get batch efficiency at the network / consumer / downstream DB call > > layers. > > 2) On the server side, switching between fetching messages for the > > different queues seemed to be expensive. I'm not sure if that was due to > an > > inefficient scheduler, lack of memory, or poor I/O management. I suspect > > Kafka wouldn't hit this as long as the messages were present in the page > > cache, but it's just something to keep in mind--how this is implemented > > matters from a performance/starvation standpoint. > > > > So from a design standpoint, I think that means that for a priority > > queueing design to minimize starvation, the design criteria should > probably > > be "returning messages based on priority, but be sure to also keep the > > consumer fully occupied" > > > > If done right, this would make our lives much easier operationally (only > > one consumer group to manage, not two) and make our consumer usage more > > efficient. > > > > > > On Thu, Jan 17, 2019 at 4:20 AM Tobias Adamson <tob...@stargazer.com.sg> > > wrote: > > > > > Use cases: prioritise current data > > > > > > When processing messages sometimes there is a need to re process old > > data. > > > It would be nice to be abled to send the old data as messages to a > > > separate topic and that would only be processed when the current topic > > > doesn’t have any messages left to process. > > > This would prevent customers getting delays in current data processing > > due > > > to message processors being busy processing old data. > > > > > > > > > > On 17 Jan 2019, at 7:55 PM, Tim Ward <tim.w...@origamienergy.com> > > wrote: > > > > > > > > Use cases: processing alerts. > > > > > > > > High priority alerts ("a large chunk of your system has stopped > > > providing service, immediate action essential") should be processed > > before > > > low priority alerts ("some minor component has put out a not-very > serious > > > warning, somebody should probably have a look at it when they get > > bored"), > > > of which there could be a long queue. > > > > > > > > Urgent alerts (a phone call telling someone "you need to do this > now") > > > should be processed before non-urgent alerts (a phone call telling > > someone > > > "FYI, such and such is going to happen in a couple of hours"). > > > > > > > > Tim Ward > > > > > > > > -----Original Message----- > > > > From: n...@afshartous.com <n...@afshartous.com> > > > > Sent: 17 January 2019 02:52 > > > > To: users@kafka.apache.org > > > > Subject: Prioritized Topics for Kafka > > > > > > > > > > > > > > > > Hi all, > > > > > > > > On the dev list we’ve been discussing a proposed new feature > > > (prioritized topics). In a nutshell, when consuming from a set of > topics > > > with assigned priorities, consumption from lower-priority topics only > > > occurs if there’s no data flowing in from a higher-priority topic. > > > > > > > > > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-349%3A+Priorities+for+Source+Topics > > > < > > > > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-349:+Priorities+for+Source+Topics > > > > > > > > > > > > One question is are there use-cases for the proposed API. If you > think > > > this would be useful and have use-cases in mind please reply with the > > > use-cases. > > > > > > > > Its also possible to implement prioritization with the existing API > by > > > using a combination of pausing, resuming, and local buffering. The > > > question is then does it make sense to introduce the proposed > > higher-level > > > API to make this easier ? > > > > > > > > The responses will be used as input to determine if we move ahead > with > > > the proposal. Thanks in advance for input. > > > > > > > > Cheers, > > > > -- > > > > Nick > > > > > > > > The contents of this email and any attachment are confidential to the > > > intended recipient(s). If you are not an intended recipient: (i) do not > > > use, disclose, distribute, copy or publish this email or its contents; > > (ii) > > > please contact the sender immediately; and (iii) delete this email. Our > > > privacy policy is available here: > > > https://origamienergy.com/privacy-policy/. Origami Energy Limited > > > (company number 8619644); Origami Storage Limited (company number > > 10436515) > > > and OSSPV001 Limited (company number 10933403), each registered in > > England > > > and each with a registered office at: Ashcombe Court, Woolsack Way, > > > Godalming, GU7 1LQ. > > > > > > > > > > -- > > > > *Jeff Widman* > > jeffwidman.com <http://www.jeffwidman.com/> | 740-WIDMAN-J (943-6265) > > <>< > > >