simingweng opened a new issue #3210: DistributedLog based stateful storage 
backend is not working
URL: https://github.com/apache/incubator-heron/issues/3210
 
 
   when we switch to `org.apache.heron.statefulstorage.dlog.DlogStorage` as the 
stateful topology state storage backend, the checkpointing manager is always 
throwing exception when it tries to save state for any heron instance into 
Bookkeeper cluster via the DistributedLog API.
   
   We're testing using Heron v0.20.1-incubating-rc1, deployed on Kubernetes 
cluster, with a StatefulSet of 3 Bookkeeper pods and a StatefulSet of 3 
Zookeeper pods.
   
   The exception we got in the checkpointing manager log is:
   
   `
   [2019-03-10 20:05:59 +0000] [WARNING] 
org.apache.heron.ckptmgr.CheckpointManagerServer: Save checkpoint not 
successful for checkpointId 1552074656671682557-1552248359 component sum-bolt 
instanceId 0 
   org.apache.heron.spi.statefulstorage.StatefulStorageException: Failed to 
persist checkpoint @ 
/arithmetic-progression-sum/1552074656671682557-1552248359/sum-bolt_0
        at 
org.apache.heron.statefulstorage.dlog.DlogStorage.storeCheckpoint(DlogStorage.java:148)
        at 
org.apache.heron.ckptmgr.CheckpointManagerServer.handleSaveInstanceStateRequest(CheckpointManagerServer.java:248)
        at 
org.apache.heron.ckptmgr.CheckpointManagerServer.onRequest(CheckpointManagerServer.java:108)
        at 
org.apache.heron.common.network.HeronServer.handlePacket(HeronServer.java:212)
        at 
org.apache.heron.common.network.HeronServer.handleRead(HeronServer.java:169)
        at 
org.apache.heron.common.basics.NIOLooper.handleSelectedKeys(NIOLooper.java:116)
        at 
org.apache.heron.common.basics.NIOLooper.access$000(NIOLooper.java:38)
        at org.apache.heron.common.basics.NIOLooper$1.run(NIOLooper.java:51)
        at 
org.apache.heron.common.basics.WakeableLooper.executeTasksOnWakeup(WakeableLooper.java:191)
        at 
org.apache.heron.common.basics.WakeableLooper.runOnce(WakeableLooper.java:110)
        at 
org.apache.heron.common.basics.WakeableLooper.loop(WakeableLooper.java:100)
        at 
org.apache.heron.ckptmgr.CheckpointManager.startAndLoop(CheckpointManager.java:175)
        at 
org.apache.heron.ckptmgr.CheckpointManager.main(CheckpointManager.java:263)
   Caused by: org.apache.distributedlog.exceptions.ZKException: Failed to 
create log 
/distributedlog/arithmetic-progression-sum/1552074656671682557-1552248359/sum-bolt_0/<default>
 : NONODE
        at 
org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore$3.processResult(ZKLogStreamMetadataStore.java:471)
        at 
org.apache.bookkeeper.zookeeper.ZooKeeperClient$3$1.processResult(ZooKeeperClient.java:472)
        at 
dlshade.org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:658)
        at 
dlshade.org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:499)
   `
   
   The cause of the issue is that when `DlogStorage` tries to open a new stream 
for writing each checkpoint data, the stream name is built in such a pattern:
   
   /{topology name}/{checkpoint id}/{component name}_{task id}
   
   the backslash in the stream name is interpreted as namespace path by 
DistributedLog, of course, the namespace is never created yet, therefore, the 
"NONODE" exception is thrown.
   
   The same DistributedLog behavior can be reproduced by just using the `dlog` 
CLI coming with the DistributedLog distribution. Using `dlog tool create -u 
{any existing namespace} -r {any string contains backslash} -e {1}` will cause 
exact the same exception.
   
   So, in short, `DlogStorage` shall not use hierarchical DistributedLog 
namespaces to organize checkpoints, because namespace is not dynamically 
created.
   
   The solution could be changing the stream name of each checkpoint to 
{topology name}-{checkpoint id}-{component name}_{task id}, and the stream will 
be created under the existing namespace `distributedlog://{zookeeper 
service}/distributedlog` which exists by default and is being used by 
`DLUploader` successfully now.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to