You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by StephanEwen <gi...@git.apache.org> on 2017/07/25 16:47:05 UTC

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

GitHub user StephanEwen opened a pull request:

    https://github.com/apache/flink/pull/4397

    [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and ConsistencyLevel for FileSystem

    ## What is the purpose of the change
    
    This change lets File Systems exposes more information about their kind and consistency. For example, whether they support real directory structures, efficient recursive deletes, or rename consistency.
    
    This information is used to hotfix the [FLINK-7266] bug where S3 cleanup becomes prohibitively expensive due to excessive and unnecessary bucket contents listing.
    
    ## Brief change log
    
      - Adds `ConsistencyLevel` and `FileSystemKind` enums do describe file systems
      - All `FileSystems` declare their kind and consistency
      - `HadoopFileSystemWrapper` infers the kind and consistency from the scheme
      - `FileStateHandle` only attempts to delete parent directory if the target file system is a proper filesystem.
    
    
    ## Verifying this change
    
    This change is verified by the addition of some unit tests:
      - Checking that the consistency levels and properties relate as expected
      - Check inference of `LocalFileSystem` consistency (Linux / Windows)
      - Check inference of HDFS wrapper file systems consistency
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): **no**
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: **no**
      - The serializers: **no**
      - The runtime per-record code paths (performance sensitive): **no**
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: **no**
    
    The worst thing that could happen here is that empty checkpoint parent directories are not cleaned up on some file systems identifying themselves as `s3` or so.
    
    ## Documentation
    
      - Does this pull request introduce a new feature? **no**
      - If yes, how is the feature documented? **not applicable**


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StephanEwen/incubator-flink s3_cleanup

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/4397.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #4397
    
----
commit 2c9ebfca42758c0bab958e9488f2e2d9777ddae9
Author: Stephan Ewen <se...@apache.org>
Date:   2017-07-25T15:19:25Z

    [FLINK-7265] [core] Introduce FileSystemKind and ConsistencyLevel for FileSystem
    
    These describe the characteristics of the file system, such as consistency and support
    for directories and efficient directory operations.

commit dead4bbb2769d7aaade029cd6c76f8a2139f69ce
Author: Stephan Ewen <se...@apache.org>
Date:   2017-07-25T15:26:38Z

    [FLINK-7266] [core] Prevent attempt for parent directory deletion for object stores

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129375055
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    --- End diff --
    
    Not sure if we actually need this enum. The summary in the javadoc contain a list of filesystem properties that are relevant to us; could we not explicitly define these in the `FileSystemKind` intead of indirectly going through the ConsistencyLevel? I.e. change the constructor of `FileSystemKind` from taking a consistency level to taking separate arguments for each property. (Let's say `ConsistencyProperties`)
    
    I'm worried that the list will keep growing over time, and looking at the `FileSystemKind` enum it looks like we duplicate things; we have a posix FileSystemKind with a posix ConsistencyLevel and so on.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129581503
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    --- End diff --
    
    The class is missing an annotation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129593298
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration defining the kind and characteristics of a {@link FileSystem}.
    + * 
    + * <p>Note that the categorization here makes statements only about <i>consistency</i> and
    + * <i>directory handling</i>. It explicitly does not look at the ability to modify files,
    + * which we assume for no operation going through Flink's File System abstraction, at the moment.
    + * This might change in the future. 
    + */
    +@PublicEvolving
    +public enum FileSystemKind {
    +
    +	/**
    +	 * A POSIX compliant file system, as for example found on UNIX / Linux.
    +	 * 
    +	 * <p>Posix file systems support directories, a consistent view, atomic renames,
    +	 * and deletion of open files.
    +	 */
    +	POSIX_COMPLIANT(ConsistencyLevel.POSIX_STYLE_CONSISTENCY),
    +
    +	/**
    +	 * A file system that gives a consistent view of its contents.
    +	 * 
    +	 * <p>File systems in this category are for example Windows file systems,
    +	 * HDFS, or MapR FS. They support directories, a consistent view, but not
    +	 * necessarily atomic file renaming, or deletion of open files.
    +	 */
    +	CONSISTENT_FILESYSTEM(ConsistencyLevel.CONSISTENT_LIST_RENAME_DELETE),
    +
    +	/**
    +	 * A consistent object store (not an actual file system).
    +	 *
    +	 * <p>"File systems" of this kind support no real directories, but  and no consistent
    +	 * renaming and delete operations.
    +	 */
    +	CONSISTENT_OBJECT_STORE(ConsistencyLevel.CONSISTENT_RENAME_DELETE),
    --- End diff --
    
    It is not used somewhere. We can remove it and possibly add it later when we need it. I think it applies to some storage systems like Azure blob store or so, without having hard evidence for that, yet.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r130058934
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +@PublicEvolving
    +public enum ConsistencyLevel {
    --- End diff --
    
    Yes, that's why I said we shouldn't do it now and instead merge quickly to fix the S3 problem. 😅 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r130044644
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +@PublicEvolving
    +public enum ConsistencyLevel {
    --- End diff --
    
    This actually strikes me as over engineering at this point...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129580758
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    +
    +	/**
    +	 * This consistency level only guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, once the writing stream has been closed.
    +	 * Any modifications, renames, deletes are not guaranteed to be immediately visible in a
    +	 * consistent manner.
    +	 * 
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not consistent.
    +	 * 
    +	 * <p>An example of a storage system  with this consistency level is Amazon's S3 object store.
    +	 * 
    +	 * <p>This is the weakest consistency level with which Flink's checkpointing can work.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: Consistent
    +	 *     <li>File deletion: NOT consistent
    +	 *     <li>File renaming: NOT consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	READ_AFTER_CREATE,
    +
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, and after renaming them.
    +	 * The non-existence is consistently visible after delete operations.
    +	 *
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not necessarily consistent.
    +	 *
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	CONSISTENT_RENAME_DELETE,
    +	
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, as well as after renaming operations.
    +	 * File deletion is immediately visible to all parties.
    +	 * 
    +	 * <p>Directory listings are consistent, meaning after file creation/rename/delete, the file
    +	 * existence or non-existence is reflected when enumerating the parent directory's contents.
    +	 *
    +	 * <p>An example of storage systems and file systems falling under this consistency level are
    +	 * HDFS, MapR FS, and the Windows file systems.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent, but not necessarily atomic
    --- End diff --
    
    Is atomic renaming guaranteed for `CONSISTENT_RENAME_DELETE`? If not, how important is this atomicity for us? Based on the names one would assume that `LIST_RENAME_DELETE` is all-around "better" than `RENAME_DELETE`, which currently isn't the case and could lead to subtle bugs if this is actually relevant for us.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129575803
  
    --- Diff: flink-runtime/src/test/java/org/apache/flink/runtime/fs/hdfs/HdfsKindTest.java ---
    @@ -0,0 +1,59 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.fs.hdfs;
    +
    +import org.apache.flink.core.fs.FileSystem;
    +import org.apache.flink.core.fs.FileSystemKind;
    +import org.apache.flink.core.fs.Path;
    +
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +
    +import static org.junit.Assert.assertEquals;
    +
    +public class HdfsKindTest {
    --- End diff --
    
    This class will need a javadoc after a rebase (for master).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129525270
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    --- End diff --
    
    I understand what you are saying. Initially I had the consistency characteristics in the `FileSystemKind`. I played around with that for a bit and felt that factoring them out is actually the right way to go.
    
    We could change the enum to a class so that we don't have to create a new enum constant for each combination of consistency attributes (I think there will be no too many more, there is a pretty clear hierarchy from file creation --> file rename/delete  --> directory view consistency).
    
    We could also extend the `ConsistencyLevel` to have methods like `supportsConsistentRename()`, which would make it easier to evaluate if a certain property is provided in the using code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    Want to see how we proceed with master. Same minimal version, or the more extensive version here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/4397


---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129592169
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    +
    +	/**
    +	 * This consistency level only guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, once the writing stream has been closed.
    +	 * Any modifications, renames, deletes are not guaranteed to be immediately visible in a
    +	 * consistent manner.
    +	 * 
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not consistent.
    +	 * 
    +	 * <p>An example of a storage system  with this consistency level is Amazon's S3 object store.
    +	 * 
    +	 * <p>This is the weakest consistency level with which Flink's checkpointing can work.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: Consistent
    +	 *     <li>File deletion: NOT consistent
    +	 *     <li>File renaming: NOT consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	READ_AFTER_CREATE,
    +
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, and after renaming them.
    +	 * The non-existence is consistently visible after delete operations.
    +	 *
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not necessarily consistent.
    +	 *
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	CONSISTENT_RENAME_DELETE,
    +	
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, as well as after renaming operations.
    +	 * File deletion is immediately visible to all parties.
    +	 * 
    +	 * <p>Directory listings are consistent, meaning after file creation/rename/delete, the file
    +	 * existence or non-existence is reflected when enumerating the parent directory's contents.
    +	 *
    +	 * <p>An example of storage systems and file systems falling under this consistency level are
    +	 * HDFS, MapR FS, and the Windows file systems.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent, but not necessarily atomic
    --- End diff --
    
    No, atomic rename is not guaranteed here. It is interesting for future work in possibly unifying some rolling sink code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    I made the tests stable across Hadoop versions and addressed the comments/annotations/formatting.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r130025503
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    --- End diff --
    
    nit: double `in`. Can fix while merging


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    Merged a minimal version of this into `release-1.3` in 854b05376a459a6197e41e141bb28a9befe481ad


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129577113
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    +
    +	/**
    +	 * This consistency level only guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, once the writing stream has been closed.
    +	 * Any modifications, renames, deletes are not guaranteed to be immediately visible in a
    +	 * consistent manner.
    +	 * 
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not consistent.
    +	 * 
    +	 * <p>An example of a storage system  with this consistency level is Amazon's S3 object store.
    --- End diff --
    
    there's a double space here


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129592880
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +public enum ConsistencyLevel {
    +
    +	/**
    +	 * This consistency level only guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, once the writing stream has been closed.
    +	 * Any modifications, renames, deletes are not guaranteed to be immediately visible in a
    +	 * consistent manner.
    +	 * 
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not consistent.
    +	 * 
    +	 * <p>An example of a storage system  with this consistency level is Amazon's S3 object store.
    +	 * 
    +	 * <p>This is the weakest consistency level with which Flink's checkpointing can work.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: Consistent
    +	 *     <li>File deletion: NOT consistent
    +	 *     <li>File renaming: NOT consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	READ_AFTER_CREATE,
    +
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, and after renaming them.
    +	 * The non-existence is consistently visible after delete operations.
    +	 *
    +	 * <p>To access a file/object consistently, the full path/key must be provided. Enumeration
    +	 * of files/objects is not necessarily consistent.
    +	 *
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent
    +	 *     <li>Directory listing: NOT consistent
    +	 * </ul>
    +	 */
    +	CONSISTENT_RENAME_DELETE,
    +	
    +	/**
    +	 * This consistency level guarantees that files are visible with a consistent
    +	 * view of their contents after their initial creation, as well as after renaming operations.
    +	 * File deletion is immediately visible to all parties.
    +	 * 
    +	 * <p>Directory listings are consistent, meaning after file creation/rename/delete, the file
    +	 * existence or non-existence is reflected when enumerating the parent directory's contents.
    +	 *
    +	 * <p>An example of storage systems and file systems falling under this consistency level are
    +	 * HDFS, MapR FS, and the Windows file systems.
    +	 * 
    +	 * <b>Summary:</b>
    +	 * <ul>
    +	 *     <li>New file creation: consistent
    +	 *     <li>File deletion: consistent
    +	 *     <li>File renaming: consistent, but not necessarily atomic
    --- End diff --
    
    `CONSISTENT_LIST_RENAME_DELETE` is better all around, yes, the javadocs of `CONSISTENT_RENAME_DELETE` was wrong, hence the confusion.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    We've long experimented with the best way to do this in Hadoop, and I think we're converting on moving off any form of enum to some `hasFeature(String)` predicate. Why? Lets you handle a more of the obscure corner cases of blobstores, and have room to play in future versions. If it's a simple bool `isPosix()` all seems well, but in future you may even there want to consider a test for whether seek + write is supported, the FS is case sensititive, etc. Strings are probably going to turn out to be the best option


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    Yes, I think either is fine because it's not user facing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    @StephanEwen Could you please close this PR now that it's also merged for master/1.4?


---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    How about I just remove the consistency level completely and reduce the `FileSystemKind` to "file system" and "object store"?
    
    I tried to think through the ecosystem of file systems quite a bit and this stuck me as a good compromise, but there are of course at least 10 other ways to do that. Seems that this discussion is mainly about "why like this and not possibly like that" and there are of course pros and cons for each approach, depending on your view of that ecosystem. We can guess how to do that now or try to evolve later as we discover more requirements and more file system types.
    
    In any case, my main concern here is to fix the FileStateHandle release for 1.3.2, so let's strip this down to the bare minimum and get this done?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    @steveloughran Thanks for the comment. We'll take this input into account for the more elaborate handling in the next version.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by StephanEwen <gi...@git.apache.org>.
Github user StephanEwen commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129524013
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration defining the kind and characteristics of a {@link FileSystem}.
    + * 
    + * <p>Note that the categorization here makes statements only about <i>consistency</i> and
    + * <i>directory handling</i>. It explicitly does not look at the ability to modify files,
    + * which we assume for no operation going through Flink's File System abstraction, at the moment.
    + * This might change in the future. 
    + */
    +@PublicEvolving
    +public enum FileSystemKind {
    +
    +	/**
    +	 * A POSIX compliant file system, as for example found on UNIX / Linux.
    +	 * 
    +	 * <p>Posix file systems support directories, a consistent view, atomic renames,
    +	 * and deletion of open files.
    +	 */
    +	POSIX_COMPLIANT(ConsistencyLevel.POSIX_STYLE_CONSISTENCY),
    +
    +	/**
    +	 * A file system that gives a consistent view of its contents.
    +	 * 
    +	 * <p>File systems in this category are for example Windows file systems,
    +	 * HDFS, or MapR FS. They support directories, a consistent view, but not
    +	 * necessarily atomic file renaming, or deletion of open files.
    +	 */
    +	CONSISTENT_FILESYSTEM(ConsistencyLevel.CONSISTENT_LIST_RENAME_DELETE),
    +
    +	/**
    +	 * A consistent object store (not an actual file system).
    +	 *
    +	 * <p>"File systems" of this kind support no real directories, but  and no consistent
    --- End diff --
    
    Will fix...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129372059
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration defining the kind and characteristics of a {@link FileSystem}.
    + * 
    + * <p>Note that the categorization here makes statements only about <i>consistency</i> and
    + * <i>directory handling</i>. It explicitly does not look at the ability to modify files,
    + * which we assume for no operation going through Flink's File System abstraction, at the moment.
    + * This might change in the future. 
    + */
    +@PublicEvolving
    +public enum FileSystemKind {
    +
    +	/**
    +	 * A POSIX compliant file system, as for example found on UNIX / Linux.
    +	 * 
    +	 * <p>Posix file systems support directories, a consistent view, atomic renames,
    +	 * and deletion of open files.
    +	 */
    +	POSIX_COMPLIANT(ConsistencyLevel.POSIX_STYLE_CONSISTENCY),
    +
    +	/**
    +	 * A file system that gives a consistent view of its contents.
    +	 * 
    +	 * <p>File systems in this category are for example Windows file systems,
    +	 * HDFS, or MapR FS. They support directories, a consistent view, but not
    +	 * necessarily atomic file renaming, or deletion of open files.
    +	 */
    +	CONSISTENT_FILESYSTEM(ConsistencyLevel.CONSISTENT_LIST_RENAME_DELETE),
    +
    +	/**
    +	 * A consistent object store (not an actual file system).
    +	 *
    +	 * <p>"File systems" of this kind support no real directories, but  and no consistent
    --- End diff --
    
    part of the javadoc appears to be missing


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r129580082
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/FileSystemKind.java ---
    @@ -0,0 +1,111 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration defining the kind and characteristics of a {@link FileSystem}.
    + * 
    + * <p>Note that the categorization here makes statements only about <i>consistency</i> and
    + * <i>directory handling</i>. It explicitly does not look at the ability to modify files,
    + * which we assume for no operation going through Flink's File System abstraction, at the moment.
    + * This might change in the future. 
    + */
    +@PublicEvolving
    +public enum FileSystemKind {
    +
    +	/**
    +	 * A POSIX compliant file system, as for example found on UNIX / Linux.
    +	 * 
    +	 * <p>Posix file systems support directories, a consistent view, atomic renames,
    +	 * and deletion of open files.
    +	 */
    +	POSIX_COMPLIANT(ConsistencyLevel.POSIX_STYLE_CONSISTENCY),
    +
    +	/**
    +	 * A file system that gives a consistent view of its contents.
    +	 * 
    +	 * <p>File systems in this category are for example Windows file systems,
    +	 * HDFS, or MapR FS. They support directories, a consistent view, but not
    +	 * necessarily atomic file renaming, or deletion of open files.
    +	 */
    +	CONSISTENT_FILESYSTEM(ConsistencyLevel.CONSISTENT_LIST_RENAME_DELETE),
    +
    +	/**
    +	 * A consistent object store (not an actual file system).
    +	 *
    +	 * <p>"File systems" of this kind support no real directories, but  and no consistent
    +	 * renaming and delete operations.
    +	 */
    +	CONSISTENT_OBJECT_STORE(ConsistencyLevel.CONSISTENT_RENAME_DELETE),
    --- End diff --
    
    This isn't used anywhere, do we have it for future filesystems? (same applies to CONSISTENT_RENAME_DELETE)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind...

Posted by aljoscha <gi...@git.apache.org>.
Github user aljoscha commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4397#discussion_r130027082
  
    --- Diff: flink-core/src/main/java/org/apache/flink/core/fs/ConsistencyLevel.java ---
    @@ -0,0 +1,119 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.core.fs;
    +
    +import org.apache.flink.annotation.PublicEvolving;
    +
    +/**
    + * An enumeration describing the level of consistency offered by a {@link FileSystem}.
    + * 
    + * <p>The consistency levels described here make statements about the visibility
    + * of file existence and file contents in the presence of <i>new file creation</i>,
    + * <i>file deletion</i>, <i>file renaming</i>, and <i>directory listing</i>.
    + * 
    + * <p>An operation is defined as consistent if the following holds: After the function
    + * call triggering the operation returns, its result is immediately reflected in
    + * in the view presented to any other party calling a file system function.
    + * 
    + * <p>Please note that these levels do not make any statements about the effects or visibility of
    + * file content modification or file appends. In fact, content modification or appending are
    + * not supported in various file systems.
    + * 
    + * <p>Some of these consistency levels indicate that the storage system does not actually
    + * qualify to be called a FileSystem, but rather a blob-/object store.
    + */
    +@PublicEvolving
    +public enum ConsistencyLevel {
    --- End diff --
    
    Have you thought about making the individual properties fields of the enum, i.e. have fields for `newFileCreation`, `fileDeletion`, `fileRenaming`, `directoryListing`. Then `READ_AFTER_CREATE` would become, for example, `READ_AFTER_CREATE(CONSISTENT, NOT_CONSISTENT, NOT_CONSISTENT, NOT_CONSISTENT)`, and so on. I'm not sure whether the fields would be booleans or another enum themselves. It seems there is at least `CONSISTENT`, `NOT_CONSISTENT`, and `ATOMIC`.
    
    That's just something to consider when merging this on master. I think we can merge as is to fix the problem people are having with checkpointing to S3.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #4397: [FLINK-7265] [FLINK-7266] Introduce FileSystemKind and Co...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/4397
  
    Test failure: ```testS3Kind(org.apache.flink.runtime.fs.hdfs.HdfsKindTest)  Time elapsed: 0.186 sec  <<< ERROR!
    java.io.IOException: No file system found with scheme s3, referenced in file URI 's3://myId:mySecret@bucket/some/bucket/some/object'.
    	at org.apache.flink.core.fs.FileSystem.getUnguardedFileSystem(FileSystem.java:356)
    	at org.apache.flink.core.fs.FileSystem.get(FileSystem.java:389)
    	at org.apache.flink.core.fs.Path.getFileSystem(Path.java:293)
    	at org.apache.flink.runtime.fs.hdfs.HdfsKindTest.testS3Kind(HdfsKindTest.java:42)```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---