I found what the problem is: I'm attempting to run a ThreadJobFactory for local testing, and looking at the config for the wikipedia-application-local-runner sample application and then the Samza config documentation, I found a footnote stating that "For non-cluster applications (ones using coordination service) one must use org.apache.samza.container.grouper.task.GroupByContainerIdsFactory". I added that in and everything is starting up smoothly.
Cheers, Malcolm McFarland Cavulus This correspondence is from HealthPlanCRM, LLC, d/b/a Cavulus. Any unauthorized or improper disclosure, copying, distribution, or use of the contents of this message is prohibited. The information contained in this message is intended only for the personal and confidential use of the recipient(s) named above. If you have received this message in error, please notify the sender immediately and delete the original message. Malcolm McFarland Cavulus This correspondence is from HealthPlanCRM, LLC, d/b/a Cavulus. Any unauthorized or improper disclosure, copying, distribution, or use of the contents of this message is prohibited. The information contained in this message is intended only for the personal and confidential use of the recipient(s) named above. If you have received this message in error, please notify the sender immediately and delete the original message. On Fri, Dec 18, 2020 at 9:34 AM Malcolm McFarland <mmcfarl...@cavulus.com> wrote: > Hi Bharath, > > I've been looking around in the logs trying to figure out which bits > you're asking for. At the risk of spamming the list, here are some excerpts > that I _think_ meet the criteria: > > 2020-12-17 17:04:11.455 [main] ZkUtils [INFO] Current version for zk root > node: /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData is 1.0, expected > version is 1.0 > 2020-12-17 17:04:11.457 [main] ZkClient [INFO] Waiting for keeper state > SyncConnected > 2020-12-17 17:04:11.458 [main] ZkUtils [INFO] Created ephemeral path: > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/processors/0000000010 > for processor: ip-192-168-100-105.us-west-2.compute.internal > 81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3 in zookeeper. > 2020-12-17 17:04:11.460 [main] ZkUtils [INFO] Found these children - > [0000000010] > 2020-12-17 17:04:11.461 [main] ZkUtils [INFO] Found these children - > [0000000010] > 2020-12-17 17:04:11.461 [main] ZkLeaderElector [INFO] tryBecomeLeader: > index = 0 for > path=/app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/processors/0000000010 > out of [0000000010] > 2020-12-17 17:04:11.461 [main] ZkLeaderElector [INFO] > [Processor-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] Eligible to become the > leader! > 2020-12-17 17:04:11.461 [main] ZkJobCoordinator [INFO] > ZkJobCoordinator::onBecomeLeader - I became the leader > 2020-12-17 17:04:11.462 [main] ZkUtils [INFO] Subscribing for child change > at:/app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/processors > 2020-12-17 17:04:11.488 [main] Metadata [INFO] Cluster ID: > ID7elx7aRYGCt7ZTdhWHvw > 2020-12-17 17:04:11.497 [main] KafkaSystemAdmin [INFO] SystemStream > partition counts for system kafka: { ..stream info.. } > 2020-12-17 17:04:11.507 [main] ScheduleAfterDebounceTime [INFO] Trying to > cancel the action: OnProcessorChange. > 2020-12-17 17:04:11.507 [main] ScheduleAfterDebounceTime [INFO] Scheduled > action: OnProcessorChange to run after: 20000 milliseconds. > 2020-12-17 17:04:11.507 [main] ZkUtils [INFO] subscribing for jm version > change > at:/app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelVersion > > 2020-12-17 17:04:32.279 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > pid=81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3Generated new JobModel with > version: 11 and processors: [81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] > 2020-12-17 17:04:32.289 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkBarrierForVersionUpgrade > [INFO] Creating barrier with version: 11, participants: > [81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3]. > 2020-12-17 17:04:32.291 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkBarrierForVersionUpgrade > [INFO] Marking the barrier state: > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelUpgradeBarrier/versionBarriers/barrier_11/barrier_state > as NEW. > 2020-12-17 17:04:32.292 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkBarrierForVersionUpgrade > [INFO] Subscribing child changes on the path: > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelUpgradeBarrier/versionBarriers/barrier_11/barrier_participants > for barrier version: 11. > 2020-12-17 17:04:32.293 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Trying to cancel the action: BarrierAction. > 2020-12-17 17:04:32.294 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Scheduled action: BarrierAction to run after: 40000 milliseconds. > 2020-12-17 17:04:32.294 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] publishing new > version: 11; oldVersion = 10(10) > 2020-12-17 17:04:32.295 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] published new > version: 11; expected data version = 11(actual data version after update = > 11) > 2020-12-17 17:04:32.295 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > pid=81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3Published new Job Model. Version = > 11 > 2020-12-17 17:04:32.295 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Trying to cancel the action: OnCleanUp. > 2020-12-17 17:04:32.296 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Scheduled action: OnCleanUp to run after: 0 milliseconds. > 2020-12-17 17:04:32.296 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Action: OnProcessorChange completed successfully. > 2020-12-17 17:04:32.296 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] About to delete > old barrier paths from > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelUpgradeBarrier/versionBarriers > 2020-12-17 17:04:32.296 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] List of all > zkNodes: [barrier_2, barrier_3, barrier_11, barrier_1, barrier_6, > barrier_7, barrier_4, barrier_5, barrier_8, barrier_9, barrier_10] > 2020-12-17 17:04:32.296 [ZkClient-EventThread-22-localhost:2181] > ScheduleAfterDebounceTime [INFO] Trying to cancel the action: > JobModelVersionChange. > 2020-12-17 17:04:32.296 [ZkClient-EventThread-22-localhost:2181] > ScheduleAfterDebounceTime [INFO] Scheduled action: JobModelVersionChange to > run after: 0 milliseconds. > 2020-12-17 17:04:32.297 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] Starting > cleanup of barrier version zkNodes. From size=11 to size 1; numberToLeave=10 > 2020-12-17 17:04:32.297 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] deleting > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelUpgradeBarrier/versionBarriers/barrier_1 > 2020-12-17 17:04:32.299 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] About to delete > jm > path=/app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModels > 2020-12-17 17:04:32.300 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] Starting > cleanup of barrier version zkNodes. From size=11 to size 1; numberToLeave=10 > 2020-12-17 17:04:32.300 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkUtils [INFO] deleting > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModels/1 > 2020-12-17 17:04:32.302 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Action: OnCleanUp completed successfully. > 2020-12-17 17:04:32.302 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] Got a > notification for new JobModel version. Path = > /app-rpc-runner-1/rpc-runner-1-2.0-coordinationData/jobModelGeneration/jobModelVersion > Version = 11 > 2020-12-17 17:04:32.305 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > pid=81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3: new JobModel is available. > Version =11; JobModel = JobModel [config={}, containers={0=ContainerModel > [id=0, tasks={SystemStreamPartition [kafka, topic1, 5]=TaskModel > [taskName=SystemStreamPartition [kafka, topic1, 5], > systemStreamPartitions=[SystemStreamPartition [kafka, topic1, 5], .. }] > 2020-12-17 17:04:32.305 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] New > JobModel does not contain pid=81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3. > Stopping this processor. New JobModel: JobModel [config={}, > containers={0=ContainerModel [id=0, tasks={SystemStreamPartition [kafka, > topic1, 5]=TaskModel [taskName=SystemStreamPartition [kafka, topic1, 5], > systemStreamPartitions=[SystemStreamPartition [kafka, topic1, 5], .. }] > 2020-12-17 17:04:32.305 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > Shutting down JobCoordinator. > 2020-12-17 17:04:32.306 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] StreamProcessor [INFO] Job > model expired. Shutting down the container: null of stream processor: > 81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3. > 2020-12-17 17:04:32.306 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] StreamProcessor [INFO] > Container: null shutdown completed for stream processor: > 81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3. > 2020-12-17 17:04:32.306 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ScheduleAfterDebounceTime > [INFO] Shutting down debounce timer! > 2020-12-17 17:04:32.306 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > Resigning leadership for processorId: 81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3 > 2020-12-17 17:04:32.306 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZkJobCoordinator [INFO] > Shutting down ZkUtils. > 2020-12-17 17:04:32.306 [ZkClient-EventThread-22-localhost:2181] > ZkEventThread [INFO] Terminate ZkClient event thread. > 2020-12-17 17:04:32.307 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] ZooKeeper [INFO] Session: > 0x100008fae2900a0 closed > 2020-12-17 17:04:32.307 [main-EventThread] ClientCnxn [INFO] EventThread > shut down for session: 0x100008fae2900a0 > 2020-12-17 17:04:32.309 [Samza Debounce > Thread-81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3] StreamProcessor [INFO] > Shutting down the executor service of the stream processor: > 81ac6a4e-3d5e-479c-9a6c-2f2d9b4372d3. > > Does this help? > > Cheers, > Malcolm McFarland > Cavulus > > This correspondence is from HealthPlanCRM, LLC, d/b/a Cavulus. Any > unauthorized or improper disclosure, copying, distribution, or use of the > contents of this message is prohibited. The information contained in this > message is intended only for the personal and confidential use of the > recipient(s) named above. If you have received this message in error, > please notify the sender immediately and delete the original message. > > Malcolm McFarland > Cavulus > > > This correspondence is from HealthPlanCRM, LLC, d/b/a Cavulus. Any > unauthorized or improper disclosure, copying, distribution, or use of the > contents of this message is prohibited. The information contained in this > message is intended only for the personal and confidential use of the > recipient(s) named above. If you have received this message in error, > please notify the sender immediately and delete the original message. > > > On Mon, Dec 14, 2020 at 10:51 AM Bharath Kumara Subramanian < > codin.mart...@gmail.com> wrote: > >> Hi Malcolm, >> >> Based on the following log >> >> INFO [org.apache.samza.zk.ZkJobCoordinator] New JobModel does not contain >> > pid=a3e86ddf-8d18-40c9-8063-1efd588cec56. Stopping this processor. New >> > JobModel: JobModel [..] >> > >> >> I'd have to guess that the processor isn't part of the quorum (list of >> processors) that was used by the leader to generate the job model in the >> first place and hence it is expected to ignore the job model change and >> shut itself down. >> >> I'd suggest >> >> 1. Take a pass at whether this processor is part of the quorum and what >> happened to its membership. >> 2. Take a pass at the leader's log to get some insights into what set >> of >> processors it started out with when generating the job model. >> >> We will need more details to investigate the issue. If you can attach the >> failed processor and leader logs, I can take a stab at it. >> >> Thanks, >> Bharath >> >> >> On Mon, Dec 14, 2020 at 10:05 AM Malcolm McFarland < >> mmcfarl...@cavulus.com> >> wrote: >> >> > Hey all, >> > >> > We have an app that's been running on v0.14.1 for the last few years, >> and >> > we're trying to drag it forward into the present with v1.5.1. I've >> tried a >> > few different approaches at updating it, including creating a >> > TaskApplication via the low-level API and also following the "Legacy >> > Applications" deploy instructions. Thus far, the legacy approach seems >> most >> > promising, but the application isn't fully starting up. It _seems_ to >> be an >> > issue with creating the JobModel; although there are no explicit >> errors, I >> > do see these log messages: >> > >> > INFO [org.apache.samza.zk.ZkJobCoordinator] Got a notification for new >> > JobModel version. Path = .. >> > INFO [org.apache.samza.zk.ZkJobCoordinator] >> > pid=a3e86ddf-8d18-40c9-8063-1efd588cec56: new JobModel is available. >> > Version =9; JobModel = JobModel [..] >> > INFO [org.apache.samza.zk.ZkJobCoordinator] New JobModel does not >> contain >> > pid=a3e86ddf-8d18-40c9-8063-1efd588cec56. Stopping this processor. New >> > JobModel: JobModel [..] >> > >> > At this point the ThreadJob shuts down cleanly. Afaict, the legacy >> > configuration is set up correctly, and mirrors our functional build >> under >> > 0.14.1. Any thoughts? >> > >> > Cheers, >> > Malcolm McFarland >> > Cavulus >> > >> > >> > This correspondence is from HealthPlanCRM, LLC, d/b/a Cavulus. Any >> > unauthorized or improper disclosure, copying, distribution, or use of >> the >> > contents of this message is prohibited. The information contained in >> this >> > message is intended only for the personal and confidential use of the >> > recipient(s) named above. If you have received this message in error, >> > please notify the sender immediately and delete the original message. >> > >> >