correction: task count remains the same (executor count can vary).



On 2/20/18, 11:20 AM, "Arun Iyer on behalf of Arun Mahadevan" 
<[email protected] on behalf of [email protected]> wrote:

>Hi Jungtaek,
>
>
>1. Right now users need to vary the database/table name in the state provider 
>config per topology. Agree, its better to include topology in the namespace.
>
>2. IMO, users should have the flexibility to store multiple key-values in the 
>state in the core API. As of now storm only supports rebalancing the tasks 
>(executor count remains the same). So users need not worry about re-sharding 
>their state as long as they use the right grouping. I think this flexibility 
>also helps us to provide useful abstractions on top.
>
>3. Re-sharding would be based on the keys (re-hashing). The component id, 
>task-id would be used to map to the namespace. So I am not clear about the 
>concern you have raised.
>
>We could support dynamic state re-sharding via underlying higher level 
>abstractions (e.g. Streams API) where we control the namespaces, keys etc and 
>would be more manageable. IMO before we do this, we can introduce stateful 
>exactly once processing via the streams API as the first step.
>
>Thanks,
>Arun
>
>
>
>On 2/19/18, 4:59 PM, "Jungtaek Lim" <[email protected]> wrote:
>
>>Hi,
>>
>>I'd like to verify my observation on current State implementation is
>>correct, so that we could fix them if necessary and make plan for
>>improvement.
>>
>>1. State is stored with namespace prefix which typically composes to
>>(component id, task id) pair and it doesn't look like having classification
>>for topology. Is this correct observation? If then I think that's worth to
>>call it as 'critical' and it must be fixed.
>>
>>2. We're allowing end-users to put key of state, and also no restriction
>>for grouping on stateful component. I feel such flexibility breaks the
>>possibility to reshard state and end-users are required to implement their
>>own reshard tool according to their topology state key distribution logic.
>>I expect it will not happen on streams API (since it should be done with
>>keyed stream) but wouldn't it better to also restrict such flexibility also
>>for core API?
>>
>>3. Suppose we are going to support state resharding (for allowing change of
>>parallelism) and we restrict to apply field grouping with key while
>>connecting stateful component.
>>Then key-value can be moved based on key (though finding and replacing task
>>id may not be trivial if component name has '-'... we have same issue on
>>metric name, so maybe time to restrict characters on topology name as well
>>as component name?).
>>Is it also true for window/partition/windowsystem state? I didn't take a
>>deep look on window state (I would find a time) but it would be great if
>>someone knowing the detail makes it clear.
>>
>>Thanks in advance,
>>Jungtaek Lim (HeartSaVioR)
>

Reply via email to