Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

...

JobModel is the data model in samza that logically represents a samza job. The JobModel hierarchy is that samza jobs have one to many containers, and each container has one to many tasks. Each data model contains relevant information, such as logical id, partition information, etc. In the standalone deployment model, the JobModel is stored in zookeeper. Coordinator stream(kafka topic) is used to store the JobModel in the yarn deployment model.

...

ZK Data Model to support host affinity:

Before resuming the event processing after an rebalancing phase, each stream processor will register the details of physical host on which it runs in the localityData zookeeper node. The goal here is to separate the locality information from the JobModel itself (In standalone, locality information of the stream processors will be stored seperately from the JobModel. JobModel will be used to hold the task assignments(processor to task assignment and task to system stream partition assignment). Each stream processor during it's startup phase will store the physical host on which it runs from into an appropriate localityData zookeeper node. MetadataStore abstraction will be used to read and write stream processor locality information for different deployment models in appropriate storage layers. There will be two implementations of MetadataStore viz CoordinatorStreamBasedMetadataStore to read/write container processor locality information into coordinator stream(a kafka topic) for yarn and ZkMetadataStore to read/write container processor locality information in zookeeper for standalone. In case of standalone, last known physical host in which each  samza task had run will be stored in zookeeper, which will then be used to assign tasks to stream processors. Stream processor will update the task locality of the tasks assigned to it before it begins processing(This is synonymous to behaviour in yarn, where locality is updated in SamzaContainer as a part of startup sequence). Local state of the tasks will be persisted in a directory(local.store.dir) provided through configuration by each processor.

...