Re: Checkpointing in Flink 1.5.0

2018-07-11 Thread Data Engineer
>
>> Hello Chesnay,
>>
>>
>>
>> Cluster (in kubernetes)-wide checkpointing directory using glusterfs
>> volume mount (thus file access protocol file:///) was working fine till
>> 1.4.2 for us. So we like to understand where the breakage happened in
>> 1.5.0.
>>
>> Can you please mention me the relevant source code files related to
>> rocksdb “custom file path” parsing logic? We would be interested to
>> investigate this.
>>
>>
>>
>> I also observed below in the log –
>>
>>
>>
>> Config uses deprecated configuration key 
>> 'state.backend.rocksdb.checkpointdir' instead of proper key 
>> 'state.backend.rocksdb.localdir'
>>
>> Regards,
>>
>> Shaswata
>>
>>
>>
>> *From:* Chesnay Schepler [mailto:ches...@apache.org ]
>>
>> *Sent:* Tuesday, July 03, 2018 5:52 PM
>> *To:* Data Engineer  
>> *Cc:* user@flink.apache.org
>> *Subject:* Re: Checkpointing in Flink 1.5.0
>>
>>
>>
>> The code appears to be working fine.
>>
>> This may happen because you're using a GlusterFS volume.
>> The RocksDBStateBackend uses java Files internally (NOT nio Paths), which
>> AFAIK only work properly against the plain local file-system.
>>
>> The GlusterFS nio FIleSystem implementation also explicitly does not
>> support conversions to File
>> <https://github.com/gluster/glusterfs-java-filesystem/blob/master/glusterfs-java-filesystem/src/main/java/com/peircean/glusterfs/GlusterPath.java#L271>
>> .
>>
>> On 03.07.2018 13:53, Chesnay Schepler wrote:
>>
>> Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some custom
>> file path parsing logic, will probe it a bit to see what the issue is.
>>
>> On 03.07.2018 13:45, Data Engineer wrote:
>>
>> 2018-07-03 11:30:35,703 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> 
>>
>> 2018-07-03 11:30:35,705 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Starting 
>> StandaloneSessionClusterEntrypoint (Version: , Rev:c61b108, 
>> Date:24.05.2018 @ 16:54:44 CEST)
>>
>> 2018-07-03 11:30:35,705 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  OS current 
>> user: flink
>>
>> 2018-07-03 11:30:35,705 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Current 
>> Hadoop/Kerberos user: 
>>
>> 2018-07-03 11:30:35,706 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM: 
>> OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.171-b10
>>
>> 2018-07-03 11:30:35,706 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Maximum 
>> heap size: 981 MiBytes
>>
>> 2018-07-03 11:30:35,706 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JAVA_HOME: 
>> /etc/alternatives/jre_openjdk/
>>
>> 2018-07-03 11:30:35,707 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  No Hadoop 
>> Dependency available
>>
>> 2018-07-03 11:30:35,707 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM Options:
>>
>> 2018-07-03 11:30:35,707 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xms1024m
>>
>> 2018-07-03 11:30:35,707 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xmx1024m
>>
>> 2018-07-03 11:30:35,708 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> -Dlog.file=/opt/flink-1.5.0/log/flink--standalonesession-0-myfl-flink-jobmanager-7b4d8c4dd4-bv6zf.log
>>
>> 2018-07-03 11:30:35,708 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> -Dlog4j.configuration=file:/opt/flink-1.5.0/conf/log4j.properties
>>
>> 2018-07-03 11:30:35,708 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> -Dlogback.configurationFile=file:/opt/flink-1.5.0/conf/logback.xml
>>
>> 2018-07-03 11:30:35,708 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Program 
>> Arguments:
>>
>> 2018-07-03 11:30:35,709 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> --configDir
>>
>> 2018-07-03 11:30:35,709 INFO  
>> org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
>> /o

Re: Checkpointing in Flink 1.5.0

2018-07-10 Thread Sampath Bhat
Chesnay - Why is the absolute file check required in the
RocksDBStateBackend.setDbStoragePaths(String ... paths). I think this is
causing the issue. Its not related to GlusterFS or file system. The same
problem can be reproduced with the following configuration on local
machine. The flink application should support checkpointing. We get the
IllegealArgumentexecption (Relative File paths not allowed)

state.backend: rocksdb
state.checkpoints.dir: file:///home/demo/checkpoints/ext_checkpoints
state.savepoints.dir: file:///home/demo/checkpoints/checkpoints/savepoints
state.backend.fs.checkpointdir:
file:///home/demo/checkpoints/checkpoints/fs_state
#state.backend.rocksdb.checkpointdir:
file:///home/demo/checkpoints/checkpoints/rocksdb_state
state.backend.rocksdb.localdir:
/home/demo/checkpoints/checkpoints/rocksdb_state

Any insights would be helpful.

On Wed, Jul 4, 2018 at 2:27 PM, Chesnay Schepler  wrote:

> Reference: https://issues.apache.org/jira/browse/FLINK-9739
>
>
> On 04.07.2018 10:46, Chesnay Schepler wrote:
>
> It's not really path-parsing logic, but path handling i suppose; see
> RocksDBStateBackend#setDbStoragePaths().
>
> I went ahead and converted said method into a simple test method, maybe
> this is enough to debug the issue.
>
> I assume this regression was caused by FLINK-6557, which refactored the
> state backend to rely on java Files instead of Flink paths.
> I'll open a JIRA to document it.
>
> The deprecation notice is not a problem.
>
> public static void testPaths(String... paths) {
>if (paths.length == 0) {
>   throw new IllegalArgumentException("empty paths");   }
>else {
>   File[] pp = new File[paths.length];  for (int i = 0; i < 
> paths.length; i++) {
>  final String rawPath = paths[i]; final String path; 
> if (rawPath == null) {
> throw new IllegalArgumentException("null path"); }
>  else {
> // we need this for backwards compatibility, to allow URIs like 
> 'file:///'...URI uri = null;try {
>uri = new Path(rawPath).toUri();}
> catch (Exception e) {
>// cannot parse as a path}
>
> if (uri != null && uri.getScheme() != null) {
>if ("file".equalsIgnoreCase(uri.getScheme())) {
>   path = uri.getPath();   }
>else {
>   throw new IllegalArgumentException("Path " + rawPath + " 
> has a non-local scheme");   }
> }
> else {
>path = rawPath;}
>  }
>
>  pp[i] = new File(path); if (!pp[i].isAbsolute()) { // my 
> suspicion is that this categorically fails for GlusterFS paths
> throw new IllegalArgumentException("Relative paths are not 
> supported"); }
>   }
>}
> }
>
>
>
> On 03.07.2018 16:35, Jash, Shaswata (Nokia - IN/Bangalore) wrote:
>
> Hello Chesnay,
>
>
>
> Cluster (in kubernetes)-wide checkpointing directory using glusterfs
> volume mount (thus file access protocol file:///) was working fine till
> 1.4.2 for us. So we like to understand where the breakage happened in
> 1.5.0.
>
> Can you please mention me the relevant source code files related to
> rocksdb “custom file path” parsing logic? We would be interested to
> investigate this.
>
>
>
> I also observed below in the log –
>
>
>
> Config uses deprecated configuration key 
> 'state.backend.rocksdb.checkpointdir' instead of proper key 
> 'state.backend.rocksdb.localdir'
>
> Regards,
>
> Shaswata
>
>
>
> *From:* Chesnay Schepler [mailto:ches...@apache.org ]
> *Sent:* Tuesday, July 03, 2018 5:52 PM
> *To:* Data Engineer  
> *Cc:* user@flink.apache.org
> *Subject:* Re: Checkpointing in Flink 1.5.0
>
>
>
> The code appears to be working fine.
>
> This may happen because you're using a GlusterFS volume.
> The RocksDBStateBackend uses java Files internally (NOT nio Paths), which
> AFAIK only work properly against the plain local file-system.
>
> The GlusterFS nio FIleSystem implementation also explicitly does not
> support conversions to File
> <https://github.com/gluster/glusterfs-java-filesystem/blob/master/glusterfs-java-filesystem/src/main/java/com/peircean/glusterfs/GlusterPath.java#L271>
> .
>
> On 03.07.2018 13:53, Chesnay Schepler wrote:
>
> Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some custom
> file path parsing logic, will probe it a bit to see what the issue is.
>
> On 03.07.2018 13:

Re: Checkpointing in Flink 1.5.0

2018-07-04 Thread Chesnay Schepler

Reference: https://issues.apache.org/jira/browse/FLINK-9739

On 04.07.2018 10:46, Chesnay Schepler wrote:
It's not really path-parsing logic, but path handling i suppose; see 
RocksDBStateBackend#setDbStoragePaths().


I went ahead and converted said method into a simple test method, 
maybe this is enough to debug the issue.


I assume this regression was caused by FLINK-6557, which refactored 
the state backend to rely on java Files instead of Flink paths.

I'll open a JIRA to document it.

The deprecation notice is not a problem.

public static void testPaths(String... paths) {
if (paths.length ==0) {
   throw new IllegalArgumentException("empty paths"); }
else {
   File[] pp =new File[paths.length]; for (int i =0; i < paths.length; i++) 
{
  final String rawPath = paths[i]; final String path; if (rawPath 
==null) {
 throw new IllegalArgumentException("null path"); }
  else {
 // we need this for backwards compatibility, to allow URIs like 
'file:///'... URI uri =null; try {

uri =new Path(rawPath).toUri(); }
 catch (Exception e) {
// cannot parse as a path }

 if (uri !=null && uri.getScheme() !=null) {
if ("file".equalsIgnoreCase(uri.getScheme())) {
   path = uri.getPath(); }
else {
   throw new IllegalArgumentException("Path " + rawPath +" has a 
non-local scheme"); }
 }
 else {
path = rawPath; }
  }

  pp[i] =new File(path); if (!pp[i].isAbsolute()) { // my suspicion is 
that this categorically fails for GlusterFS paths
 throw new IllegalArgumentException("Relative paths are not 
supported"); }
   }
}
}


On 03.07.2018 16:35, Jash, Shaswata (Nokia - IN/Bangalore) wrote:


Hello Chesnay,

Cluster (in kubernetes)-wide checkpointing directory using glusterfs 
volume mount (thus file access protocol file:///) was working fine 
till 1.4.2 for us. So we like to understand where the breakage 
happened in 1.5.0.


Can you please mention me the relevant source code files related to 
rocksdb “custom file path” parsing logic? We would be interested to 
investigate this.


I also observed below in the log –

Config uses deprecated configuration key 
'state.backend.rocksdb.checkpointdir' instead of proper key 
'state.backend.rocksdb.localdir'


Regards,

Shaswata

*From:*Chesnay Schepler [mailto:ches...@apache.org]
*Sent:* Tuesday, July 03, 2018 5:52 PM
*To:* Data Engineer 
*Cc:* user@flink.apache.org
*Subject:* Re: Checkpointing in Flink 1.5.0

The code appears to be working fine.

This may happen because you're using a GlusterFS volume.
The RocksDBStateBackend uses java Files internally (NOT nio Paths), 
which AFAIK only work properly against the plain local file-system.


The GlusterFS nio FIleSystem implementation also explicitly does not 
support conversions to File 
<https://github.com/gluster/glusterfs-java-filesystem/blob/master/glusterfs-java-filesystem/src/main/java/com/peircean/glusterfs/GlusterPath.java#L271>.


On 03.07.2018 13:53, Chesnay Schepler wrote:

Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some
custom file path parsing logic, will probe it a bit to see what
the issue is.

On 03.07.2018 13:45, Data Engineer wrote:

2018-07-03 11:30:35,703 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-




2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Starting StandaloneSessionClusterEntrypoint (Version:

, Rev:c61b108, Date:24.05.2018 @ 16:54:44 CEST)

2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  OS current user: flink


2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Current Hadoop/Kerberos user: 


2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  JVM: OpenJDK 64-Bit Server VM - Oracle Corporation -

1.8/25.171-b10

2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Maximum heap size: 981 MiBytes


2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  JAVA_HOME: /etc/alternatives/jre_openjdk/


2018-07-03 11:30:35,707 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  No Hadoop Dependency available


2018-07-03 11:30:35,707 INFO 
or

Re: Checkpointing in Flink 1.5.0

2018-07-04 Thread Chesnay Schepler
It's not really path-parsing logic, but path handling i suppose; see 
RocksDBStateBackend#setDbStoragePaths().


I went ahead and converted said method into a simple test method, maybe 
this is enough to debug the issue.


I assume this regression was caused by FLINK-6557, which refactored the 
state backend to rely on java Files instead of Flink paths.

I'll open a JIRA to document it.

The deprecation notice is not a problem.

public static void testPaths(String... paths) {
   if (paths.length ==0) {
  throw new IllegalArgumentException("empty paths"); }
   else {
  File[] pp =new File[paths.length]; for (int i =0; i < paths.length; i++) {
 final String rawPath = paths[i]; final String path; if (rawPath 
==null) {
throw new IllegalArgumentException("null path"); }
 else {
// we need this for backwards compatibility, to allow URIs like 
'file:///'... URI uri =null; try {

   uri =new Path(rawPath).toUri(); }
catch (Exception e) {
   // cannot parse as a path }

if (uri !=null && uri.getScheme() !=null) {
   if ("file".equalsIgnoreCase(uri.getScheme())) {
  path = uri.getPath(); }
   else {
  throw new IllegalArgumentException("Path " + rawPath +" has a 
non-local scheme"); }
}
else {
   path = rawPath; }
 }

 pp[i] =new File(path); if (!pp[i].isAbsolute()) { // my suspicion is 
that this categorically fails for GlusterFS paths
throw new IllegalArgumentException("Relative paths are not 
supported"); }
  }
   }
}



On 03.07.2018 16:35, Jash, Shaswata (Nokia - IN/Bangalore) wrote:


Hello Chesnay,

Cluster (in kubernetes)-wide checkpointing directory using glusterfs 
volume mount (thus file access protocol file:///) was working fine 
till 1.4.2 for us. So we like to understand where the breakage 
happened in 1.5.0.


Can you please mention me the relevant source code files related to 
rocksdb “custom file path” parsing logic? We would be interested to 
investigate this.


I also observed below in the log –

Config uses deprecated configuration key 
'state.backend.rocksdb.checkpointdir' instead of proper key 
'state.backend.rocksdb.localdir'


Regards,

Shaswata

*From:*Chesnay Schepler [mailto:ches...@apache.org]
*Sent:* Tuesday, July 03, 2018 5:52 PM
*To:* Data Engineer 
*Cc:* user@flink.apache.org
*Subject:* Re: Checkpointing in Flink 1.5.0

The code appears to be working fine.

This may happen because you're using a GlusterFS volume.
The RocksDBStateBackend uses java Files internally (NOT nio Paths), 
which AFAIK only work properly against the plain local file-system.


The GlusterFS nio FIleSystem implementation also explicitly does not 
support conversions to File 
<https://github.com/gluster/glusterfs-java-filesystem/blob/master/glusterfs-java-filesystem/src/main/java/com/peircean/glusterfs/GlusterPath.java#L271>.


On 03.07.2018 13:53, Chesnay Schepler wrote:

Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some
custom file path parsing logic, will probe it a bit to see what
the issue is.

On 03.07.2018 13:45, Data Engineer wrote:

2018-07-03 11:30:35,703 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-




2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Starting StandaloneSessionClusterEntrypoint (Version:

, Rev:c61b108, Date:24.05.2018 @ 16:54:44 CEST)

2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  OS current user: flink


2018-07-03 11:30:35,705 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Current Hadoop/Kerberos user: 


2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  JVM: OpenJDK 64-Bit Server VM - Oracle Corporation -

1.8/25.171-b10

2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  Maximum heap size: 981 MiBytes


2018-07-03 11:30:35,706 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  JAVA_HOME: /etc/alternatives/jre_openjdk/


2018-07-03 11:30:35,707 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  No Hadoop Dependency available


2018-07-03 11:30:35,707 INFO 
org.apache.flink.runtime.entrypoint.ClusterEntrypoint
-  JVM Options:


2018-07-03 11:30:35,707 INFO 
or

RE: Checkpointing in Flink 1.5.0

2018-07-03 Thread Jash, Shaswata (Nokia - IN/Bangalore)
Hello Chesnay,

Cluster (in kubernetes)-wide checkpointing directory using glusterfs volume 
mount (thus file access protocol file:///) was working fine till 1.4.2 for us. 
So we like to understand where the breakage happened in 1.5.0.
Can you please mention me the relevant source code files related to rocksdb 
“custom file path” parsing logic? We would be interested to investigate this.

I also observed below in the log –


Config uses deprecated configuration key 'state.backend.rocksdb.checkpointdir' 
instead of proper key 'state.backend.rocksdb.localdir'
Regards,
Shaswata

From: Chesnay Schepler [mailto:ches...@apache.org]
Sent: Tuesday, July 03, 2018 5:52 PM
To: Data Engineer 
Cc: user@flink.apache.org
Subject: Re: Checkpointing in Flink 1.5.0

The code appears to be working fine.

This may happen because you're using a GlusterFS volume.
The RocksDBStateBackend uses java Files internally (NOT nio Paths), which AFAIK 
only work properly against the plain local file-system.

The GlusterFS nio FIleSystem implementation also explicitly does not support 
conversions to 
File<https://github.com/gluster/glusterfs-java-filesystem/blob/master/glusterfs-java-filesystem/src/main/java/com/peircean/glusterfs/GlusterPath.java#L271>.

On 03.07.2018 13:53, Chesnay Schepler wrote:
Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some custom file 
path parsing logic, will probe it a bit to see what the issue is.

On 03.07.2018 13:45, Data Engineer wrote:

2018-07-03 11:30:35,703 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 


2018-07-03 11:30:35,705 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Starting 
StandaloneSessionClusterEntrypoint (Version: , Rev:c61b108, 
Date:24.05.2018 @ 16:54:44 CEST)

2018-07-03 11:30:35,705 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  OS current 
user: flink

2018-07-03 11:30:35,705 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Current 
Hadoop/Kerberos user: 

2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM: OpenJDK 
64-Bit Server VM - Oracle Corporation - 1.8/25.171-b10

2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Maximum heap 
size: 981 MiBytes

2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JAVA_HOME: 
/etc/alternatives/jre_openjdk/

2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  No Hadoop 
Dependency available

2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM Options:

2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xms1024m

2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xmx1024m

2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog.file=/opt/flink-1.5.0/log/flink--standalonesession-0-myfl-flink-jobmanager-7b4d8c4dd4-bv6zf.log

2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog4j.configuration=file:/opt/flink-1.5.0/conf/log4j.properties

2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlogback.configurationFile=file:/opt/flink-1.5.0/conf/logback.xml

2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Program 
Arguments:

2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --configDir

2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
/opt/flink-1.5.0/conf

2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
--executionMode

2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster

2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --host

2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster

2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Classpath: 
/opt/flink-1.5.0/lib/flink-cep_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-connectors-1.5.0.jar:/opt/flink-1.5.0/lib/flink-gelly_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-ml_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-table_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::

2018-07-03 11:30:35,710 INFO  
org.apache.flink.runti

Re: Checkpointing in Flink 1.5.0

2018-07-03 Thread Chesnay Schepler

The code appears to be working fine.

This may happen because you're using a GlusterFS volume.
The RocksDBStateBackend uses java Files internally (NOT nio Paths), 
which AFAIK only work properly against the plain local file-system.


The GlusterFS nio FIleSystem implementation also explicitly does not 
support conversions to File 
.


On 03.07.2018 13:53, Chesnay Schepler wrote:
Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some 
custom file path parsing logic, will probe it a bit to see what the 
issue is.


On 03.07.2018 13:45, Data Engineer wrote:

2018-07-03 11:30:35,703 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 

2018-07-03 11:30:35,705 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint   
  -  Starting StandaloneSessionClusterEntrypoint (Version: , 
Rev:c61b108, Date:24.05.2018 @ 16:54:44 CEST)
2018-07-03 11:30:35,705 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  OS current 
user: flink
2018-07-03 11:30:35,705 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint   
  -  Current Hadoop/Kerberos user: 
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM: OpenJDK 
64-Bit Server VM - Oracle Corporation - 1.8/25.171-b10
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Maximum heap 
size: 981 MiBytes
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JAVA_HOME: 
/etc/alternatives/jre_openjdk/
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  No Hadoop 
Dependency available
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM Options:
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xms1024m
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xmx1024m
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog.file=/opt/flink-1.5.0/log/flink--standalonesession-0-myfl-flink-jobmanager-7b4d8c4dd4-bv6zf.log
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog4j.configuration=file:/opt/flink-1.5.0/conf/log4j.properties
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlogback.configurationFile=file:/opt/flink-1.5.0/conf/logback.xml
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Program 
Arguments:
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --configDir
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
/opt/flink-1.5.0/conf
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
--executionMode
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --host
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Classpath: 
/opt/flink-1.5.0/lib/flink-cep_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-connectors-1.5.0.jar:/opt/flink-1.5.0/lib/flink-gelly_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-ml_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-table_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 

2018-07-03 11:30:35,712 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - Registered UNIX 
signal handlers for [TERM, HUP, INT]
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: blob.server.port, 4124
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: jobmanager.rpc.address, myfl-flink-jobmanager
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: jobmanager.rpc.port, 4123
2018-07-03 11:30:35,721 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configurat

Re: Checkpointing in Flink 1.5.0

2018-07-03 Thread Chesnay Schepler
Thanks. Looks like RocksDBStateBackend.setDbStoragePaths has some custom 
file path parsing logic, will probe it a bit to see what the issue is.


On 03.07.2018 13:45, Data Engineer wrote:

2018-07-03 11:30:35,703 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 

2018-07-03 11:30:35,705 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint   
  -  Starting StandaloneSessionClusterEntrypoint (Version: , 
Rev:c61b108, Date:24.05.2018 @ 16:54:44 CEST)
2018-07-03 11:30:35,705 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  OS current 
user: flink
2018-07-03 11:30:35,705 INFO  org.apache.flink.runtime.entrypoint.ClusterEntrypoint   
  -  Current Hadoop/Kerberos user: 
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM: OpenJDK 
64-Bit Server VM - Oracle Corporation - 1.8/25.171-b10
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Maximum heap 
size: 981 MiBytes
2018-07-03 11:30:35,706 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JAVA_HOME: 
/etc/alternatives/jre_openjdk/
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  No Hadoop 
Dependency available
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM Options:
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xms1024m
2018-07-03 11:30:35,707 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - -Xmx1024m
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog.file=/opt/flink-1.5.0/log/flink--standalonesession-0-myfl-flink-jobmanager-7b4d8c4dd4-bv6zf.log
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlog4j.configuration=file:/opt/flink-1.5.0/conf/log4j.properties
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
-Dlogback.configurationFile=file:/opt/flink-1.5.0/conf/logback.xml
2018-07-03 11:30:35,708 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Program 
Arguments:
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --configDir
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
/opt/flink-1.5.0/conf
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 
--executionMode
2018-07-03 11:30:35,709 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - --host
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - cluster
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  Classpath: 
/opt/flink-1.5.0/lib/flink-cep_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-connectors-1.5.0.jar:/opt/flink-1.5.0/lib/flink-gelly_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-ml_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-table_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
2018-07-03 11:30:35,710 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - 

2018-07-03 11:30:35,712 INFO  
org.apache.flink.runtime.entrypoint.ClusterEntrypoint - Registered UNIX 
signal handlers for [TERM, HUP, INT]
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: blob.server.port, 4124
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: jobmanager.rpc.address, myfl-flink-jobmanager
2018-07-03 11:30:35,720 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: jobmanager.rpc.port, 4123
2018-07-03 11:30:35,721 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: jobmanager.heap.mb, 1024
2018-07-03 11:30:35,721 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: taskmanager.heap.mb, 1024
2018-07-03 11:30:35,721 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: taskmanager.rpc.port, 4122
2018-07-03 11:30:35,721 INFO  
org.apache.flink.configuration.GlobalConfiguration- Loading 
configuration property: taskmanager.data.port, 4121
2018-07-03 11:30:35,721 INFO  

Re: Checkpointing in Flink 1.5.0

2018-07-03 Thread Data Engineer
2018-07-03 11:30:35,703 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -

2018-07-03 11:30:35,705 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Starting StandaloneSessionClusterEntrypoint (Version: ,
Rev:c61b108, Date:24.05.2018 @ 16:54:44 CEST)
2018-07-03 11:30:35,705 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  OS
current user: flink
2018-07-03 11:30:35,705 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Current Hadoop/Kerberos user: 
2018-07-03 11:30:35,706 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM:
OpenJDK 64-Bit Server VM - Oracle Corporation - 1.8/25.171-b10
2018-07-03 11:30:35,706 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Maximum heap size: 981 MiBytes
2018-07-03 11:30:35,706 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
JAVA_HOME: /etc/alternatives/jre_openjdk/
2018-07-03 11:30:35,707 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  No
Hadoop Dependency available
2018-07-03 11:30:35,707 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -  JVM
Options:
2018-07-03 11:30:35,707 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
-Xms1024m
2018-07-03 11:30:35,707 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
-Xmx1024m
2018-07-03 11:30:35,708 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
-Dlog.file=/opt/flink-1.5.0/log/flink--standalonesession-0-myfl-flink-jobmanager-7b4d8c4dd4-bv6zf.log
2018-07-03 11:30:35,708 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
-Dlog4j.configuration=file:/opt/flink-1.5.0/conf/log4j.properties
2018-07-03 11:30:35,708 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
-Dlogback.configurationFile=file:/opt/flink-1.5.0/conf/logback.xml
2018-07-03 11:30:35,708 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Program Arguments:
2018-07-03 11:30:35,709 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
--configDir
2018-07-03 11:30:35,709 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
/opt/flink-1.5.0/conf
2018-07-03 11:30:35,709 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
--executionMode
2018-07-03 11:30:35,709 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
cluster
2018-07-03 11:30:35,710 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
--host
2018-07-03 11:30:35,710 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
cluster
2018-07-03 11:30:35,710 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Classpath: 
/opt/flink-1.5.0/lib/flink-cep_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-connectors-1.5.0.jar:/opt/flink-1.5.0/lib/flink-gelly_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-ml_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/flink-table_2.11-1.5.0.jar:/opt/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
2018-07-03 11:30:35,710 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -

2018-07-03 11:30:35,712 INFO
org.apache.flink.runtime.entrypoint.ClusterEntrypoint -
Registered UNIX signal handlers for [TERM, HUP, INT]
2018-07-03 11:30:35,720 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: blob.server.port, 4124
2018-07-03 11:30:35,720 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: jobmanager.rpc.address,
myfl-flink-jobmanager
2018-07-03 11:30:35,720 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: jobmanager.rpc.port, 4123
2018-07-03 11:30:35,721 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: jobmanager.heap.mb, 1024
2018-07-03 11:30:35,721 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: taskmanager.heap.mb, 1024
2018-07-03 11:30:35,721 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: taskmanager.rpc.port, 4122
2018-07-03 11:30:35,721 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: taskmanager.data.port, 4121
2018-07-03 11:30:35,721 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: taskmanager.query.port, 4125
2018-07-03 11:30:35,722 INFO
org.apache.flink.configuration.GlobalConfiguration-
Loading configuration property: rest.port, 8081
2018-07-03 11:30:35,762 INFO
org.apache.flink.configuration.

Re: Checkpointing in Flink 1.5.0

2018-07-03 Thread Chesnay Schepler

Doesn't sound like intended behavior, can you give us the stacktrace?

On 03.07.2018 13:17, Data Engineer wrote:
 The Flink documentation says that we need to specify the filesystem 
type (file://, hdfs://) when configuring the rocksdb backend dir.

https://ci.apache.org/projects/flink/flink-docs-release-1.5/ops/state/state_backends.html#the-rocksdbstatebackend

But when I do this, I get an error on job submission saying that 
relative paths are not permitted in the rocksdb stand backend.

I am submitting the job via flink cli (bin/flink run).

Also, even though I give a local file system path 
"file:///home/abc/share", it is a shared GlusterFS volume mount, so it 
will be accessible by the JobManager and all TaskManagers.


I removed the filesystem type from the rocksdb backend dir 
configuration, and though the job got submitted, the rocksdb 
checkpoint directory was not created.

I have enabled checkpointing in my Flink application.

I am using Flink 1.5.0.

Any help or pointers would be appreciated.