Problem: upgrade 1.2 to 1.3 caused loss of clean shutdown on SIGTERM

2020-01-06 Thread Thunder Stumpges
We are attempting to upgrade from samza 1.2 to 1.3 in hopes of fixing https://issues.apache.org/jira/browse/SAMZA-2198 where there was a deadlock in the shutdown code which prevented completing a clean shutdown. After the upgrade, it appears like NONE of the shutdown hooks / code are being called

Problem : upgrade 1.2 to 1.3 LocalApplicationRunner + default system in SamzaApplication.describe results in null MetadataStoreFactory

2020-01-06 Thread Thunder Stumpges
Hey dev team. Just upgrading our stand alone low-level tasks to Samza 1.3. We use the LocalApplicationRunner and initialize most of our application within SamzaApplication.describe() including setting up "withDefaultSystem(systemDescriptor)" However it seems that earlier on in the process, the

Re: [DISCUSS] SEP-26: Add SystemProducer for Azure Blob Storage

2020-01-06 Thread Lakshmi Manasa
Thank you Cameron for the suggestion. I have added info about when the System Producer accepts a byte array and how to manage schemas. Please let me know if there are any more suggestions. Thanks, Manasa On Mon, Jan 6, 2020 at 10:38 AM Cameron Lee wrote: > Thanks for the proposal. > I just have

Re: [DISCUSS] SEP-22: Container Placements in Samza

2020-01-06 Thread Sanil Jain
Hi all, Refreshing this back to see if there is any further feedback on this SEP. Thanks Sanil On Thu, 5 Dec 2019 at 16:25, Sanil Jain wrote: > Hi all, > > I have created SEP-22: Container Placements in Samza, which adds abilities > to move containers for a Samza job seamlessly from one host t

Re: [DISCUSS] SEP-24: Cluster-based Job Coordinator Dependency Isolation

2020-01-06 Thread Cameron Lee
Hi all, I am just refreshing this thread to check if there is any further feedback on this SEP. Thank you, Cameron On Fri, Nov 22, 2019 at 11:24 AM Cameron Lee wrote: > Hi all, > We created SEP-24: Cluster-based Job Coordinator Dependency Isolation. > Please find the SEP wiki here ( > https://cw

Re: [DISCUSS] SEP-26: Add SystemProducer for Azure Blob Storage

2020-01-06 Thread Cameron Lee
Thanks for the proposal. I just have one comment: It looks like part of the API for the new system producer is that the OutgoingMessageEnvelope.message can be an Avro record or a byte array, depending on how the user wants to manage schemas. Can you please add some detail to the SEP about that part

[RESULT][VOTE] SEP-25: PR Title And Description Guidelines

2020-01-06 Thread Prateek Maheshwari
The vote on SEP-25 has been open for > 72 hours and has 4 binding and 4 non-binding +1s. SEP-25 is accepted. Thanks, Prateek

Re: [VOTE] SEP 25: PR Title and Description Guidelines

2020-01-06 Thread Prateek Maheshwari
The VOTE has been open for > 3 business days and has 3 binding +1s, so the proposal has been accepted. I'll send out a RESULT email. Thanks everyone for reviewing and voting. - Prateek On Fri, Dec 20, 2019 at 7:40 AM Yang Zhang wrote: > +1 (non-binding) > > On Thu, Dec 19, 2019 at 5:28 PM Jake

Re: [DISCUSS] SEP-26: Add SystemProducer for Azure Blob Storage

2020-01-06 Thread Prateek Maheshwari
Thanks for the proposal Manasa, it looks good to me. If there are no major questions or objections by EOD let's move to the VOTE. Thanks, Prateek On Wed, Dec 18, 2019 at 4:27 PM Lakshmi Manasa wrote: > Hi all, > > We created SEP-26: Add SystemProducer for Azure Blob Storage. > > Please find SEP

Re: Reduce kafka partition for a topic samza is using

2020-01-06 Thread Bharath Kumara Subramanian
Hi Debraj, Kafka doesn't support reducing the partition size and only supports increasing the partition size of a topic. One way to accomplish it would be to create a new topic with the desired partition count and reroute data from the old topic. Although, it will be good to first understand the u

Re: Reduce kafka partition for a topic samza is using

2020-01-06 Thread Debraj Manna
Anyone any thoughts on this? On Sat, Jan 4, 2020 at 5:16 PM Debraj Manna wrote: > I am using samza on yarn with Kafka. I need to reduce the number of > partitions in kafka. I am ok with some data loss. Can someone suggest what > should be the recommended way of doing this? > > Samza Job Config l