Hey Tommy,
Yeah. That totally makes sense. Thanks for explaining it.  :)

Thanks!
Navina

On Fri, Sep 16, 2016 at 12:12 PM, Tommy Becker <tobec...@tivo.com> wrote:

> Hey Navina,
>
> This was consistently reproducible both locally and in our integration
> test environment. We have auto.create.topics.enable on our brokers (or more
> accurately, we do not have it disabled; it's the default). I did not mean
> to imply there is a problem with the logic of the change in SAMZA-971; I
> understand the desire to make fewer calls, but at the time I did not have
> time to dig in and see exactly what the root cause of the difference was. I
> think I've found it now though.
>
> Prior to the 971 fix, we eventually wind up in
> KafkaSystemAdmin.getTopicsAndPartitionsByBroker(), which contains this
> code:
>
> KafkaUtil.maybeThrowException(topicMetadata.errorCode)
>
> What I found was that this was indeed throwing a
> LeaderNotAvailableException in the case where the topic did not already
> exist. This has the effect of triggering a retry in
> KafkaSystemAdmin.getSystemStreamMetadata(), and this continues until the
> broker has finished creating the topic and returns the correct partition
> metadata. The optimized path introduced by the SAMZA-971 fix goes into
> KafkaSystemAdmin.getSystemStreamPartitionCounts() which does not check
> this errorCode, and simply returns an empty set of partitions. Does that
> make sense?
>
>
> -Tommy
>
>
>
>
>
>
> On 09/15/2016 09:54 PM, Navina Ramesh wrote:
>
> Hi Tommy,
>
> Yi and I discussed about it and initially, we thought it could have
> something to do with the topic auto-creation setting on your Kafka server.
> Is it enabled or disabled in your case?
>
> I kind of suspect that the request timeout is insufficient. However, we do
> have retries on Samza to fetch the metadata. So, even if topic does get
> auto-created and metadata fetch is delayed, it will try to fetch the
> metadata again. Not very clear why SAMZA-971 has anything to do with this.
> That JIRA just reduces the number of calls we make to the broker.
>
> Another question, are you able to reproduce this issue ?
>
> Thanks!
> Navina
>
> On Wed, Sep 14, 2016 at 1:33 PM, Tommy Becker <tobec...@tivo.com><mailto:
> tobec...@tivo.com> wrote:
>
>
>
> Thanks for the response, and done.
>
> https://issues.apache.org/jira/browse/SAMZA-1018
>
> On 09/14/2016 01:14 PM, Yi Pan wrote:
>
> Hi, Tommy,
>
> Could you open a JIRA for this one? Also, could you include the Kafka
> broker version in this test?
>
> Thanks!
>
> -Yi
>
> On Wed, Sep 14, 2016 at 6:06 AM, Tommy Becker <tobec...@tivo.com><mailto:
> tobec...@tivo.com><mailto:
>
> tobec...@tivo.com><mailto:tobec...@tivo.com> wrote:
>
>
>
> We are testing an upgrade to 0.10.1 from 0.9.1 and noticed a regression.
> When starting a stream job that consumes a topic that does not yet exist,
> the job dies with the following exception:
>
> Exception in thread "main" java.lang.IllegalArgumentException: No tasks
> found. Likely due to no input partitions. Can't run a job with no tasks.
>      at org.apache.samza.container.grouper.task.GroupByContainerCoun
> t.validateTasks(GroupByContainerCount.java:193)
>      at org.apache.samza.container.grouper.task.GroupByContainerCoun
> t.balance(GroupByContainerCount.java:86)
>      at org.apache.samza.coordinator.JobModelManager$.refreshJobMode
> l(JobCoordinator.scala:278)
>      at org.apache.samza.coordinator.JobModelManager$.jobModelGenera
> tor$1(JobCoordinator.scala:211)
>      at org.apache.samza.coordinator.JobModelManager$.initializeJobM
> odel(JobCoordinator.scala:217)
>      at org.apache.samza.coordinator.JobModelManager$.getJobCoordina
> tor(JobCoordinator.scala:122)
>      at org.apache.samza.coordinator.JobModelManager$.apply(JobCoord
> inator.scala:106)
>      at org.apache.samza.coordinator.JobModelManager$.apply(JobCoord
> inator.scala:112)
>      at org.apache.samza.job.local.ThreadJobFactory.getJob(ThreadJob
> Factory.scala:40)
>      at org.apache.samza.job.JobRunner.run(JobRunner.scala:129)
>      at org.apache.samza.job.JobRunner$.main(JobRunner.scala:66)
>      at org.apache.samza.job.JobRunner.main(JobRunner.scala)
>
>
>
>
>
> The root cause seems to be commit 920f803a2e3dab809f4d7bb518259b0f4164407f
> from SAMZA-971. From what I can tell passing partitionsMetadataOnly = true
> to the StreamMetadataCache in JobModelManager#getInputStreamPartitions is
> what's causing this this behavior. The input topic is still created, but
> the proper partition metadata is not returned, resulting in an empty set
> being returned. The behavior of Kafka here is screwy, but this still seems
> like a regression. The old behavior is nice because it doesn't require that
> producer systems come up before the stream processors.
>
> --
> Tommy Becker
> Senior Software Engineer
>
> Digitalsmiths
> A TiVo Company
>
> www.digitalsmiths.com<http://www.digitalsmiths.com><http://w
> ww.digitalsmiths.com><http://www.digitalsmiths.com><http://w
> ww.digitalsmiths.com><http://www.digitalsmiths.com><http://w
> ww.digitalsmiths.com><http://www.digitalsmiths.com>
> tobec...@tivo.com<mailto:tobec...@tivo.com><mailto:tobec...@tivo.com
> ><mailto:tobec...@tivo.com><mailto:tobec...@tivo.com
>
>
>
> <mailto:tobec...@tivo.com><mailto:tobec...@tivo.com>
>
>
>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
>
>
>
>
>
>
> --
> Tommy Becker
> Senior Software Engineer
>
> Digitalsmiths
> A TiVo Company
>
> www.digitalsmiths.com<http://www.digitalsmiths.com><http://w
> ww.digitalsmiths.com><http://www.digitalsmiths.com>
> tobec...@tivo.com<mailto:tobec...@tivo.com><mailto:tobec...@tivo.com
> ><mailto:tobec...@tivo.com>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
>
>
>
>
>
>
>
>
> --
> Tommy Becker
> Senior Software Engineer
>
> Digitalsmiths
> A TiVo Company
>
> www.digitalsmiths.com<http://www.digitalsmiths.com>
> tobec...@tivo.com<mailto:tobec...@tivo.com>
>
> ________________________________
>
> This email and any attachments may contain confidential and privileged
> material for the sole use of the intended recipient. Any review, copying,
> or distribution of this email (or any attachments) by others is prohibited.
> If you are not the intended recipient, please contact the sender
> immediately and permanently delete this email and any attachments. No
> employee or agent of TiVo Inc. is authorized to conclude any binding
> agreement on behalf of TiVo Inc. by email. Binding agreements with TiVo
> Inc. may only be made by a signed written agreement.
>



-- 
Navina R.

Reply via email to