[ 
https://issues.apache.org/jira/browse/FLINK-18091?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17126769#comment-17126769
 ] 

Congxian Qiu(klion26) edited comment on FLINK-18091 at 6/5/20, 1:43 PM:
------------------------------------------------------------------------

Test on a real cluster, Result excepted.
 * savepoint relocate can restore successfully
 * checkpoint relocate will be failed with FileNotFoundException

commit : 8ca67c962a5575aba3a43b8cfd4a79ffc8069bd4

The Long log attached below:

{{_username/ip/port and other sensitive information has been masked._}}
 # For Savepoint

 
{code:java}
[~/flink-1.11-SNAPSHOT]$ ./bin/flink savepoint 9bcc2546a841b36a39c46fbe13a2b631 
hdfs:///user/xxxxxx/congxianqiu/savepoint -yid application_1591259429117_0007
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in 
[jar:file:/data/work/congxianqiu/flink-1.11-SNAPSHOT/lib/log4j-slf4j-impl-2.12.1.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in 
[jar:file:/data/xxxxxx/hadoop/hadoop-2.7.4/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
2020-06-05 20:27:43,039 WARN  
org.apache.flink.yarn.configuration.YarnLogConfigUtil        [] - The 
configuration directory ('/data/work/congxianqiu/flink-1.11-SNAPSHOT/conf') 
already contains a LOG4J config file.If you want to use logback, then please 
delete or rename the log configuration file.
2020-06-05 20:27:43,422 INFO  org.apache.flink.yarn.YarnClusterDescriptor       
           [] - No path for the flink jar passed. Using the location of class 
org.apache.flink.yarn.YarnClusterDescriptor to locate the jar
2020-06-05 20:27:43,513 INFO  org.apache.flink.yarn.YarnClusterDescriptor       
           [] - Found Web Interface 10-215-128-84:35572 of application 
'application_1591259429117_0007'.
Triggering savepoint for job 9bcc2546a841b36a39c46fbe13a2b631.
Waiting for response...
Savepoint completed. Path: 
hdfs://ip:port/user/xxxxxx/congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
You can resume your program from this savepoint with the run command.

[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint
Found 1 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
[~/congxianqiu/flink-1.11-SNAPSHOT]$ hadoop fs -ls 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
Found 2 items
-rw-r--r--   3 xxxxxx supergroup      74170 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33/6508ac9e-0d2a-4583-96ad-1d67fb5b1c8a
-rw-r--r--   3 xxxxxx supergroup       1205 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33/_metadata



[~/flink-1.11-SNAPSHOT]$ hadoop fs -mv 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33 
congxianqiu/savepoint/newsavepointpath
[ ~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint
Found 1 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint/newsavepointpath
Found 2 items
-rw-r--r--   3 xxxxxx supergroup      74170 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath/6508ac9e-0d2a-4583-96ad-1d67fb5b1c8a
-rw-r--r--   3 xxxxxx supergroup       1205 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath/_metadata


[~/flink-1.11-SNAPSHOT]$ ./bin/flink run -s 
hdfs:///user/xxxxxx/congxianqiu/newsavepointpath/_metadata -m yarn-cluster -c 
com.klion26.data.FlinkDemo 
/data/work/congxianqiu/flink-1.11-SNAPSHOT/ft_local/Flink-Demo-1.0-SNAPSHOT.jar
SLF4J: Class path contains multiple SLF4J bindings.


>>>>>> jobmanager.log
restored succesfully, can do checkpoint successfully.
2020-06-05 21:11:10,053 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Starting job 
b2fbfa6527391f035e8eebd791c2f64e from savepoint 
hdfs:///user/xxxxxx/congxianqiu/savepoint/newsavepointpath/_metadata ()
2020-06-05 21:11:10,198 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Reset the 
checkpoint ID of job b2fbfa6527391f035e8eebd791c2f64e to 3.
2020-06-05 21:11:10,198 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Restoring job 
b2fbfa6527391f035e8eebd791c2f64e from latest valid checkpoint: Checkpoint 2 @ 0 
for b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:11:10,206 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - No master 
state to restore
.......
2020-06-05 21:11:16,117 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Checkpoint 
triggering task Source: Custom Source (1/1) of job 
b2fbfa6527391f035e8eebd791c2f64e is not in state RUNNING but SCHEDULED instead. 
Aborting checkpoint.
2020-06-05 21:11:19,456 INFO  org.apache.flink.yarn.YarnResourceManager         
           [] - Registering TaskManager with ResourceID 
container_e18_1591259429117_0019_01_000002 
(akka.tcp://flink@10-215-128-83:56603/user/rpc/taskmanager_0) at ResourceManager
2020-06-05 21:11:19,566 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (1/1) (bc206dbf0e964487e5ba8c4355cb691e) switched from SCHEDULED 
to DEPLOYING.
2020-06-05 21:11:19,566 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Deploying 
Source: Custom Source (1/1) (attempt #0) to 
container_e18_1591259429117_0019_01_000002 @ 10-215-128-83 (dataPort=45167)
2020-06-05 21:11:19,572 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (7ae861cf453455d722d5d4ece0c10d1a) switched from SCHEDULED to 
DEPLOYING.
2020-06-05 21:11:19,573 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Deploying Map 
-> Sink: Unnamed (1/1) (attempt #0) to 
container_e18_1591259429117_0019_01_000002 @ 10-215-128-83 (dataPort=45167)
2020-06-05 21:11:20,467 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (7ae861cf453455d722d5d4ece0c10d1a) switched from DEPLOYING to 
RUNNING.
2020-06-05 21:11:20,468 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (1/1) (bc206dbf0e964487e5ba8c4355cb691e) switched from DEPLOYING 
to RUNNING.
2020-06-05 21:12:16,199 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 3 @ 1591362736116 for job b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:12:16,854 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Completed 
checkpoint 3 for job b2fbfa6527391f035e8eebd791c2f64e (106237 bytes in 736 ms).
2020-06-05 21:13:16,172 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 4 @ 1591362796116 for job b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:13:16,680 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Completed 
checkpoint 4 for job b2fbfa6527391f035e8eebd791c2f64e (32823 bytes in 542 ms).
{code}
 

 

2. log for checkpoint
{code:java}
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/      
Found 3 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:15 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/chk-6
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:15 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/shared
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:11 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/taskowned


[ ~/flink-1.11-SNAPSHOT]$ hadoop fs -mv 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e 
congxianqiu/checkpoint/movecheckpoint
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/checkpoint/movecheckpoint
Found 3 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/chk-6
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/shared
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/taskowned



jobmanager.log
020-06-05 21:28:13,728 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (8784ec1b22ac1ca1dd8778505e434048)
 switched from DEPLOYING to RUNNING.
2020-06-05 21:28:13,745 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (8784ec1b22ac1ca1dd8778505e434048)
 switched from RUNNING to FAILED on 
org.apache.flink.runtime.jobmaster.slotpool.SingleLogicalSlot@1967324.
java.lang.Exception: Exception while creating StreamOperatorStateContext.
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:204)
 ~[flin
k-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:247)
 ~[flink-dist_2.11-1.11-SNAPSHOT.j
ar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.runtime.tasks.OperatorChain.initializeStateAndOpenOperators(OperatorChain.java:290)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar
:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$beforeInvoke$0(StreamTask.java:472)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
 ~[flink-dist_2.11-1.11-SNAPSHOT.
jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.beforeInvoke(StreamTask.java:468)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:521) 
~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:720) 
~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:545) 
~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at java.lang.Thread.run(Thread.java:748) ~[?:1.8.0_144]
Caused by: org.apache.flink.util.FlinkException: Could not restore keyed state 
backend for StreamMap_20ba6b65f97481d5570070de90e4e791_(1/1) from any of the 1
 provided restore options.
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:135)
 ~[flink-dist_2.11-1.11-SNAPS
HOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:317)
 ~[flink-dist_2
.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:144)
 ~[flin
k-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        ... 9 more
Caused by: org.apache.flink.runtime.state.BackendBuildingException: Caught 
unexpected exception.
        at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackendBuilder.build(RocksDBKeyedStateBackendBuilder.java:330)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.contrib.streaming.state.RocksDBStateBackend.createKeyedStateBackend(RocksDBStateBackend.java:535)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.lambda$keyedStatedBackend$1(StreamTaskStateInitializerImpl.java:301)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.attemptCreateAndRestore(BackendRestorerProcedure.java:142)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.BackendRestorerProcedure.createAndRestore(BackendRestorerProcedure.java:121)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:317)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        at 
org.apache.flink.streaming.api.operators.StreamTaskStateInitializerImpl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:144)
 ~[flink-dist_2.11-1.11-SNAPSHOT.jar:1.11-SNAPSHOT]
        ... 9 more
Caused by: org.apache.hadoop.ipc.RemoteException: File does not exist: 
/user/xxxxxx/congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/shared/56704ae2-d
04c-4073-aa6b-843a40e15bbe
        at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
        at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1836)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1808)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1723)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:588)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java
:366)
        at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2217)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2213)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2213)


        at org.apache.hadoop.ipc.Client.call(Client.java:1476) 
~[hadoop-common-2.7.4.jar:?]
        at org.apache.hadoop.ipc.Client.call(Client.java:1413) 
~[hadoop-common-2.7.4.jar:?]
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
 ~[hadoop-common-2.7.4.jar:?]
        at com.sun.proxy.$Proxy35.getBlockLocations(Unknown Source) ~[?:?]
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:255)
 ~[hadoop-hdfs-
2.7.4.jar:?]
{code}
 

 


was (Author: klion26):
Test on a real cluster, Result excepted.
 * savepoint relocate can restore successfully
 * checkpoint relocate will be failed with FileNotFoundException

commit : 8ca67c962a5575aba3a43b8cfd4a79ffc8069bd4

The Long log attached below:

{{_username/ip/port and other sensitive information has been masked._}}
 # For Savepoint

 
{code:java}
[~/flink-1.11-SNAPSHOT]$ ./bin/flink savepoint 9bcc2546a841b36a39c46fbe13a2b631 
hdfs:///user/xxxxxx/congxianqiu/savepoint -yid application_1591259429117_0007
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in 
[jar:file:/data/work/congxianqiu/flink-1.11-SNAPSHOT/lib/log4j-slf4j-impl-2.12.1.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in 
[jar:file:/data/xxxxxx/hadoop/hadoop-2.7.4/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation.
SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
2020-06-05 20:27:43,039 WARN  
org.apache.flink.yarn.configuration.YarnLogConfigUtil        [] - The 
configuration directory ('/data/work/congxianqiu/flink-1.11-SNAPSHOT/conf') 
already contains a LOG4J config file.If you want to use logback, then please 
delete or rename the log configuration file.
2020-06-05 20:27:43,422 INFO  org.apache.flink.yarn.YarnClusterDescriptor       
           [] - No path for the flink jar passed. Using the location of class 
org.apache.flink.yarn.YarnClusterDescriptor to locate the jar
2020-06-05 20:27:43,513 INFO  org.apache.flink.yarn.YarnClusterDescriptor       
           [] - Found Web Interface 10-215-128-84:35572 of application 
'application_1591259429117_0007'.
Triggering savepoint for job 9bcc2546a841b36a39c46fbe13a2b631.
Waiting for response...
Savepoint completed. Path: 
hdfs://ip:port/user/xxxxxx/congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
You can resume your program from this savepoint with the run command.

[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint
Found 1 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
[~/congxianqiu/flink-1.11-SNAPSHOT]$ hadoop fs -ls 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33
Found 2 items
-rw-r--r--   3 xxxxxx supergroup      74170 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33/6508ac9e-0d2a-4583-96ad-1d67fb5b1c8a
-rw-r--r--   3 xxxxxx supergroup       1205 2020-06-05 20:27 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33/_metadata



[~/flink-1.11-SNAPSHOT]$ hadoop fs -mv 
congxianqiu/savepoint/savepoint-9bcc25-4ed827357f33 
congxianqiu/savepoint/newsavepointpath
[ ~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint
Found 1 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/savepoint/newsavepointpath
Found 2 items
-rw-r--r--   3 xxxxxx supergroup      74170 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath/6508ac9e-0d2a-4583-96ad-1d67fb5b1c8a
-rw-r--r--   3 xxxxxx supergroup       1205 2020-06-05 20:27 
congxianqiu/savepoint/newsavepointpath/_metadata


[~/flink-1.11-SNAPSHOT]$ ./bin/flink run -s 
hdfs:///user/xxxxxx/congxianqiu/newsavepointpath/_metadata -m yarn-cluster -c 
com.klion26.data.FlinkDemo 
/data/work/congxianqiu/flink-1.11-SNAPSHOT/ft_local/Flink-Demo-1.0-SNAPSHOT.jar
SLF4J: Class path contains multiple SLF4J bindings.


>>>>>> jobmanager.log
2020-06-05 21:11:10,053 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Starting job 
b2fbfa6527391f035e8eebd791c2f64e from savepoint 
hdfs:///user/xxxxxx/congxianqiu/savepoint/newsavepointpath/_metadata ()
2020-06-05 21:11:10,198 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Reset the 
checkpoint ID of job b2fbfa6527391f035e8eebd791c2f64e to 3.
2020-06-05 21:11:10,198 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Restoring job 
b2fbfa6527391f035e8eebd791c2f64e from latest valid checkpoint: Checkpoint 2 @ 0 
for b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:11:10,206 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - No master 
state to restore
.......
2020-06-05 21:11:16,117 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Checkpoint 
triggering task Source: Custom Source (1/1) of job 
b2fbfa6527391f035e8eebd791c2f64e is not in state RUNNING but SCHEDULED instead. 
Aborting checkpoint.
2020-06-05 21:11:19,456 INFO  org.apache.flink.yarn.YarnResourceManager         
           [] - Registering TaskManager with ResourceID 
container_e18_1591259429117_0019_01_000002 
(akka.tcp://flink@10-215-128-83:56603/user/rpc/taskmanager_0) at ResourceManager
2020-06-05 21:11:19,566 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (1/1) (bc206dbf0e964487e5ba8c4355cb691e) switched from SCHEDULED 
to DEPLOYING.
2020-06-05 21:11:19,566 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Deploying 
Source: Custom Source (1/1) (attempt #0) to 
container_e18_1591259429117_0019_01_000002 @ 10-215-128-83 (dataPort=45167)
2020-06-05 21:11:19,572 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (7ae861cf453455d722d5d4ece0c10d1a) switched from SCHEDULED to 
DEPLOYING.
2020-06-05 21:11:19,573 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Deploying Map 
-> Sink: Unnamed (1/1) (attempt #0) to 
container_e18_1591259429117_0019_01_000002 @ 10-215-128-83 (dataPort=45167)
2020-06-05 21:11:20,467 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Map -> Sink: 
Unnamed (1/1) (7ae861cf453455d722d5d4ece0c10d1a) switched from DEPLOYING to 
RUNNING.
2020-06-05 21:11:20,468 INFO  
org.apache.flink.runtime.executiongraph.ExecutionGraph       [] - Source: 
Custom Source (1/1) (bc206dbf0e964487e5ba8c4355cb691e) switched from DEPLOYING 
to RUNNING.
2020-06-05 21:12:16,199 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 3 @ 1591362736116 for job b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:12:16,854 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Completed 
checkpoint 3 for job b2fbfa6527391f035e8eebd791c2f64e (106237 bytes in 736 ms).
2020-06-05 21:13:16,172 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 4 @ 1591362796116 for job b2fbfa6527391f035e8eebd791c2f64e.
2020-06-05 21:13:16,680 INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Completed 
checkpoint 4 for job b2fbfa6527391f035e8eebd791c2f64e (32823 bytes in 542 ms).
{code}
 

 

2. log for checkpoint

 
{code:java}
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/      
Found 3 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:15 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/chk-6
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:15 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/shared
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:11 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/taskowned


[ ~/flink-1.11-SNAPSHOT]$ hadoop fs -mv 
congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e 
congxianqiu/checkpoint/movecheckpoint
[~/flink-1.11-SNAPSHOT]$ hadoop fs -ls congxianqiu/checkpoint/movecheckpoint
Found 3 items
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/chk-6
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/shared
drwxr-xr-x   - xxxxxx supergroup          0 2020-06-05 21:23 
congxianqiu/checkpoint/movecheckpoint/taskowned



jobmanager.log

Caused by: org.apache.hadoop.ipc.RemoteException: File does not exist: 
/user/xxxxxx/congxianqiu/checkpoint/b2fbfa6527391f035e8eebd791c2f64e/shared/56704ae2-d
04c-4073-aa6b-843a40e15bbe
        at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:71)
        at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:61)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocationsInt(FSNamesystem.java:1836)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1808)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:1723)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:588)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java
:366)
        at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:616)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:982)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2217)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2213)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2213)


        at org.apache.hadoop.ipc.Client.call(Client.java:1476) 
~[hadoop-common-2.7.4.jar:?]
        at org.apache.hadoop.ipc.Client.call(Client.java:1413) 
~[hadoop-common-2.7.4.jar:?]
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
 ~[hadoop-common-2.7.4.jar:?]
        at com.sun.proxy.$Proxy35.getBlockLocations(Unknown Source) ~[?:?]
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:255)
 ~[hadoop-hdfs-
2.7.4.jar:?]
{code}
 

 

> Test Relocatable Savepoints
> ---------------------------
>
>                 Key: FLINK-18091
>                 URL: https://issues.apache.org/jira/browse/FLINK-18091
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Tests
>            Reporter: Stephan Ewen
>            Assignee: Congxian Qiu(klion26)
>            Priority: Major
>              Labels: release-testing
>             Fix For: 1.11.0
>
>
> The test should do the following:
>  * take a savepoint. needs to make sure the job has enough state that there 
> is more than just the "_metadata" file
>  * copy it to another directory
>  * start the job from that savepoint by addressing the metadata file and by 
> addressing the savepoint directory
> We should also test that an incremental checkpoint that gets moved fails with 
> a reasonable exception.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to