You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2018/09/25 06:51:58 UTC

[flink] branch release-1.6 updated (3558d15 -> 67fa338)

This is an automated email from the ASF dual-hosted git repository.

sewen pushed a change to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 3558d15  [FLINK-10157][State TTL] Allow `null` user values in map state with TTL
     new aff9994  Revert "[FLINK-9061] Add entropy injection to S3 file system"
     new e08f557  [hotfix] Add JavaDocs to CheckpointMetadataOutputStream
     new f2d8633  [FLINK-9061] [fs] Add entropy injector for file systems
     new badb72d  [FLINK-9061] [checkpoints] FsStatebackend optionally injects entropy into state data file paths
     new a2b2041  [FLINK-9061] [s3] Make S3 Presto file system entropy injecting
     new 67fa338  [FLINK-10383] [s3] Prevent Hadoop configs on classpath to interfere with S3 configuration

The 6 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 ...emKind.java => EntropyInjectingFileSystem.java} |   25 +-
 .../org/apache/flink/core/fs/EntropyInjector.java  |  135 ++
 .../java/org/apache/flink/core/fs/FileSystem.java  |   44 +-
 ...ileSystemKind.java => OutputStreamAndPath.java} |   31 +-
 .../org/apache/flink/core/fs/WriteOptions.java     |  144 --
 .../java/org/apache/flink/util/StringUtils.java    |    7 +-
 .../apache/flink/core/fs/EntropyInjectorTest.java  |  148 ++
 .../org/apache/flink/util/StringUtilsTest.java     |    8 +-
 .../flink/runtime/fs/hdfs/HadoopFileSystem.java    |   25 -
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../src/test/resources/core-default-testing.xml}   |    0
 .../src/test/resources/core-site.xml               | 2323 +-------------------
 .../flink/fs/s3presto/S3FileSystemFactory.java     |   49 +-
 .../flink/fs/s3presto/S3PrestoFileSystem.java      |  245 +--
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../flink/fs/s3presto/PrestoS3EntropyTest.java}    |   25 +-
 .../fs/s3presto/PrestoS3FileSystemEntropyTest.java |  133 --
 .../flink/fs/s3presto/PrestoS3FileSystemTest.java  |   26 +-
 .../{core-site.xml => core-default-testing.xml}    |    0
 .../src/test/resources/core-site.xml               | 1989 +----------------
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../{core-site.xml => core-default-testing.xml}    |    0
 .../src/test/resources/core-site.xml               | 2323 +-------------------
 .../state/CheckpointMetadataOutputStream.java      |   15 +
 .../FsCheckpointMetadataOutputStream.java          |    3 +-
 .../filesystem/FsCheckpointStorageLocation.java    |    6 +-
 .../filesystem/FsCheckpointStreamFactory.java      |   18 +-
 .../FsCheckpointStateOutputStreamTest.java         |   12 +-
 .../filesystem/FsStateBackendEntropyTest.java      |   84 +-
 29 files changed, 570 insertions(+), 7296 deletions(-)
 copy flink-core/src/main/java/org/apache/flink/core/fs/{FileSystemKind.java => EntropyInjectingFileSystem.java} (50%)
 create mode 100644 flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjector.java
 copy flink-core/src/main/java/org/apache/flink/core/fs/{FileSystemKind.java => OutputStreamAndPath.java} (64%)
 delete mode 100644 flink-core/src/main/java/org/apache/flink/core/fs/WriteOptions.java
 create mode 100644 flink-core/src/test/java/org/apache/flink/core/fs/EntropyInjectorTest.java
 copy flink-filesystems/{flink-swift-fs-hadoop/src/test/resources/core-site.xml => flink-s3-fs-hadoop/src/test/resources/core-default-testing.xml} (100%)
 copy flink-filesystems/{flink-s3-fs-hadoop/src/test/java/org/apache/flink/fs/s3hadoop/HadoopS3FileSystemTest.java => flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3EntropyTest.java} (56%)
 delete mode 100644 flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemEntropyTest.java
 copy flink-filesystems/flink-s3-fs-presto/src/test/resources/{core-site.xml => core-default-testing.xml} (100%)
 copy flink-filesystems/flink-swift-fs-hadoop/src/test/resources/{core-site.xml => core-default-testing.xml} (100%)


[flink] 03/06: [FLINK-9061] [fs] Add entropy injector for file systems

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit f2d8633f3d805ebd2900b54e300817d7384d84dd
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Mon Sep 24 20:40:57 2018 +0200

    [FLINK-9061] [fs] Add entropy injector for file systems
---
 .../flink/core/fs/EntropyInjectingFileSystem.java  |  49 +++++++
 .../org/apache/flink/core/fs/EntropyInjector.java  | 135 +++++++++++++++++++
 .../apache/flink/core/fs/OutputStreamAndPath.java  |  47 +++++++
 .../java/org/apache/flink/util/StringUtils.java    |  33 +++++
 .../apache/flink/core/fs/EntropyInjectorTest.java  | 148 +++++++++++++++++++++
 .../org/apache/flink/util/StringUtilsTest.java     |  12 ++
 6 files changed, 424 insertions(+)

diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjectingFileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjectingFileSystem.java
new file mode 100644
index 0000000..14a15be
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjectingFileSystem.java
@@ -0,0 +1,49 @@
+/*
+ * 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 interface to be implemented by a {@link FileSystem} that is aware of entropy injection.
+ *
+ * <p>Entropy injection is a technique to spread files/objects across more parallel shards of
+ * a distributed storage (typically object store) by adding random characters to the beginning
+ * of the path/key and hence spearing the keys across a wider domain of prefixes.
+ *
+ * <p>Entropy injection typically works by having a recognized marker string in paths
+ * and replacing that marker with random characters.
+ *
+ * <p>This interface is used in conjunction with the {@link EntropyInjector} (as a poor man's
+ * way to build a mix-in in Java).
+ */
+@PublicEvolving
+public interface EntropyInjectingFileSystem {
+
+	/**
+	 * Gets the marker string that represents the substring of a path to be replaced
+	 * by the entropy characters.
+	 */
+	String getEntropyInjectionKey();
+
+	/**
+	 * Creates a string with random entropy to be injected into a path.
+	 */
+	String generateEntropy();
+}
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjector.java b/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjector.java
new file mode 100644
index 0000000..5bcb618
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/EntropyInjector.java
@@ -0,0 +1,135 @@
+/*
+ * 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;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+/**
+ * This class offers utilities for entropy injection for FileSystems that implement
+ * {@link EntropyInjectingFileSystem}.
+ */
+@PublicEvolving
+public class EntropyInjector {
+
+	/**
+	 * Handles entropy injection across regular and entropy-aware file systems.
+	 *
+	 * <p>If the given file system is entropy-aware (a implements {@link EntropyInjectingFileSystem}),
+	 * then this method replaces the entropy marker in the path with random characters.
+	 * The entropy marker is defined by {@link EntropyInjectingFileSystem#getEntropyInjectionKey()}.
+	 *
+	 * <p>If the given file system does not implement {@code EntropyInjectingFileSystem},
+	 * then this method delegates to {@link FileSystem#create(Path, WriteMode)} and
+	 * returns the same path in the resulting {@code OutputStreamAndPath}.
+	 */
+	public static OutputStreamAndPath createEntropyAware(
+			FileSystem fs,
+			Path path,
+			WriteMode writeMode) throws IOException {
+
+		if (!(fs instanceof EntropyInjectingFileSystem)) {
+			return new OutputStreamAndPath(fs.create(path, writeMode), path);
+		}
+
+		final EntropyInjectingFileSystem efs = (EntropyInjectingFileSystem) fs;
+		final Path pathWithEntropy = resolveEntropy(path, efs, true);
+
+		final FSDataOutputStream out = fs.create(pathWithEntropy, writeMode);
+		return new OutputStreamAndPath(out, pathWithEntropy);
+	}
+
+	/**
+	 * Removes the entropy marker string from the path, if the given file system is an
+	 * entropy-injecting file system (implements {@link EntropyInjectingFileSystem}) and
+	 * the entropy marker key is present. Otherwise, this returns the path as is.
+	 *
+	 * @param path The path to filter.
+	 * @return The path without the marker string.
+	 */
+	public static Path removeEntropyMarkerIfPresent(FileSystem fs, Path path) {
+		if (fs instanceof EntropyInjectingFileSystem) {
+			try {
+				return resolveEntropy(path, (EntropyInjectingFileSystem) fs, false);
+			}
+			catch (IOException e) {
+				// this should never happen, because the path was valid before. rethrow to silence the compiler
+				throw new FlinkRuntimeException(e.getMessage(), e);
+			}
+		}
+		else {
+			return path;
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	@VisibleForTesting
+	static Path resolveEntropy(Path path, EntropyInjectingFileSystem efs, boolean injectEntropy) throws IOException {
+		final String entropyInjectionKey = efs.getEntropyInjectionKey();
+
+		if (entropyInjectionKey == null) {
+			return path;
+		}
+		else {
+			final URI originalUri = path.toUri();
+			final String checkpointPath = originalUri.getPath();
+
+			final int indexOfKey = checkpointPath.indexOf(entropyInjectionKey);
+			if (indexOfKey == -1) {
+				return path;
+			}
+			else {
+				final StringBuilder buffer = new StringBuilder(checkpointPath.length());
+				buffer.append(checkpointPath, 0, indexOfKey);
+
+				if (injectEntropy) {
+					buffer.append(efs.generateEntropy());
+				}
+
+				buffer.append(checkpointPath, indexOfKey + entropyInjectionKey.length(), checkpointPath.length());
+
+				final String rewrittenPath = buffer.toString();
+				try {
+					return new Path(new URI(
+							originalUri.getScheme(),
+							originalUri.getAuthority(),
+							rewrittenPath,
+							originalUri.getQuery(),
+							originalUri.getFragment()).normalize());
+				}
+				catch (URISyntaxException e) {
+					// this could only happen if the injected entropy string contains invalid characters
+					throw new IOException("URI format error while processing path for entropy injection", e);
+				}
+			}
+		}
+	}
+
+	// ------------------------------------------------------------------------
+
+	/** This class is not meant to be instantiated. */
+	private EntropyInjector() {}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/OutputStreamAndPath.java b/flink-core/src/main/java/org/apache/flink/core/fs/OutputStreamAndPath.java
new file mode 100644
index 0000000..62c9479
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/OutputStreamAndPath.java
@@ -0,0 +1,47 @@
+/*
+ * 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 static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * An output stream and a path.
+ */
+public final class OutputStreamAndPath {
+
+	private final FSDataOutputStream stream;
+
+	private final Path path;
+
+	/**
+	 * Creates a OutputStreamAndPath.
+	 */
+	public OutputStreamAndPath(FSDataOutputStream stream, Path path) {
+		this.stream = checkNotNull(stream);
+		this.path = checkNotNull(path);
+	}
+
+	public FSDataOutputStream stream() {
+		return stream;
+	}
+
+	public Path path() {
+		return path;
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
index 208a301..a5fff4c 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Random;
 
+import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -247,6 +248,38 @@ public final class StringUtils {
 	}
 
 	/**
+	 * Creates a random alphanumeric string of given length.
+	 *
+	 * @param rnd The random number generator to use.
+	 * @param length The number of alphanumeric characters to append.
+	 */
+	public static String generateRandomAlphanumericString(Random rnd, int length) {
+		checkNotNull(rnd);
+		checkArgument(length >= 0);
+
+		StringBuilder buffer = new StringBuilder(length);
+		for (int i = 0; i < length; i++) {
+			buffer.append(nextAlphanumericChar(rnd));
+		}
+		return buffer.toString();
+	}
+
+	private static char nextAlphanumericChar(Random rnd) {
+		int which = rnd.nextInt(62);
+		char c;
+		if (which < 10) {
+			c = (char) ('0' + which);
+		}
+		else if (which < 36) {
+			c = (char) ('A' - 10 + which);
+		}
+		else {
+			c = (char) ('a' - 36 + which);
+		}
+		return c;
+	}
+
+	/**
 	 * Writes a String to the given output.
 	 * The written string can be read with {@link #readString(DataInputView)}.
 	 *
diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/EntropyInjectorTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/EntropyInjectorTest.java
new file mode 100644
index 0000000..a425826
--- /dev/null
+++ b/flink-core/src/test/java/org/apache/flink/core/fs/EntropyInjectorTest.java
@@ -0,0 +1,148 @@
+/*
+ * 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.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.local.LocalFileSystem;
+
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests for the {@link EntropyInjector}.
+ */
+public class EntropyInjectorTest {
+
+	@ClassRule
+	public static final TemporaryFolder TMP_FOLDER = new TemporaryFolder();
+
+	@Test
+	public void testEmptyPath() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("test", "ignored");
+		Path path = new Path("hdfs://localhost:12345");
+
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testFullUriNonMatching() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("_entropy_key_", "ignored");
+		Path path = new Path("s3://hugo@myawesomehost:55522/path/to/the/file");
+
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testFullUriMatching() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("s0mek3y", "12345678");
+		Path path = new Path("s3://hugo@myawesomehost:55522/path/s0mek3y/the/file");
+
+		assertEquals(new Path("s3://hugo@myawesomehost:55522/path/12345678/the/file"), EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(new Path("s3://hugo@myawesomehost:55522/path/the/file"), EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testPathOnlyNonMatching() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("_entropy_key_", "ignored");
+		Path path = new Path("/path/file");
+
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(path, EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testPathOnlyMatching() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("_entropy_key_", "xyzz");
+		Path path = new Path("/path/_entropy_key_/file");
+
+		assertEquals(new Path("/path/xyzz/file"), EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(new Path("/path/file"), EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testEntropyNotFullSegment() throws Exception {
+		EntropyInjectingFileSystem efs = new TestEntropyInjectingFs("_entropy_key_", "pqr");
+		Path path = new Path("s3://myhost:122/entropy-_entropy_key_-suffix/file");
+
+		assertEquals(new Path("s3://myhost:122/entropy-pqr-suffix/file"), EntropyInjector.resolveEntropy(path, efs, true));
+		assertEquals(new Path("s3://myhost:122/entropy--suffix/file"), EntropyInjector.resolveEntropy(path, efs, false));
+	}
+
+	@Test
+	public void testCreateEntropyAwarePlainFs() throws Exception {
+		File folder = TMP_FOLDER.newFolder();
+		Path path = new Path(Path.fromLocalFile(folder), "_entropy_/file");
+
+		OutputStreamAndPath out = EntropyInjector.createEntropyAware(
+				LocalFileSystem.getSharedInstance(), path, WriteMode.NO_OVERWRITE);
+
+		out.stream().close();
+
+		assertEquals(path, out.path());
+		assertTrue(new File (new File(folder, "_entropy_"), "file").exists());
+	}
+
+	@Test
+	public void testCreateEntropyAwareEntropyFs() throws Exception {
+		File folder = TMP_FOLDER.newFolder();
+		Path path = new Path(Path.fromLocalFile(folder), "_entropy_/file");
+		Path pathWithEntropy = new Path(Path.fromLocalFile(folder), "test-entropy/file");
+
+		FileSystem fs = new TestEntropyInjectingFs("_entropy_", "test-entropy");
+
+		OutputStreamAndPath out = EntropyInjector.createEntropyAware(fs, path, WriteMode.NO_OVERWRITE);
+
+		out.stream().close();
+
+		assertEquals(new Path(Path.fromLocalFile(folder), "test-entropy/file"), out.path());
+		assertTrue(new File (new File(folder, "test-entropy"), "file").exists());
+	}
+
+	// ------------------------------------------------------------------------
+
+	private static final class TestEntropyInjectingFs extends LocalFileSystem implements EntropyInjectingFileSystem {
+
+		private final String key;
+
+		private final String entropy;
+
+		TestEntropyInjectingFs(String key, String entropy) {
+			this.key = key;
+			this.entropy = entropy;
+		}
+
+		@Override
+		public String getEntropyInjectionKey() {
+			return key;
+		}
+
+		@Override
+		public String generateEntropy() {
+			return entropy;
+		}
+	}
+}
diff --git a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
index 5f705b4..7aa8e6e 100644
--- a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
@@ -20,8 +20,11 @@ package org.apache.flink.util;
 
 import org.junit.Test;
 
+import java.util.Random;
+
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Tests for the {@link StringUtils}.
@@ -56,4 +59,13 @@ public class StringUtilsTest extends TestLogger {
 		String hex = StringUtils.byteToHexString(byteArray);
 		assertEquals("019f314a", hex);
 	}
+
+	@Test
+	public void testGenerateAlphanumeric() {
+		String str = StringUtils.generateRandomAlphanumericString(new Random(), 256);
+
+		if (!str.matches("[a-zA-Z0-9]{256}")) {
+			fail("Not alphanumeric: " + str);
+		}
+	}
 }


[flink] 01/06: Revert "[FLINK-9061] Add entropy injection to S3 file system"

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit aff9994862331a81ab93dd206073a0503d784c84
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Sep 20 21:57:22 2018 +0200

    Revert "[FLINK-9061] Add entropy injection to S3 file system"
    
    This was incorrectly applied to the 1.6 branch
---
 .../java/org/apache/flink/core/fs/FileSystem.java  |  44 +--
 .../org/apache/flink/core/fs/WriteOptions.java     | 144 ----------
 .../java/org/apache/flink/util/StringUtils.java    |  32 ---
 .../org/apache/flink/util/StringUtilsTest.java     |  14 -
 .../flink/runtime/fs/hdfs/HadoopFileSystem.java    |  25 --
 .../flink/fs/s3presto/S3FileSystemFactory.java     | 100 +------
 .../flink/fs/s3presto/S3PrestoFileSystem.java      | 301 ---------------------
 .../fs/s3presto/PrestoS3FileSystemEntropyTest.java | 133 ---------
 .../flink/fs/s3presto/PrestoS3FileSystemTest.java  |  26 +-
 .../state/filesystem/FsCheckpointStorage.java      |  17 +-
 .../filesystem/FsCheckpointStreamFactory.java      |   7 +-
 .../FsCheckpointStateOutputStreamTest.java         |  12 +-
 .../filesystem/FsStateBackendEntropyTest.java      |  99 -------
 13 files changed, 48 insertions(+), 906 deletions(-)

diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
index ba2113a..d451109 100644
--- a/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
+++ b/flink-core/src/main/java/org/apache/flink/core/fs/FileSystem.java
@@ -576,8 +576,29 @@ public abstract class FileSystem {
 	/**
 	 * Opens an FSDataOutputStream at the indicated Path.
 	 *
-	 * @deprecated Deprecated in favor of {@link #create(Path, WriteOptions)} which offers better extensibility
-	 *             to options that are supported only by some filesystems implementations.
+	 * <p>This method is deprecated, because most of its parameters are ignored by most file systems.
+	 * To control for example the replication factor and block size in the Hadoop Distributed File system,
+	 * make sure that the respective Hadoop configuration file is either linked from the Flink configuration,
+	 * or in the classpath of either Flink or the user code.
+	 *
+	 * @param f
+	 *        the file name to open
+	 * @param overwrite
+	 *        if a file with this name already exists, then if true,
+	 *        the file will be overwritten, and if false an error will be thrown.
+	 * @param bufferSize
+	 *        the size of the buffer to be used.
+	 * @param replication
+	 *        required block replication for the file.
+	 * @param blockSize
+	 *        the size of the file blocks
+	 *
+	 * @throws IOException Thrown, if the stream could not be opened because of an I/O, or because
+	 *                     a file already exists at that path and the write mode indicates to not
+	 *                     overwrite the file.
+	 *
+	 * @deprecated Deprecated because not well supported across types of file systems.
+	 *             Control the behavior of specific file systems via configurations instead.
 	 */
 	@Deprecated
 	public FSDataOutputStream create(
@@ -628,25 +649,6 @@ public abstract class FileSystem {
 	public abstract FSDataOutputStream create(Path f, WriteMode overwriteMode) throws IOException;
 
 	/**
-	 * Creates a new file at the given path and opens an FSDataOutputStream to that new file.
-	 *
-	 * <p>This method takes various options, some of which are not supported by all file systems
-	 * (such as controlling block size).
-	 *
-	 * <p>Implementation note: This method should be abstract, but is currently not in order to not break
-	 * backwards compatibility of this class with earlier Flink versions.
-	 *
-	 * @param f The path for the new file.
-	 * @param options The options to parametrize the file and stream creation.
-	 * @return The stream to the new file at the target path.
-	 *
-	 * @throws IOException Thrown if an error occurs while creating the file or opening the stream.
-	 */
-	public FSDataOutputStream create(Path f, WriteOptions options) throws IOException {
-		return create(f, options.getOverwrite());
-	}
-
-	/**
 	 * Renames the file/directory src to dst.
 	 *
 	 * @param src
diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/WriteOptions.java b/flink-core/src/main/java/org/apache/flink/core/fs/WriteOptions.java
deleted file mode 100644
index 70f4973..0000000
--- a/flink-core/src/main/java/org/apache/flink/core/fs/WriteOptions.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.Public;
-import org.apache.flink.core.fs.FileSystem.WriteMode;
-
-import javax.annotation.Nullable;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * Write options that can be passed to the methods that write files.
- */
-@Public
-public class WriteOptions {
-
-	private WriteMode overwrite = WriteMode.NO_OVERWRITE;
-
-	@Nullable
-	private BlockOptions blockSettings;
-
-	private boolean injectEntropy;
-
-	// ------------------------------------------------------------------------
-	//  getters & setters
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Gets the overwrite option.
-	 */
-	public WriteMode getOverwrite() {
-		return overwrite;
-	}
-
-	/**
-	 * Sets the overwrite option.
-	 *
-	 * <p>Method returns this object for fluent function call chaining.
-	 */
-	public WriteOptions setOverwrite(WriteMode overwrite) {
-		this.overwrite = checkNotNull(overwrite);
-		return this;
-	}
-
-	/**
-	 * Gets the block writing settings, like size and replication factor.
-	 * Returns null if no settings are defined.
-	 */
-	@Nullable
-	public BlockOptions getBlockSettings() {
-		return blockSettings;
-	}
-
-	/**
-	 * Sets the block settings, for file systems working with block replication and
-	 * exposing those settings
-	 *
-	 * <p>Method returns this object for fluent function call chaining.
-	 */
-	public WriteOptions setBlockSettings(@Nullable BlockOptions blockSettings) {
-		this.blockSettings = blockSettings;
-		return this;
-	}
-
-	/**
-	 * Gets whether to inject entropy into the path.
-	 */
-	public boolean isInjectEntropy() {
-		return injectEntropy;
-	}
-
-	/**
-	 * Sets whether to inject entropy into the path.
-	 *
-	 * <p>Entropy injection is only supported select filesystems like S3 to overcome
-	 * scalability issues in the sharding. For this option to have any effect, the
-	 * file system must be configured to replace an entropy key with entropy, and the
-	 * path that is written to must contain the entropy key.
-	 *
-	 * <p>Method returns this object for fluent function call chaining.
-	 */
-	public WriteOptions setInjectEntropy(boolean injectEntropy) {
-		this.injectEntropy = injectEntropy;
-		return this;
-	}
-
-	// ------------------------------------------------------------------------
-	//  nested options classes
-	// ------------------------------------------------------------------------
-
-	/**
-	 * Settings for block replication. Interpreted only by filesystems that are based
-	 * expose block replication settings.
-	 */
-	@Public
-	public static class BlockOptions {
-
-		/** The size of the blocks, in bytes. */
-		private long blockSize;
-
-		/** The number of times the block should be replicated. */
-		private int replicationFactor;
-
-		public BlockOptions(long blockSize, int replicationFactor) {
-			checkArgument(blockSize > 0, "blockSize must be >0");
-			checkArgument(replicationFactor > 0, "replicationFactor must be >=1");
-
-			this.blockSize = blockSize;
-			this.replicationFactor = replicationFactor;
-		}
-
-		/**
-		 * Gets the block size, in bytes.
-		 */
-		public long getBlockSize() {
-			return blockSize;
-		}
-
-		/**
-		 * Gets the number of times the block should be replicated.
-		 */
-		public int getReplicationFactor() {
-			return replicationFactor;
-		}
-	}
-}
diff --git a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
index c3b3808..208a301 100644
--- a/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/StringUtils.java
@@ -30,7 +30,6 @@ import java.io.IOException;
 import java.util.Arrays;
 import java.util.Random;
 
-import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -248,37 +247,6 @@ public final class StringUtils {
 	}
 
 	/**
-	 * Appends a random alphanumeric string of given length to the given string buffer.
-	 *
-	 * @param rnd The random number generator to use.
-	 * @param buffer The buffer to append to.
-	 * @param length The number of alphanumeric characters to append.
-	 */
-	public static void appendRandomAlphanumericString(Random rnd, StringBuilder buffer, int length) {
-		checkNotNull(rnd);
-		checkArgument(length >= 0);
-
-		for (int i = 0; i < length; i++) {
-			buffer.append(nextAlphanumericChar(rnd));
-		}
-	}
-
-	private static char nextAlphanumericChar(Random rnd) {
-		int which = rnd.nextInt(62);
-		char c;
-		if (which < 10) {
-			c = (char) ('0' + which);
-		}
-		else if (which < 36) {
-			c = (char) ('A' - 10 + which);
-		}
-		else {
-			c = (char) ('a' - 36 + which);
-		}
-		return c;
-	}
-
-	/**
 	 * Writes a String to the given output.
 	 * The written string can be read with {@link #readString(DataInputView)}.
 	 *
diff --git a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
index 1c9abf2..5f705b4 100644
--- a/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/StringUtilsTest.java
@@ -20,11 +20,8 @@ package org.apache.flink.util;
 
 import org.junit.Test;
 
-import java.util.Random;
-
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
 
 /**
  * Tests for the {@link StringUtils}.
@@ -59,15 +56,4 @@ public class StringUtilsTest extends TestLogger {
 		String hex = StringUtils.byteToHexString(byteArray);
 		assertEquals("019f314a", hex);
 	}
-
-	@Test
-	public void testAppendAlphanumeric() {
-		StringBuilder bld = new StringBuilder();
-		StringUtils.appendRandomAlphanumericString(new Random(), bld, 256);
-		String str = bld.toString();
-
-		if (!str.matches("[a-zA-Z0-9]+")) {
-			fail("Not alphanumeric: " + str);
-		}
-	}
 }
diff --git a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
index bceed5e..065ba5a 100644
--- a/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
+++ b/flink-filesystems/flink-hadoop-fs/src/main/java/org/apache/flink/runtime/fs/hdfs/HadoopFileSystem.java
@@ -19,20 +19,16 @@
 package org.apache.flink.runtime.fs.hdfs;
 
 import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileStatus;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.FileSystemKind;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.core.fs.RecoverableWriter;
-import org.apache.flink.core.fs.WriteOptions;
-import org.apache.flink.core.fs.WriteOptions.BlockOptions;
 
 import java.io.IOException;
 import java.net.URI;
 import java.util.Locale;
 
-import static org.apache.flink.util.Preconditions.checkArgument;
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
@@ -40,9 +36,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
  */
 public class HadoopFileSystem extends FileSystem {
 
-	/** The write buffer size used by default. */
-	public static final int DEFAULT_WRITE_BUFFER_SIZE = 4096;
-
 	/** The wrapped Hadoop File System. */
 	private final org.apache.hadoop.fs.FileSystem fs;
 
@@ -150,24 +143,6 @@ public class HadoopFileSystem extends FileSystem {
 	}
 
 	@Override
-	public FSDataOutputStream create(Path f, WriteOptions options) throws IOException {
-		BlockOptions blockSettings = options.getBlockSettings();
-		if (blockSettings == null) {
-			return create(f, options.getOverwrite());
-		}
-		else {
-			checkArgument(blockSettings.getReplicationFactor() <= Short.MAX_VALUE,
-					"block replication factor out of bounds");
-
-			return create(f,
-					options.getOverwrite() == WriteMode.OVERWRITE,
-					DEFAULT_WRITE_BUFFER_SIZE,
-					(short) blockSettings.getReplicationFactor(),
-					blockSettings.getBlockSize());
-		}
-	}
-
-	@Override
 	public boolean delete(final Path f, final boolean recursive) throws IOException {
 		return this.fs.delete(toHadoopPath(f), recursive);
 	}
diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
index 230d18b..a04f9c9 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
+++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
@@ -19,20 +19,12 @@
 package org.apache.flink.fs.s3presto;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.IllegalConfigurationException;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.FileSystemFactory;
+import org.apache.flink.runtime.fs.hdfs.AbstractFileSystemFactory;
 import org.apache.flink.runtime.fs.hdfs.HadoopConfigLoader;
 import org.apache.flink.util.FlinkRuntimeException;
 
 import com.facebook.presto.hive.PrestoS3FileSystem;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Collections;
@@ -42,31 +34,7 @@ import java.util.Set;
 /**
  * Simple factory for the S3 file system.
  */
-public class S3FileSystemFactory implements FileSystemFactory {
-
-	/**
-	 * The substring to be replaced by random entropy in checkpoint paths.
-	 */
-	public static final ConfigOption<String> ENTROPY_INJECT_KEY_OPTION = ConfigOptions
-			.key("s3.entropy.key")
-			.noDefaultValue()
-			.withDescription(
-					"This option can be used to improve performance due to sharding issues on Amazon S3. " +
-					"For file creations with entropy injection, this key will be replaced by random " +
-					"alphanumeric characters. For other file creations, the key will be filtered out.");
-
-	/**
-	 * The number of entropy characters, in case entropy injection is configured.
-	 */
-	public static final ConfigOption<Integer> ENTROPY_INJECT_LENGTH_OPTION = ConfigOptions
-			.key("s3.entropy.length")
-			.defaultValue(4)
-			.withDescription(
-					"When '" + ENTROPY_INJECT_KEY_OPTION.key() + "' is set, this option defines the number of " +
-					"random characters to replace the entropy key with.");
-
-	private static final Logger LOG = LoggerFactory.getLogger(S3FileSystemFactory.class);
-
+public class S3FileSystemFactory extends AbstractFileSystemFactory {
 	private static final Set<String> PACKAGE_PREFIXES_TO_SHADE =
 		new HashSet<>(Collections.singletonList("com.amazonaws."));
 
@@ -82,55 +50,8 @@ public class S3FileSystemFactory implements FileSystemFactory {
 			{ "presto.s3.secret.key", "presto.s3.secret-key" }
 	};
 
-	private static final String INVALID_ENTROPY_KEY_CHARS = "^.*[~#@*+%{}<>\\[\\]|\"\\\\].*$";
-
-	private final HadoopConfigLoader hadoopConfigLoader = createHadoopConfigLoader();
-
-	private Configuration flinkConfig;
-
-	@Override
-	public void configure(Configuration config) {
-		flinkConfig = config;
-		hadoopConfigLoader.setFlinkConfig(config);
-	}
-
-	@Override
-	public FileSystem create(URI fsUri) throws IOException {
-		LOG.debug("Creating S3 FileSystem backed by Presto S3 FileSystem");
-		LOG.debug("Loading Hadoop configuration for Presto S3 File System");
-
-		try {
-			// instantiate the presto file system
-			org.apache.hadoop.conf.Configuration hadoopConfig = hadoopConfigLoader.getOrLoadHadoopConfig();
-			org.apache.hadoop.fs.FileSystem fs = new PrestoS3FileSystem();
-			fs.initialize(getInitURI(fsUri, hadoopConfig), hadoopConfig);
-
-			// load the entropy injection settings
-			String entropyInjectionKey = flinkConfig.getString(ENTROPY_INJECT_KEY_OPTION);
-			int numEntropyChars = -1;
-
-			if (entropyInjectionKey != null) {
-				if (entropyInjectionKey.matches(INVALID_ENTROPY_KEY_CHARS)) {
-					throw new IllegalConfigurationException("Invalid character in value for " +
-							ENTROPY_INJECT_KEY_OPTION.key() + " : " + entropyInjectionKey);
-				}
-
-				numEntropyChars = flinkConfig.getInteger(ENTROPY_INJECT_LENGTH_OPTION);
-
-				if (numEntropyChars <= 0) {
-					throw new IllegalConfigurationException(
-							ENTROPY_INJECT_LENGTH_OPTION.key() + " must configure a value > 0");
-				}
-			}
-
-			return new S3PrestoFileSystem(fs, entropyInjectionKey, numEntropyChars);
-		}
-		catch (IOException e) {
-			throw e;
-		}
-		catch (Exception e) {
-			throw new IOException(e.getMessage(), e);
-		}
+	public S3FileSystemFactory() {
+		super("Presto S3 File System", createHadoopConfigLoader());
 	}
 
 	@Override
@@ -144,7 +65,13 @@ public class S3FileSystemFactory implements FileSystemFactory {
 			"presto.s3.", PACKAGE_PREFIXES_TO_SHADE, CONFIG_KEYS_TO_SHADE, FLINK_SHADING_PREFIX);
 	}
 
-	static URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig) {
+	@Override
+	protected org.apache.hadoop.fs.FileSystem createHadoopFileSystem() {
+		return new PrestoS3FileSystem();
+	}
+
+	@Override
+	protected URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig) {
 		final String scheme = fsUri.getScheme();
 		final String authority = fsUri.getAuthority();
 		final URI initUri;
@@ -161,11 +88,10 @@ public class S3FileSystemFactory implements FileSystemFactory {
 		return initUri;
 	}
 
-	static URI createURI(String str) {
+	private URI createURI(String str) {
 		try {
 			return new URI(str);
-		}
-		catch (URISyntaxException e) {
+		} catch (URISyntaxException e) {
 			throw new FlinkRuntimeException("Error in s3 aws URI - " + str, e);
 		}
 	}
diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java
deleted file mode 100644
index e6a6ae4..0000000
--- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java
+++ /dev/null
@@ -1,301 +0,0 @@
-/*
- * 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.fs.s3presto;
-
-import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.core.fs.BlockLocation;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.FileSystemKind;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.WriteOptions;
-import org.apache.flink.runtime.fs.hdfs.HadoopBlockLocation;
-import org.apache.flink.runtime.fs.hdfs.HadoopDataInputStream;
-import org.apache.flink.runtime.fs.hdfs.HadoopDataOutputStream;
-import org.apache.flink.runtime.fs.hdfs.HadoopFileStatus;
-import org.apache.flink.util.StringUtils;
-
-import javax.annotation.Nullable;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.concurrent.ThreadLocalRandom;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A Flink FileSystem against S3, wrapping the Presto Hadoop S3 File System implementation.
- *
- * <p>This class bases heavily on the {@link org.apache.flink.runtime.fs.hdfs.HadoopFileSystem} class.
- * Code is copied here for the sake of minimal changes to the original class within a minor release.
- */
-class S3PrestoFileSystem extends FileSystem {
-
-	/** The wrapped Hadoop File System. */
-	private final org.apache.hadoop.fs.FileSystem fs;
-
-	@Nullable
-	private final String entropyInjectionKey;
-
-	private final int entropyLength;
-
-	/**
-	 * Wraps the given Hadoop File System object as a Flink File System object.
-	 * The given Hadoop file system object is expected to be initialized already.
-	 *
-	 * @param hadoopFileSystem The Hadoop FileSystem that will be used under the hood.
-	 */
-	public S3PrestoFileSystem(org.apache.hadoop.fs.FileSystem hadoopFileSystem) {
-		this(hadoopFileSystem, null, -1);
-	}
-
-	/**
-	 * Wraps the given Hadoop File System object as a Flink File System object.
-	 * The given Hadoop file system object is expected to be initialized already.
-	 *
-	 * <p>This constructor additionally configures the entropy injection for the file system.
-	 *
-	 * @param hadoopFileSystem The Hadoop FileSystem that will be used under the hood.
-	 * @param entropyInjectionKey The substring that will be replaced by entropy or removed.
-	 * @param entropyLength The number of random alphanumeric characters to inject as entropy.
-	 */
-	public S3PrestoFileSystem(
-			org.apache.hadoop.fs.FileSystem hadoopFileSystem,
-			@Nullable String entropyInjectionKey,
-			int entropyLength) {
-
-		if (entropyInjectionKey != null && entropyLength <= 0) {
-			throw new IllegalArgumentException("Entropy length must be >= 0 when entropy injection key is set");
-		}
-
-		this.fs = checkNotNull(hadoopFileSystem, "hadoopFileSystem");
-		this.entropyInjectionKey = entropyInjectionKey;
-		this.entropyLength = entropyLength;
-	}
-
-	// ------------------------------------------------------------------------
-	//  properties
-	// ------------------------------------------------------------------------
-
-	public org.apache.hadoop.fs.FileSystem getHadoopFileSystem() {
-		return fs;
-	}
-
-	@Nullable
-	public String getEntropyInjectionKey() {
-		return entropyInjectionKey;
-	}
-
-	public int getEntropyLength() {
-		return entropyLength;
-	}
-
-	// ------------------------------------------------------------------------
-	//  file system methods
-	// ------------------------------------------------------------------------
-
-	@Override
-	public Path getWorkingDirectory() {
-		return new Path(fs.getWorkingDirectory().toUri());
-	}
-
-	public Path getHomeDirectory() {
-		return new Path(fs.getHomeDirectory().toUri());
-	}
-
-	@Override
-	public URI getUri() {
-		return fs.getUri();
-	}
-
-	@Override
-	public FileStatus getFileStatus(final Path f) throws IOException {
-		org.apache.hadoop.fs.FileStatus status = fs.getFileStatus(toHadoopPath(f));
-		return new HadoopFileStatus(status);
-	}
-
-	@Override
-	public BlockLocation[] getFileBlockLocations(
-			final FileStatus file,
-			final long start,
-			final long len) throws IOException {
-
-		if (!(file instanceof HadoopFileStatus)) {
-			throw new IOException("file is not an instance of HadoopFileStatus");
-		}
-
-		final HadoopFileStatus f = (HadoopFileStatus) file;
-
-		final org.apache.hadoop.fs.BlockLocation[] blkLocations = fs.getFileBlockLocations(f.getInternalFileStatus(),
-				start, len);
-
-		// Wrap up HDFS specific block location objects
-		final HadoopBlockLocation[] distBlkLocations = new HadoopBlockLocation[blkLocations.length];
-		for (int i = 0; i < distBlkLocations.length; i++) {
-			distBlkLocations[i] = new HadoopBlockLocation(blkLocations[i]);
-		}
-
-		return distBlkLocations;
-	}
-
-	@Override
-	public HadoopDataInputStream open(final Path f, final int bufferSize) throws IOException {
-		final org.apache.hadoop.fs.Path path = toHadoopPath(f);
-		final org.apache.hadoop.fs.FSDataInputStream fdis = this.fs.open(path, bufferSize);
-		return new HadoopDataInputStream(fdis);
-	}
-
-	@Override
-	public HadoopDataInputStream open(final Path f) throws IOException {
-		final org.apache.hadoop.fs.Path path = toHadoopPath(f);
-		final org.apache.hadoop.fs.FSDataInputStream fdis = fs.open(path);
-		return new HadoopDataInputStream(fdis);
-	}
-
-	@Override
-	public HadoopDataOutputStream create(final Path f, final WriteMode overwrite) throws IOException {
-		final org.apache.hadoop.fs.FSDataOutputStream fsDataOutputStream =
-				fs.create(toHadoopPath(f), overwrite == WriteMode.OVERWRITE);
-		return new HadoopDataOutputStream(fsDataOutputStream);
-	}
-
-	@Override
-	public FSDataOutputStream create(final Path f, final WriteOptions options) throws IOException {
-		final org.apache.hadoop.fs.Path path = options.isInjectEntropy()
-				? toHadoopPathInjectEntropy(f)
-				: toHadoopPath(f);
-
-		final org.apache.hadoop.fs.FSDataOutputStream fsDataOutputStream = fs.create(
-				path, options.getOverwrite() == WriteMode.OVERWRITE);
-
-		return new HadoopDataOutputStream(fsDataOutputStream);
-	}
-
-	@Override
-	public boolean delete(final Path f, final boolean recursive) throws IOException {
-		return fs.delete(toHadoopPath(f), recursive);
-	}
-
-	@Override
-	public boolean exists(Path f) throws IOException {
-		return fs.exists(toHadoopPath(f));
-	}
-
-	@Override
-	public FileStatus[] listStatus(final Path f) throws IOException {
-		final org.apache.hadoop.fs.FileStatus[] hadoopFiles = fs.listStatus(toHadoopPath(f));
-		final FileStatus[] files = new FileStatus[hadoopFiles.length];
-
-		// Convert types
-		for (int i = 0; i < files.length; i++) {
-			files[i] = new HadoopFileStatus(hadoopFiles[i]);
-		}
-
-		return files;
-	}
-
-	@Override
-	public boolean mkdirs(final Path f) throws IOException {
-		return fs.mkdirs(toHadoopPath(f));
-	}
-
-	@Override
-	public boolean rename(final Path src, final Path dst) throws IOException {
-		return fs.rename(toHadoopPath(src), toHadoopPath(dst));
-	}
-
-	@SuppressWarnings("deprecation")
-	@Override
-	public long getDefaultBlockSize() {
-		return fs.getDefaultBlockSize();
-	}
-
-	@Override
-	public boolean isDistributedFS() {
-		return true;
-	}
-
-	@Override
-	public FileSystemKind getKind() {
-		return FileSystemKind.OBJECT_STORE;
-	}
-
-	// ------------------------------------------------------------------------
-	//  entropy utilities
-	// ------------------------------------------------------------------------
-
-	@VisibleForTesting
-	org.apache.hadoop.fs.Path toHadoopPath(Path path) throws IOException {
-		return rewritePathForEntropyKey(path, false);
-	}
-
-	@VisibleForTesting
-	org.apache.hadoop.fs.Path toHadoopPathInjectEntropy(Path path) throws IOException {
-		return rewritePathForEntropyKey(path, true);
-	}
-
-	private org.apache.hadoop.fs.Path rewritePathForEntropyKey(Path path, boolean addEntropy) throws IOException {
-		if (entropyInjectionKey == null) {
-			return convertToHadoopPath(path);
-		}
-		else {
-			final URI originalUri = path.toUri();
-			final String checkpointPath = originalUri.getPath();
-
-			final int indexOfKey = checkpointPath.indexOf(entropyInjectionKey);
-			if (indexOfKey == -1) {
-				return convertToHadoopPath(path);
-			}
-			else {
-				final StringBuilder buffer = new StringBuilder(checkpointPath.length());
-				buffer.append(checkpointPath, 0, indexOfKey);
-
-				if (addEntropy) {
-					StringUtils.appendRandomAlphanumericString(ThreadLocalRandom.current(), buffer, entropyLength);
-				}
-
-				buffer.append(checkpointPath, indexOfKey + entropyInjectionKey.length(), checkpointPath.length());
-
-				final String rewrittenPath = buffer.toString();
-				try {
-					return convertToHadoopPath(new URI(
-							originalUri.getScheme(),
-							originalUri.getAuthority(),
-							rewrittenPath,
-							originalUri.getQuery(),
-							originalUri.getFragment()));
-				}
-				catch (URISyntaxException e) {
-					// this should actually never happen, because the URI was valid before
-					throw new IOException("URI format error while processing path for entropy injection", e);
-				}
-			}
-		}
-	}
-
-	private static org.apache.hadoop.fs.Path convertToHadoopPath(URI uri) {
-		return new org.apache.hadoop.fs.Path(uri);
-	}
-
-	private static org.apache.hadoop.fs.Path convertToHadoopPath(Path path) {
-		return convertToHadoopPath(path.toUri());
-	}
-}
diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemEntropyTest.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemEntropyTest.java
deleted file mode 100644
index 587b02e..0000000
--- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemEntropyTest.java
+++ /dev/null
@@ -1,133 +0,0 @@
-/*
- * 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.fs.s3presto;
-
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.WriteOptions;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Matchers.anyBoolean;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-/**
- * Tests for the entropy injection in the {@link S3PrestoFileSystem}.
- */
-public class PrestoS3FileSystemEntropyTest {
-
-	@Test
-	public void testEmptyPath() throws Exception {
-		Path path = new Path("hdfs://localhost:12345");
-		S3PrestoFileSystem fs = createFs("test", 4);
-
-		assertEquals(toHadoopPath(path), fs.toHadoopPathInjectEntropy(path));
-		assertEquals(toHadoopPath(path), fs.toHadoopPath(path));
-	}
-
-	@Test
-	public void testFullUriNonMatching() throws Exception {
-		Path path = new Path("s3://hugo@myawesomehost:55522/path/to/the/file");
-		S3PrestoFileSystem fs = createFs("_entropy_key_", 4);
-
-		assertEquals(toHadoopPath(path), fs.toHadoopPathInjectEntropy(path));
-		assertEquals(toHadoopPath(path), fs.toHadoopPath(path));
-	}
-
-	@Test
-	public void testFullUriMatching() throws Exception {
-		Path path = new Path("s3://hugo@myawesomehost:55522/path/s0mek3y/the/file");
-		S3PrestoFileSystem fs = createFs("s0mek3y", 8);
-
-		org.apache.hadoop.fs.Path withEntropy = fs.toHadoopPathInjectEntropy(path);
-		org.apache.hadoop.fs.Path withoutEntropy = fs.toHadoopPath(path);
-
-		validateMatches(withEntropy, "s3://hugo@myawesomehost:55522/path/[a-zA-Z0-9]{8}/the/file");
-		assertEquals(new org.apache.hadoop.fs.Path("s3://hugo@myawesomehost:55522/path/the/file"), withoutEntropy);
-	}
-
-	@Test
-	public void testPathOnlyNonMatching() throws Exception {
-		Path path = new Path("/path/file");
-		S3PrestoFileSystem fs = createFs("_entropy_key_", 4);
-
-		assertEquals(toHadoopPath(path), fs.toHadoopPathInjectEntropy(path));
-		assertEquals(toHadoopPath(path), fs.toHadoopPath(path));
-	}
-
-	@Test
-	public void testPathOnlyMatching() throws Exception {
-		Path path = new Path("/path/_entropy_key_/file");
-		S3PrestoFileSystem fs = createFs("_entropy_key_", 4);
-
-		org.apache.hadoop.fs.Path withEntropy = fs.toHadoopPathInjectEntropy(path);
-		org.apache.hadoop.fs.Path withoutEntropy = fs.toHadoopPath(path);
-
-		validateMatches(withEntropy, "/path/[a-zA-Z0-9]{4}/file");
-		assertEquals(new org.apache.hadoop.fs.Path("/path/file"), withoutEntropy);
-	}
-
-	@Test
-	public void testEntropyNotFullSegment() throws Exception {
-		Path path = new Path("s3://myhost:122/entropy-_entropy_key_-suffix/file");
-		S3PrestoFileSystem fs = createFs("_entropy_key_", 3);
-
-		org.apache.hadoop.fs.Path withEntropy = fs.toHadoopPathInjectEntropy(path);
-		org.apache.hadoop.fs.Path withoutEntropy = fs.toHadoopPath(path);
-
-		validateMatches(withEntropy, "s3://myhost:122/entropy-[a-zA-Z0-9]{3}-suffix/file");
-		assertEquals(new org.apache.hadoop.fs.Path("s3://myhost:122/entropy--suffix/file"), withoutEntropy);
-	}
-
-	@Test
-	public void testWriteOptionWithEntropy() throws Exception {
-		FileSystem underlyingFs = mock(FileSystem.class);
-		when(underlyingFs.create(any(org.apache.hadoop.fs.Path.class), anyBoolean())).thenReturn(mock(FSDataOutputStream.class));
-		ArgumentCaptor<org.apache.hadoop.fs.Path> pathCaptor = ArgumentCaptor.forClass(org.apache.hadoop.fs.Path.class);
-
-		Path path = new Path("s3://hugo@myawesomehost:55522/path/s0mek3y/the/file");
-		S3PrestoFileSystem fs = new S3PrestoFileSystem(underlyingFs, "s0mek3y", 11);
-
-		fs.create(path, new WriteOptions().setInjectEntropy(true));
-		verify(underlyingFs).create(pathCaptor.capture(), anyBoolean());
-
-		validateMatches(pathCaptor.getValue(), "s3://hugo@myawesomehost:55522/path/[a-zA-Z0-9]{11}/the/file");
-	}
-
-	private static void validateMatches(org.apache.hadoop.fs.Path path, String pattern) {
-		if (!path.toString().matches(pattern)) {
-			fail("Path " + path + " does not match " + pattern);
-		}
-	}
-
-	private static S3PrestoFileSystem createFs(String entropyKey, int entropyLen) {
-		return new S3PrestoFileSystem(mock(FileSystem.class), entropyKey, entropyLen);
-	}
-
-	private org.apache.hadoop.fs.Path toHadoopPath(Path path) {
-		return new org.apache.hadoop.fs.Path(path.toUri());
-	}
-}
diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java
index 9afad57..4eeb2d4 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java
+++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3FileSystemTest.java
@@ -21,12 +21,12 @@ package org.apache.flink.fs.s3presto;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.runtime.fs.hdfs.HadoopConfigLoader;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
 
 import com.amazonaws.auth.AWSCredentialsProvider;
 import com.amazonaws.auth.AWSStaticCredentialsProvider;
 import com.amazonaws.services.s3.AmazonS3Client;
 import com.facebook.presto.hive.PrestoS3FileSystem;
-import org.junit.After;
 import org.junit.Test;
 
 import java.lang.reflect.Field;
@@ -41,11 +41,6 @@ import static org.junit.Assert.assertTrue;
  */
 public class PrestoS3FileSystemTest {
 
-	@After
-	public void resetFileSystemConfig() throws Exception {
-		FileSystem.initialize(new Configuration());
-	}
-
 	@Test
 	public void testConfigPropagation() throws Exception{
 		final Configuration conf = new Configuration();
@@ -95,29 +90,14 @@ public class PrestoS3FileSystemTest {
 			hadoopConfig.get("presto.s3.credentials-provider"));
 	}
 
-	@Test
-	public void testEntropyInjectionConfig() throws Exception {
-		final Configuration conf = new Configuration();
-		conf.setString("s3.entropy.key", "__entropy__");
-		conf.setInteger("s3.entropy.length", 7);
-
-		FileSystem.initialize(conf);
-
-		FileSystem fs = FileSystem.get(new URI("s3://test"));
-		S3PrestoFileSystem s3fs = (S3PrestoFileSystem) fs;
-
-		assertEquals("__entropy__", s3fs.getEntropyInjectionKey());
-		assertEquals(7, s3fs.getEntropyLength());
-	}
-
 	// ------------------------------------------------------------------------
 	//  utilities
 	// ------------------------------------------------------------------------
 
 	private static void validateBasicCredentials(FileSystem fs) throws Exception {
-		assertTrue(fs instanceof S3PrestoFileSystem);
+		assertTrue(fs instanceof HadoopFileSystem);
 
-		org.apache.hadoop.fs.FileSystem hadoopFs = ((S3PrestoFileSystem) fs).getHadoopFileSystem();
+		org.apache.hadoop.fs.FileSystem hadoopFs = ((HadoopFileSystem) fs).getHadoopFileSystem();
 		assertTrue(hadoopFs instanceof PrestoS3FileSystem);
 
 		try (PrestoS3FileSystem prestoFs = (PrestoS3FileSystem) hadoopFs) {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
index 1549e01..af80af7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
@@ -32,7 +32,6 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
-import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * An implementation of durable checkpoint storage to file systems.
@@ -55,25 +54,11 @@ public class FsCheckpointStorage extends AbstractFsCheckpointStorage {
 			JobID jobId,
 			int fileSizeThreshold) throws IOException {
 
-		this(checkpointBaseDirectory.getFileSystem(),
-				checkpointBaseDirectory,
-				defaultSavepointDirectory,
-				jobId,
-				fileSizeThreshold);
-	}
-
-	public FsCheckpointStorage(
-			FileSystem fs,
-			Path checkpointBaseDirectory,
-			@Nullable Path defaultSavepointDirectory,
-			JobID jobId,
-			int fileSizeThreshold) throws IOException {
-
 		super(jobId, defaultSavepointDirectory);
 
 		checkArgument(fileSizeThreshold >= 0);
 
-		this.fileSystem = checkNotNull(fs);
+		this.fileSystem = checkpointBaseDirectory.getFileSystem();
 		this.checkpointsDirectory = getCheckpointDirectoryForJob(checkpointBaseDirectory, jobId);
 		this.sharedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_SHARED_STATE_DIR);
 		this.taskOwnedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_TASK_OWNED_STATE_DIR);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
index 228c5b4..609ef69 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
@@ -21,7 +21,6 @@ package org.apache.flink.runtime.state.filesystem;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.WriteOptions;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.CheckpointedStateScope;
 import org.apache.flink.runtime.state.StreamStateHandle;
@@ -346,11 +345,7 @@ public class FsCheckpointStreamFactory implements CheckpointStreamFactory {
 			for (int attempt = 0; attempt < 10; attempt++) {
 				try {
 					Path statePath = createStatePath();
-					FSDataOutputStream outStream = fs.create(
-							statePath,
-							new WriteOptions()
-									.setOverwrite(FileSystem.WriteMode.NO_OVERWRITE)
-									.setInjectEntropy(true));
+					FSDataOutputStream outStream = fs.create(statePath, FileSystem.WriteMode.NO_OVERWRITE);
 
 					// success, managed to open the stream
 					this.statePath = statePath;
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
index c962146..8bafdf7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStateOutputStreamTest.java
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.ConfigConstants;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.WriteOptions;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
 import org.apache.flink.runtime.state.StreamStateHandle;
@@ -139,13 +138,13 @@ public class FsCheckpointStateOutputStreamTest {
 	 */
 	@Test
 	public void testCleanupWhenClosingStream() throws IOException {
+
 		final FileSystem fs = mock(FileSystem.class);
 		final FSDataOutputStream outputStream = mock(FSDataOutputStream.class);
 
 		final ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
 
 		when(fs.create(pathCaptor.capture(), any(FileSystem.WriteMode.class))).thenReturn(outputStream);
-		when(fs.create(pathCaptor.capture(), any(WriteOptions.class))).thenReturn(outputStream);
 
 		CheckpointStreamFactory.CheckpointStateOutputStream stream = new FsCheckpointStreamFactory.FsCheckpointStateOutputStream(
 			Path.fromLocalFile(tempDir.newFolder()),
@@ -155,6 +154,9 @@ public class FsCheckpointStateOutputStreamTest {
 
 		// this should create the underlying file stream
 		stream.write(new byte[] {1, 2, 3, 4, 5});
+
+		verify(fs).create(any(Path.class), any(FileSystem.WriteMode.class));
+
 		stream.close();
 
 		verify(fs).delete(eq(pathCaptor.getValue()), anyBoolean());
@@ -168,11 +170,9 @@ public class FsCheckpointStateOutputStreamTest {
 		final FileSystem fs = mock(FileSystem.class);
 		final FSDataOutputStream outputStream = mock(FSDataOutputStream.class);
 
-		final ArgumentCaptor<Path> pathCaptor = ArgumentCaptor.forClass(Path.class);
+		final ArgumentCaptor<Path>  pathCaptor = ArgumentCaptor.forClass(Path.class);
 
 		when(fs.create(pathCaptor.capture(), any(FileSystem.WriteMode.class))).thenReturn(outputStream);
-		when(fs.create(pathCaptor.capture(), any(WriteOptions.class))).thenReturn(outputStream);
-
 		doThrow(new IOException("Test IOException.")).when(outputStream).close();
 
 		CheckpointStreamFactory.CheckpointStateOutputStream stream = new FsCheckpointStreamFactory.FsCheckpointStateOutputStream(
@@ -184,6 +184,8 @@ public class FsCheckpointStateOutputStreamTest {
 		// this should create the underlying file stream
 		stream.write(new byte[] {1, 2, 3, 4, 5});
 
+		verify(fs).create(any(Path.class), any(FileSystem.WriteMode.class));
+
 		try {
 			stream.closeAndGetHandle();
 			fail("Expected IOException");
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
deleted file mode 100644
index d1aa118..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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.state.filesystem;
-
-import org.apache.flink.api.common.JobID;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.core.fs.WriteOptions;
-import org.apache.flink.core.fs.local.LocalFileSystem;
-import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
-import org.apache.flink.runtime.state.CheckpointStorageLocation;
-import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
-import org.apache.flink.runtime.state.CheckpointedStateScope;
-
-import org.junit.After;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.mockito.ArgumentCaptor;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.reset;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-
-/**
- * Tests verifying that the FsStateBackend passes the entropy injection option
- * to the FileSystem for state payload files, but not for metadata files.
- */
-public class FsStateBackendEntropyTest {
-
-	@Rule
-	public final TemporaryFolder tmp = new TemporaryFolder();
-
-	@After
-	public void resetFileSystems() throws Exception {
-		FileSystem.initialize(new Configuration());
-	}
-
-	@Test
-	public void testInjection() throws Exception {
-		FileSystem fs = spy(LocalFileSystem.getSharedInstance());
-		ArgumentCaptor<WriteOptions> optionsCaptor = ArgumentCaptor.forClass(WriteOptions.class);
-
-		Path checkpointDir = Path.fromLocalFile(tmp.newFolder());
-
-		FsCheckpointStorage storage = new FsCheckpointStorage(
-				fs, checkpointDir, null, new JobID(), 1024);
-
-		CheckpointStorageLocation location = storage.initializeLocationForCheckpoint(96562);
-
-		// check entropy in task-owned state
-		try (CheckpointStateOutputStream stream = storage.createTaskOwnedStateStream()) {
-			stream.flush();
-		}
-
-		verify(fs, times(1)).create(any(Path.class), optionsCaptor.capture());
-		assertTrue(optionsCaptor.getValue().isInjectEntropy());
-		reset(fs);
-
-		// check entropy in the exclusive/shared state
-		try (CheckpointStateOutputStream stream =
-				location.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE)) {
-
-			stream.flush();
-		}
-
-		verify(fs, times(1)).create(any(Path.class), optionsCaptor.capture());
-		assertTrue(optionsCaptor.getValue().isInjectEntropy());
-		reset(fs);
-
-		// check that there is no entropy in the metadata
-		// check entropy in the exclusive/shared state
-		try (CheckpointMetadataOutputStream stream = location.createMetadataOutputStream()) {
-			stream.flush();
-		}
-
-		verify(fs, times(0)).create(any(Path.class), any(WriteOptions.class));
-	}
-}


[flink] 06/06: [FLINK-10383] [s3] Prevent Hadoop configs on classpath to interfere with S3 configuration

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 67fa338c2f4ff7cc5d28f9eac90f170aed7cefcc
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Thu Sep 20 20:23:04 2018 +0200

    [FLINK-10383] [s3] Prevent Hadoop configs on classpath to interfere with S3 configuration
    
    The S3 connectors are based on a self-contained shaded Hadoop and should not load implicitly
    configs from the classpath, like Hadoop does it. Instead, they should only use config value
    from the Flink configuration.
---
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../src/test/resources/core-default-testing.xml}   |    0
 .../src/test/resources/core-site.xml               | 2323 +-------------------
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../{core-site.xml => core-default-testing.xml}    |    0
 .../src/test/resources/core-site.xml               | 1989 +----------------
 .../java/org/apache/hadoop/conf/Configuration.java |   16 +-
 .../{core-site.xml => core-default-testing.xml}    |    0
 .../src/test/resources/core-site.xml               | 2323 +-------------------
 9 files changed, 69 insertions(+), 6614 deletions(-)

diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
index 86822de..104a3ee 100644
--- a/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -652,21 +652,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
     }
   }
  
-  static{
-    //print deprecation warning if hadoop-site.xml is found in classpath
-    ClassLoader cL = Thread.currentThread().getContextClassLoader();
-    if (cL == null) {
-      cL = Configuration.class.getClassLoader();
-    }
-    if(cL.getResource("hadoop-site.xml")!=null) {
-      LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
-          "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
-          + "mapred-site.xml and hdfs-site.xml to override properties of " +
-          "core-default-shaded.xml, mapred-default.xml and hdfs-default.xml " +
-          "respectively");
-    }
+  static {
     addDefaultResource("core-default-shaded.xml");
-    addDefaultResource("core-site.xml");
+    addDefaultResource("core-default-testing.xml");
   }
   
   private Properties properties;
diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-default-testing.xml
similarity index 100%
copy from flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml
copy to flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-default-testing.xml
diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml
index b875e97..bddcdc8 100644
--- a/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml
+++ b/flink-filesystems/flink-s3-fs-hadoop/src/test/resources/core-site.xml
@@ -1,2312 +1,31 @@
 <?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
 <!--
-   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
+  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.
+  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.
 -->
 
-<!-- Do not modify this file directly.  Instead, copy entries that you -->
-<!-- wish to modify from this file into core-site.xml and change them -->
-<!-- there.  If core-site.xml does not already exist, create it.      -->
+<!--
+A configuration as it may accidentally be in the classpath when Hadoop is
+in the classpath. Used for tests checking that the presence of such a config
+does not pollute the settings.
+-->
 
 <configuration>
-
-<!--- global properties -->
-
-<property>
-  <name>hadoop.common.configuration.version</name>
-  <value>0.23.0</value>
-  <description>version of this configuration file</description>
-</property>
-
-<property>
-  <name>hadoop.tmp.dir</name>
-  <value>/tmp/hadoop-${user.name}</value>
-  <description>A base for other temporary directories.</description>
-</property>
-
-<property>
-  <name>io.native.lib.available</name>
-  <value>true</value>
-  <description>Controls whether to use native libraries for bz2 and zlib
-    compression codecs or not. The property does not control any other native
-    libraries.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.filter.initializers</name>
-  <value>org.apache.hadoop.http.lib.StaticUserWebFilter</value>
-  <description>A comma separated list of class names. Each class in the list
-  must extend org.apache.hadoop.http.FilterInitializer. The corresponding
-  Filter will be initialized. Then, the Filter will be applied to all user
-  facing jsp and servlet web pages.  The ordering of the list defines the
-  ordering of the filters.</description>
-</property>
-
-<!--- security properties -->
-
-<property>
-  <name>hadoop.security.authorization</name>
-  <value>false</value>
-  <description>Is service-level authorization enabled?</description>
-</property>
-
-<property>
-  <name>hadoop.security.instrumentation.requires.admin</name>
-  <value>false</value>
-  <description>
-    Indicates if administrator ACLs are required to access
-    instrumentation servlets (JMX, METRICS, CONF, STACKS).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.authentication</name>
-  <value>simple</value>
-  <description>Possible values are simple (no authentication), and kerberos
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping</name>
-  <value>org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback</value>
-  <description>
-    Class for user to group mapping (get groups for a given user) for ACL.
-    The default implementation,
-    org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback,
-    will determine if the Java Native Interface (JNI) is available. If JNI is
-    available the implementation will use the API within hadoop to resolve a
-    list of groups for a user. If JNI is not available then the shell
-    implementation, ShellBasedUnixGroupsMapping, is used.  This implementation
-    shells out to the Linux/Unix environment with the
-    <code>bash -c groups</code> command to resolve a list of groups for a user.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.interface</name>
-  <description>
-    The name of the Network Interface from which the service should determine
-    its host name for Kerberos login. e.g. eth2. In a multi-homed environment,
-    the setting can be used to affect the _HOST subsitution in the service
-    Kerberos principal. If this configuration value is not set, the service
-    will use its default hostname as returned by
-    InetAddress.getLocalHost().getCanonicalHostName().
-
-    Most clusters will not require this setting.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.nameserver</name>
-  <description>
-    The host name or IP address of the name server (DNS) which a service Node
-    should use to determine its own host name for Kerberos Login. Requires
-    hadoop.security.dns.interface.
-
-    Most clusters will not require this setting.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.log-slow-lookups.enabled</name>
-  <value>false</value>
-  <description>
-    Time name lookups (via SecurityUtil) and log them if they exceed the
-    configured threshold.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.log-slow-lookups.threshold.ms</name>
-  <value>1000</value>
-  <description>
-    If slow lookup logging is enabled, this threshold is used to decide if a
-    lookup is considered slow enough to be logged.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.secs</name>
-  <value>300</value>
-  <description>
-    This is the config controlling the validity of the entries in the cache
-    containing the user->group mapping. When this duration has expired,
-    then the implementation of the group mapping provider is invoked to get
-    the groups of the user and then cached back.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.negative-cache.secs</name>
-  <value>30</value>
-  <description>
-    Expiration time for entries in the the negative user-to-group mapping
-    caching, in seconds. This is useful when invalid users are retrying
-    frequently. It is suggested to set a small value for this expiration, since
-    a transient error in group lookup could temporarily lock out a legitimate
-    user.
-
-    Set this to zero or negative value to disable negative user-to-group caching.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.warn.after.ms</name>
-  <value>5000</value>
-  <description>
-    If looking up a single user to group takes longer than this amount of
-    milliseconds, we will log a warning message.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.background.reload</name>
-  <value>false</value>
-  <description>
-    Whether to reload expired user->group mappings using a background thread
-    pool. If set to true, a pool of
-    hadoop.security.groups.cache.background.reload.threads is created to
-    update the cache in the background.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.background.reload.threads</name>
-  <value>3</value>
-  <description>
-    Only relevant if hadoop.security.groups.cache.background.reload is true.
-    Controls the number of concurrent background user->group cache entry
-    refreshes. Pending refresh requests beyond this value are queued and
-    processed when a thread is free.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.connection.timeout.ms</name>
-  <value>60000</value>
-  <description>
-    This property is the connection timeout (in milliseconds) for LDAP
-    operations. If the LDAP provider doesn't establish a connection within the
-    specified period, it will abort the connect attempt. Non-positive value
-    means no LDAP connection timeout is specified in which case it waits for the
-    connection to establish until the underlying network times out.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.read.timeout.ms</name>
-  <value>60000</value>
-  <description>
-    This property is the read timeout (in milliseconds) for LDAP
-    operations. If the LDAP provider doesn't get a LDAP response within the
-    specified period, it will abort the read attempt. Non-positive value
-    means no read timeout is specified in which case it waits for the response
-    infinitely.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.url</name>
-  <value></value>
-  <description>
-    The URL of the LDAP server to use for resolving user groups when using
-    the LdapGroupsMapping user to group mapping.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl</name>
-  <value>false</value>
-  <description>
-    Whether or not to use SSL when connecting to the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore</name>
-  <value></value>
-  <description>
-    File path to the SSL keystore that contains the SSL certificate required
-    by the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the LDAP SSL keystore.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.user</name>
-  <value></value>
-  <description>
-    The distinguished name of the user to bind as when connecting to the LDAP
-    server. This may be left blank if the LDAP server supports anonymous binds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the bind user.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.base</name>
-  <value></value>
-  <description>
-    The search base for the LDAP connection. This is a distinguished name,
-    and will typically be the root of the LDAP directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.user</name>
-  <value>(&amp;(objectClass=user)(sAMAccountName={0}))</value>
-  <description>
-    An additional filter to use when searching for LDAP users. The default will
-    usually be appropriate for Active Directory installations. If connecting to
-    an LDAP server with a non-AD schema, this should be replaced with
-    (&amp;(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to
-    denote where the username fits into the filter.
-
-    If the LDAP server supports posixGroups, Hadoop can enable the feature by
-    setting the value of this property to "posixAccount" and the value of
-    the hadoop.security.group.mapping.ldap.search.filter.group property to
-    "posixGroup".
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.group</name>
-  <value>(objectClass=group)</value>
-  <description>
-    An additional filter to use when searching for LDAP groups. This should be
-    changed when resolving groups against a non-Active Directory installation.
-
-    See the description of hadoop.security.group.mapping.ldap.search.filter.user
-    to enable posixGroups support.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.member</name>
-  <value>member</value>
-  <description>
-    The attribute of the group object that identifies the users that are
-    members of the group. The default will usually be appropriate for
-    any LDAP installation.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.group.name</name>
-  <value>cn</value>
-  <description>
-    The attribute of the group object that identifies the group name. The
-    default will usually be appropriate for all LDAP systems.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.posix.attr.uid.name</name>
-  <value>uidNumber</value>
-  <description>
-    The attribute of posixAccount to use when groups for membership.
-    Mostly useful for schemas wherein groups have memberUids that use an
-    attribute other than uidNumber.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.posix.attr.gid.name</name>
-  <value>gidNumber</value>
-  <description>
-    The attribute of posixAccount indicating the group id.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.directory.search.timeout</name>
-  <value>10000</value>
-  <description>
-    The attribute applied to the LDAP SearchControl properties to set a
-    maximum time limit when searching and awaiting a result.
-    Set to 0 if infinite wait period is desired.
-    Default is 10 seconds. Units in milliseconds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers</name>
-  <value></value>
-  <description>
-    Comma separated of names of other providers to provide user to group
-    mapping. Used by CompositeGroupsMapping.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers.combined</name>
-  <value>true</value>
-  <description>
-    true or false to indicate whether groups from the providers are combined or
-    not. The default value is true. If true, then all the providers will be
-    tried to get groups and all the groups are combined to return as the final
-    results. Otherwise, providers are tried one by one in the configured list
-    order, and if any groups are retrieved from any provider, then the groups
-    will be returned without trying the left ones.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.service.user.name.key</name>
-  <value></value>
-  <description>
-    For those cases where the same RPC protocol is implemented by multiple
-    servers, this configuration is required for specifying the principal
-    name to use for the service when the client wishes to make an RPC call.
-  </description>
-</property>
-
-
-<property>
-    <name>hadoop.security.uid.cache.secs</name>
-    <value>14400</value>
-    <description>
-        This is the config controlling the validity of the entries in the cache
-        containing the userId to userName and groupId to groupName used by
-        NativeIO getFstat().
-    </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.protection</name>
-  <value>authentication</value>
-  <description>A comma-separated list of protection values for secured sasl
-      connections. Possible values are authentication, integrity and privacy.
-      authentication means authentication only and no integrity or privacy;
-      integrity implies authentication and integrity are enabled; and privacy
-      implies all of authentication, integrity and privacy are enabled.
-      hadoop.security.saslproperties.resolver.class can be used to override
-      the hadoop.rpc.protection for a connection at the server side.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.saslproperties.resolver.class</name>
-  <value></value>
-  <description>SaslPropertiesResolver used to resolve the QOP used for a
-      connection. If not specified, the full set of values specified in
-      hadoop.rpc.protection is used while determining the QOP used for the
-      connection. If a class is specified, then the QOP values returned by
-      the class will be used while determining the QOP used for the connection.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.sensitive-config-keys</name>
-  <value>secret$,password$,ssl.keystore.pass$,fs.s3.*[Ss]ecret.?[Kk]ey,fs.azure.account.key.*,dfs.webhdfs.oauth2.[a-z]+.token,hadoop.security.sensitive-config-keys</value>
-  <description>A comma-separated list of regular expressions to match against
-      configuration keys that should be redacted where appropriate, for
-      example, when logging modified properties during a reconfiguration,
-      private credentials should not be logged.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.workaround.non.threadsafe.getpwuid</name>
-  <value>true</value>
-  <description>Some operating systems or authentication modules are known to
-  have broken implementations of getpwuid_r and getpwgid_r, such that these
-  calls are not thread-safe. Symptoms of this problem include JVM crashes
-  with a stack trace inside these functions. If your system exhibits this
-  issue, enable this configuration parameter to include a lock around the
-  calls as a workaround.
-
-  An incomplete list of some systems known to have this issue is available
-  at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations
-  </description>
-</property>
-
-<property>
-  <name>hadoop.kerberos.kinit.command</name>
-  <value>kinit</value>
-  <description>Used to periodically renew Kerberos credentials when provided
-  to Hadoop. The default setting assumes that kinit is in the PATH of users
-  running the Hadoop client. Change this to the absolute path to kinit if this
-  is not the case.
-  </description>
-</property>
-
-<property>
-    <name>hadoop.kerberos.min.seconds.before.relogin</name>
-    <value>60</value>
-    <description>The minimum time between relogin attempts for Kerberos, in
-    seconds.
-    </description>
-</property>
-
-<property>
-  <name>hadoop.security.auth_to_local</name>
-  <value></value>
-  <description>Maps kerberos principals to local user names</description>
-</property>
-
-<!-- i/o properties -->
-<property>
-  <name>io.file.buffer.size</name>
-  <value>4096</value>
-  <description>The size of buffer for use in sequence files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>io.bytes.per.checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  io.file.buffer.size.</description>
-</property>
-
-<property>
-  <name>io.skip.checksum.errors</name>
-  <value>false</value>
-  <description>If true, when a checksum error is encountered while
-  reading a sequence file, entries are skipped, instead of throwing an
-  exception.</description>
-</property>
-
-<property>
-  <name>io.compression.codecs</name>
-  <value></value>
-  <description>A comma-separated list of the compression codec classes that can
-  be used for compression/decompression. In addition to any classes specified
-  with this property (which take precedence), codec classes on the classpath
-  are discovered using a Java ServiceLoader.</description>
-</property>
-
-<property>
-  <name>io.compression.codec.bzip2.library</name>
-  <value>system-native</value>
-  <description>The native-code library to be used for compression and
-  decompression by the bzip2 codec.  This library could be specified
-  either by by name or the full pathname.  In the former case, the
-  library is located by the dynamic linker, usually searching the
-  directories specified in the environment variable LD_LIBRARY_PATH.
-
-  The value of "system-native" indicates that the default system
-  library should be used.  To indicate that the algorithm should
-  operate entirely in Java, specify "java-builtin".</description>
-</property>
-
-<property>
-  <name>io.serializations</name>
-  <value>org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value>
-  <description>A list of serialization classes that can be used for
-  obtaining serializers and deserializers.</description>
-</property>
-
-<property>
-  <name>io.seqfile.local.dir</name>
-  <value>${hadoop.tmp.dir}/io/local</value>
-  <description>The local directory where sequence file stores intermediate
-  data files during merge.  May be a comma-separated list of
-  directories on different devices in order to spread disk i/o.
-  Directories that do not exist are ignored.
-  </description>
-</property>
-
-<property>
-  <name>io.map.index.skip</name>
-  <value>0</value>
-  <description>Number of index entries to skip between each entry.
-  Zero by default. Setting this to values larger than zero can
-  facilitate opening large MapFiles using less memory.</description>
-</property>
-
-<property>
-  <name>io.map.index.interval</name>
-  <value>128</value>
-  <description>
-    MapFile consist of two files - data file (tuples) and index file
-    (keys). For every io.map.index.interval records written in the
-    data file, an entry (record-key, data-file-position) is written
-    in the index file. This is to allow for doing binary search later
-    within the index file to look up records by their keys and get their
-    closest positions in the data file.
-  </description>
-</property>
-
-<!-- file system properties -->
-
-<property>
-  <name>fs.defaultFS</name>
-  <value>file:///</value>
-  <description>The name of the default file system.  A URI whose
-  scheme and authority determine the FileSystem implementation.  The
-  uri's scheme determines the config property (fs.SCHEME.impl) naming
-  the FileSystem implementation class.  The uri's authority is used to
-  determine the host, port, etc. for a filesystem.</description>
-</property>
-
-<property>
-  <name>fs.default.name</name>
-  <value>file:///</value>
-  <description>Deprecated. Use (fs.defaultFS) property
-  instead</description>
-</property>
-
-<property>
-  <name>fs.trash.interval</name>
-  <value>0</value>
-  <description>Number of minutes after which the checkpoint
-  gets deleted.  If zero, the trash feature is disabled.
-  This option may be configured both on the server and the
-  client. If trash is disabled server side then the client
-  side configuration is checked. If trash is enabled on the
-  server side then the value configured on the server is
-  used and the client configuration value is ignored.
-  </description>
-</property>
-
-<property>
-  <name>fs.trash.checkpoint.interval</name>
-  <value>0</value>
-  <description>Number of minutes between trash checkpoints.
-  Should be smaller or equal to fs.trash.interval. If zero,
-  the value is set to the value of fs.trash.interval.
-  Every time the checkpointer runs it creates a new checkpoint
-  out of current and removes checkpoints created more than
-  fs.trash.interval minutes ago.
-  </description>
-</property>
-
-<property>
-  <name>fs.protected.directories</name>
-  <value></value>
-  <description>A comma-separated list of directories which cannot
-    be deleted even by the superuser unless they are empty. This
-    setting can be used to guard important system directories
-    against accidental deletion due to administrator error.
-  </description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.file.impl</name>
-  <value>org.apache.hadoop.fs.local.LocalFs</value>
-  <description>The AbstractFileSystem for file: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.har.impl</name>
-  <value>org.apache.hadoop.fs.HarFs</value>
-  <description>The AbstractFileSystem for har: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.hdfs.impl</name>
-  <value>org.apache.hadoop.fs.Hdfs</value>
-  <description>The FileSystem for hdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.viewfs.impl</name>
-  <value>org.apache.hadoop.fs.viewfs.ViewFs</value>
-  <description>The AbstractFileSystem for view file system for viewfs: uris
-  (ie client side mount table:).</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.ftp.impl</name>
-  <value>org.apache.hadoop.fs.ftp.FtpFs</value>
-  <description>The FileSystem for Ftp: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.webhdfs.impl</name>
-  <value>org.apache.hadoop.fs.WebHdfs</value>
-  <description>The FileSystem for webhdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.swebhdfs.impl</name>
-  <value>org.apache.hadoop.fs.SWebHdfs</value>
-  <description>The FileSystem for swebhdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.ftp.host</name>
-  <value>0.0.0.0</value>
-  <description>FTP filesystem connects to this server</description>
-</property>
-
-<property>
-  <name>fs.ftp.host.port</name>
-  <value>21</value>
-  <description>
-    FTP filesystem connects to fs.ftp.host on this port
-  </description>
-</property>
-
-<property>
-  <name>fs.df.interval</name>
-  <value>60000</value>
-  <description>Disk usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.du.interval</name>
-  <value>600000</value>
-  <description>File space usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.s3.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when writing files to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3</value>
-  <description>Determines where on the local filesystem the s3:/s3n: filesystem
-  should store files before sending them to S3
-  (or after retrieving them from S3).
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.maxRetries</name>
-  <value>4</value>
-  <description>The maximum number of retries for reading or writing files to S3,
-  before we signal failure to the application.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.sleepTimeSeconds</name>
-  <value>10</value>
-  <description>The number of seconds to sleep between each S3 retry.
-  </description>
-</property>
-
-<property>
-  <name>fs.automatic.close</name>
-  <value>true</value>
-  <description>By default, FileSystem instances are automatically closed at program
-  exit using a JVM shutdown hook. Setting this property to false disables this
-  behavior. This is an advanced option that should only be used by server applications
-  requiring a more carefully orchestrated shutdown sequence.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when reading files using the native S3
-  filesystem (s3n: URIs).</description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.enabled</name>
-  <value>false</value>
-  <description>Setting this property to true enables multiple uploads to
-  native S3 filesystem. When uploading a file, it is split into blocks
-  if the size is larger than fs.s3n.multipart.uploads.block.size.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.block.size</name>
-  <value>67108864</value>
-  <description>The block size for multipart uploads to native S3 filesystem.
-  Default size is 64MB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.copy.block.size</name>
-  <value>5368709120</value>
-  <description>The block size for multipart copy in native S3 filesystem.
-  Default size is 5GB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.server-side-encryption-algorithm</name>
-  <value></value>
-  <description>Specify a server-side encryption algorithm for S3.
-  Unset by default, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.access.key</name>
-  <description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <description>AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <description>
-	  Comma-separated class names of credential provider classes which implement
-	  com.amazonaws.auth.AWSCredentialsProvider.
-
-	  These are loaded and queried in sequence for a valid set of credentials.
-	  Each listed class must implement one of the following means of
-	  construction, which are attempted in order:
-	  1. a public constructor accepting java.net.URI and
-	  org.apache.hadoop.conf.Configuration,
-	  2. a public static method named getInstance that accepts no
-	  arguments and returns an instance of
-	  com.amazonaws.auth.AWSCredentialsProvider, or
-	  3. a public default constructor.
-
-	  Specifying
-	  org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider
-	  allows
-	  anonymous access to a publicly accessible S3 bucket without any credentials.
-	  Please note that allowing anonymous access to an S3 bucket compromises
-	  security and therefore is unsuitable for most use cases. It can be useful
-	  for accessing public data sets without requiring AWS credentials.
-
-	  If unspecified, then the default list of credential provider classes,
-	  queried in sequence, is:
-	  1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider:
-	  supports static
-	  configuration of AWS access key ID and secret access key. See also
-	  fs.s3a.access.key and fs.s3a.secret.key.
-	  2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
-	  configuration of AWS access key ID and secret access key in
-	  environment variables named AWS_ACCESS_KEY_ID and
-	  AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
-	  3.
-	  org.apache.hadoop.fs.s3a.SharedInstanceProfileCredentialsProvider:
-	  a shared instance of
-	  com.amazonaws.auth.InstanceProfileCredentialsProvider from the AWS
-	  SDK, which supports use of instance profile credentials if running
-	  in an EC2 VM. Using this shared instance potentially reduces load
-	  on the EC2 instance metadata service for multi-threaded
-	  applications.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.session.token</name>
-  <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
-    as one of the providers.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.security.credential.provider.path</name>
-  <value />
-  <description>
-    Optional comma separated list of credential providers, a list
-    which is prepended to that set in hadoop.security.credential.provider.path
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.maximum</name>
-  <value>15</value>
-  <description>Controls the maximum number of simultaneous connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.ssl.enabled</name>
-  <value>true</value>
-  <description>Enables or disables SSL connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.endpoint</name>
-  <description>AWS S3 endpoint to connect to. An up-to-date list is
-    provided in the AWS Documentation: regions and endpoints. Without this
-    property, the standard region (s3.amazonaws.com) is assumed.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.path.style.access</name>
-  <value>false</value>
-  <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
-    Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.host</name>
-  <description>Hostname of the (optional) proxy server for S3 connections.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.port</name>
-  <description>Proxy server port. If this property is not set
-    but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with
-    the value of fs.s3a.connection.ssl.enabled).</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.username</name>
-  <description>Username for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.password</name>
-  <description>Password for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.domain</name>
-  <description>Domain for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.workstation</name>
-  <description>Workstation for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.attempts.maximum</name>
-  <value>20</value>
-  <description>How many times we should retry commands on transient errors.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.establish.timeout</name>
-  <value>5000</value>
-  <description>Socket connection setup timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.timeout</name>
-  <value>200000</value>
-  <description>Socket connection timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.socket.send.buffer</name>
-  <value>8192</value>
-  <description>Socket send buffer hint to amazon connector. Represented in bytes.</description>
-</property>
-
-<property>
-  <name>fs.s3a.socket.recv.buffer</name>
-  <value>8192</value>
-  <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
-</property>
-
-<property>
-  <name>fs.s3a.paging.maximum</name>
-  <value>5000</value>
-  <description>How many keys to request from S3 when doing
-     directory listings at a time.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.max</name>
-  <value>10</value>
-  <description>The total number of threads available in the filesystem for data
-    uploads *or any other queued filesystem operation*.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.keepalivetime</name>
-  <value>60</value>
-  <description>Number of seconds a thread can be idle before being
-    terminated.</description>
-</property>
-
-<property>
-  <name>fs.s3a.max.total.tasks</name>
-  <value>5</value>
-  <description>The number of operations which can be queued for execution</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.size</name>
-  <value>100M</value>
-  <description>How big (in bytes) to split upload or copy operations up into.
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.threshold</name>
-  <value>2147483647</value>
-  <description>How big (in bytes) to split upload or copy operations up into.
-    This also controls the partition size in renamed files, as rename() involves
-    copying the source file(s).
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multiobjectdelete.enable</name>
-  <value>true</value>
-  <description>When enabled, multiple single-object delete requests are replaced by
-    a single 'delete multiple objects'-request, reducing the number of requests.
-    Beware: legacy S3-compatible object stores might not support this request.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.acl.default</name>
-  <description>Set a canned ACL for newly created and copied objects. Value may be Private,
-      PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
-      or BucketOwnerFullControl.</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge</name>
-  <value>false</value>
-  <description>True if you want to purge existing multipart uploads that may not have been
-    completed/aborted correctly. The corresponding purge age is defined in
-    fs.s3a.multipart.purge.age.
-    If set, when the filesystem is instantiated then all outstanding uploads
-    older than the purge age will be terminated -across the entire bucket.
-    This will impact multipart uploads by other applications and users. so should
-    be used sparingly, with an age value chosen to stop failed uploads, without
-    breaking ongoing operations.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge.age</name>
-  <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.server-side-encryption-algorithm</name>
-  <description>Specify a server-side encryption algorithm for s3a: file system.
-    Unset by default, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.signing-algorithm</name>
-  <description>Override the default signing algorithm so legacy
-    implementations can still be used</description>
-</property>
-
-<property>
-  <name>fs.s3a.block.size</name>
-  <value>32M</value>
-  <description>Block size to use when reading files using s3a: file system.
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3a</value>
-  <description>Comma separated list of directories that will be used to buffer file
-    uploads to.</description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload</name>
-  <value>false</value>
-  <description>
-    Use the incremental block-based fast upload mechanism with
-    the buffering mechanism set in fs.s3a.fast.upload.buffer.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload.buffer</name>
-  <value>disk</value>
-  <description>
-    The buffering mechanism to use when using S3A fast upload
-    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
-    This configuration option has no effect if fs.s3a.fast.upload is false.
-
-    "disk" will use the directories listed in fs.s3a.buffer.dir as
-    the location(s) to save data prior to being uploaded.
-
-    "array" uses arrays in the JVM heap
-
-    "bytebuffer" uses off-heap memory within the JVM.
-
-    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
-    of blocks set by:
-
-        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
-
-    If using either of these mechanisms, keep this value low
-
-    The total number of threads performing work across all threads is set by
-    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
-    work items.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload.active.blocks</name>
-  <value>4</value>
-  <description>
-    Maximum Number of blocks a single output stream can have
-    active (uploading, or queued to the central FileSystem
-    instance's pool of queued operations.
-
-    This stops a single stream overloading the shared thread pool.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.readahead.range</name>
-  <value>64K</value>
-  <description>Bytes to read ahead during a seek() before closing and
-  re-opening the S3 HTTP connection. This option will be overridden if
-  any call to setReadahead() is made to an open stream.
-  A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.user.agent.prefix</name>
-  <value></value>
-  <description>
-    Sets a custom value that will be prepended to the User-Agent header sent in
-    HTTP requests to the S3 back-end by S3AFileSystem.  The User-Agent header
-    always includes the Hadoop version number followed by a string generated by
-    the AWS SDK.  An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6".
-    If this optional property is set, then its value is prepended to create a
-    customized User-Agent.  For example, if this configuration property was set
-    to "MyApp", then an example of the resulting User-Agent would be
-    "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6".
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-  <description>The implementation class of the S3A Filesystem</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3A</value>
-  <description>The implementation class of the S3A AbstractFileSystem.</description>
-</property>
-
-<property>
-  <name>io.seqfile.compress.blocksize</name>
-  <value>1000000</value>
-  <description>The minimum block size for compression in block compressed
-          SequenceFiles.
-  </description>
-</property>
-
- <property>
-  <name>io.mapfile.bloom.size</name>
-  <value>1048576</value>
-  <description>The size of BloomFilter-s used in BloomMapFile. Each time this many
-  keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter).
-  Larger values minimize the number of filters, which slightly increases the performance,
-  but may waste too much space if the total number of keys is usually much smaller
-  than this number.
-  </description>
-</property>
-
-<property>
-  <name>io.mapfile.bloom.error.rate</name>
-  <value>0.005</value>
-  <description>The rate of false positives in BloomFilter-s used in BloomMapFile.
-  As this value decreases, the size of BloomFilter-s increases exponentially. This
-  value is the probability of encountering false positives (default is 0.5%).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.util.hash.type</name>
-  <value>murmur</value>
-  <description>The default implementation of Hash. Currently this can take one of the
-  two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash.
-  </description>
-</property>
-
-
-<!-- ipc properties -->
-
-<property>
-  <name>ipc.client.idlethreshold</name>
-  <value>4000</value>
-  <description>Defines the threshold number of connections after which
-               connections will be inspected for idleness.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.kill.max</name>
-  <value>10</value>
-  <description>Defines the maximum number of clients to disconnect in one go.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connection.maxidletime</name>
-  <value>10000</value>
-  <description>The maximum time in msec after which a client will bring down the
-               connection to the server.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries</name>
-  <value>10</value>
-  <description>Indicates the number of retries a client will make to establish
-               a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.retry.interval</name>
-  <value>1000</value>
-  <description>Indicates the number of milliseconds a client will wait for
-    before retrying to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.timeout</name>
-  <value>20000</value>
-  <description>Indicates the number of milliseconds a client will wait for the
-               socket to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries.on.timeouts</name>
-  <value>45</value>
-  <description>Indicates the number of retries a client will make on socket timeout
-               to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.tcpnodelay</name>
-  <value>true</value>
-  <description>Use TCP_NODELAY flag to bypass Nagle's algorithm transmission delays.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.low-latency</name>
-  <value>false</value>
-  <description>Use low-latency QoS markers for IPC connections.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.ping</name>
-  <value>true</value>
-  <description>Send a ping to the server when timeout on reading the response,
-  if set to true. If no failure is detected, the client retries until at least
-  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
-  </description>
-</property>
-
-<property>
-  <name>ipc.ping.interval</name>
-  <value>60000</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  The client will send ping when the interval is passed without receiving bytes,
-  if ipc.client.ping is set to true.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.rpc-timeout.ms</name>
-  <value>0</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  If ipc.client.ping is set to true and this rpc-timeout is greater than
-  the value of ipc.ping.interval, the effective value of the rpc-timeout is
-  rounded up to multiple of ipc.ping.interval.
-  </description>
-</property>
-
-<property>
-  <name>ipc.server.listen.queue.size</name>
-  <value>128</value>
-  <description>Indicates the length of the listen queue for servers accepting
-               client connections.
-  </description>
-</property>
-
-<property>
-    <name>ipc.server.log.slow.rpc</name>
-    <value>false</value>
-    <description>This setting is useful to troubleshoot performance issues for
-     various services. If this value is set to true then we log requests that
-     fall into 99th percentile as well as increment RpcSlowCalls counter.
-    </description>
-</property>
-
-<property>
-  <name>ipc.maximum.data.length</name>
-  <value>67108864</value>
-  <description>This indicates the maximum IPC message length (bytes) that can be
-    accepted by the server. Messages larger than this value are rejected by the
-    immediately to avoid possible OOMs. This setting should rarely need to be
-    changed.
-  </description>
-</property>
-
-<property>
-  <name>ipc.maximum.response.length</name>
-  <value>134217728</value>
-  <description>This indicates the maximum IPC message length (bytes) that can be
-    accepted by the client. Messages larger than this value are rejected
-    immediately to avoid possible OOMs. This setting should rarely need to be
-    changed.  Set to 0 to disable.
-  </description>
-</property>
-
-<!-- Proxy Configuration -->
-
-<property>
-  <name>hadoop.security.impersonation.provider.class</name>
-  <value></value>
-  <description>A class which implements ImpersonationProvider interface, used to
-       authorize whether one user can impersonate a specific user.
-       If not specified, the DefaultImpersonationProvider will be used.
-       If a class is specified, then that class will be used to determine
-       the impersonation capability.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.default</name>
-  <value>org.apache.hadoop.net.StandardSocketFactory</value>
-  <description> Default SocketFactory to use. This parameter is expected to be
-    formatted as "package.FactoryClassName".
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.ClientProtocol</name>
-  <value></value>
-  <description> SocketFactory to use to connect to a DFS. If null or empty, use
-    hadoop.rpc.socket.class.default. This socket factory is also used by
-    DFSClient to create sockets to DataNodes.
-  </description>
-</property>
-
-
-
-<property>
-  <name>hadoop.socks.server</name>
-  <value></value>
-  <description> Address (host:port) of the SOCKS server to be used by the
-    SocksSocketFactory.
-  </description>
-</property>
-
-<!-- Topology Configuration -->
-<property>
-  <name>net.topology.node.switch.mapping.impl</name>
-  <value>org.apache.hadoop.net.ScriptBasedMapping</value>
-  <description> The default implementation of the DNSToSwitchMapping. It
-    invokes a script specified in net.topology.script.file.name to resolve
-    node names. If the value for net.topology.script.file.name is not set, the
-    default value of DEFAULT_RACK is returned for all node names.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.impl</name>
-  <value>org.apache.hadoop.net.NetworkTopology</value>
-  <description> The default implementation of NetworkTopology which is classic three layer one.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.file.name</name>
-  <value></value>
-  <description> The script name that should be invoked to resolve DNS names to
-    NetworkTopology names. Example: the script would take host.foo.bar as an
-    argument, and return /rack1 as the output.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.number.args</name>
-  <value>100</value>
-  <description> The max number of args that the script configured with
-    net.topology.script.file.name should be run with. Each arg is an
-    IP address.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.table.file.name</name>
-  <value></value>
-  <description> The file name for a topology file, which is used when the
-    net.topology.node.switch.mapping.impl property is set to
-    org.apache.hadoop.net.TableMapping. The file format is a two column text
-    file, with columns separated by whitespace. The first column is a DNS or
-    IP address and the second column specifies the rack where the address maps.
-    If no entry corresponding to a host in the cluster is found, then
-    /default-rack is assumed.
-  </description>
-</property>
-
-<!-- Local file system -->
-<property>
-  <name>file.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>file.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  file.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>file.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>file.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>file.replication</name>
-  <value>1</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3 File System -->
-
-<property>
-  <name>s3.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3native File System -->
-
-<property>
-  <name>s3native.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3native.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3native.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3native.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3native.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3native.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- FTP file system -->
-<property>
-  <name>ftp.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>ftp.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  ftp.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>ftp.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>ftp.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>ftp.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- Tfile -->
-
-<property>
-  <name>tfile.io.chunk.size</name>
-  <value>1048576</value>
-  <description>
-    Value chunk size in bytes. Default  to
-    1MB. Values of the length less than the chunk size is
-    guaranteed to have known value length in read time (See also
-    TFile.Reader.Scanner.Entry.isValueLengthKnown()).
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.output.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataOutputStream in bytes.
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.input.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataInputStream in bytes.
-  </description>
-</property>
-
-<!-- HTTP web-consoles Authentication -->
-
-<property>
-  <name>hadoop.http.authentication.type</name>
-  <value>simple</value>
-  <description>
-    Defines authentication used for Oozie HTTP endpoint.
-    Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.token.validity</name>
-  <value>36000</value>
-  <description>
-    Indicates how long (in seconds) an authentication token is valid before it has
-    to be renewed.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.signature.secret.file</name>
-  <value>${user.home}/hadoop-http-auth-signature-secret</value>
-  <description>
-    The signature secret for signing the authentication tokens.
-    The same secret should be used for JT/NN/DN/TT configurations.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.cookie.domain</name>
-  <value></value>
-  <description>
-    The domain to use for the HTTP cookie that stores the authentication token.
-    In order to authentiation to work correctly across all Hadoop nodes web-consoles
-    the domain must be correctly set.
-    IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings.
-    For this setting to work properly all nodes in the cluster must be configured
-    to generate URLs with hostname.domain names on it.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.simple.anonymous.allowed</name>
-  <value>true</value>
-  <description>
-    Indicates if anonymous requests are allowed when using 'simple' authentication.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.principal</name>
-  <value>HTTP/_HOST@LOCALHOST</value>
-  <description>
-    Indicates the Kerberos principal to be used for HTTP endpoint.
-    The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.keytab</name>
-  <value>${user.home}/hadoop.keytab</value>
-  <description>
-    Location of the keytab file with the credentials for the principal.
-    Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop.
-  </description>
-</property>
-
-<!-- HTTP CORS support -->
-<property>
-  <description>Enable/disable the cross-origin (CORS) filter.</description>
-  <name>hadoop.http.cross-origin.enabled</name>
-  <value>false</value>
-</property>
-
-<property>
-  <description>Comma separated list of origins that are allowed for web
-    services needing cross-origin (CORS) support. Wildcards (*) and patterns
-    allowed</description>
-  <name>hadoop.http.cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
-
-<property>
-  <description>Comma separated list of methods that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
-
-<property>
-  <description>Comma separated list of headers that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
-
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-    for web services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.methods</name>
-  <value></value>
-  <description>
-    List of fencing methods to use for service fencing. May contain
-    builtin methods (eg shell and sshfence) or user-defined method.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.connect-timeout</name>
-  <value>30000</value>
-  <description>
-    SSH connection timeout, in milliseconds, to use with the builtin
-    sshfence fencer.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.private-key-files</name>
-  <value></value>
-  <description>
-    The SSH private key files to use with the builtin sshfence fencer.
-  </description>
-</property>
-
-
-<!-- Static Web User Filter properties. -->
-<property>
-  <description>
-    The user name to filter as, on static web filters
-    while rendering content. An example use is the HDFS
-    web UI (user to be used for browsing files).
-  </description>
-  <name>hadoop.http.staticuser.user</name>
-  <value>dr.who</value>
-</property>
-
-<property>
-  <name>ha.zookeeper.quorum</name>
-  <description>
-    A list of ZooKeeper server addresses, separated by commas, that are
-    to be used by the ZKFailoverController in automatic failover.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.session-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    The session timeout to use when the ZKFC connects to ZooKeeper.
-    Setting this value to a lower value implies that server crashes
-    will be detected more quickly, but risks triggering failover too
-    aggressively in the case of a transient error or network blip.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.parent-znode</name>
-  <value>/hadoop-ha</value>
-  <description>
-    The ZooKeeper znode under which the ZK failover controller stores
-    its information. Note that the nameservice ID is automatically
-    appended to this znode, so it is not normally necessary to
-    configure this, even in a federated environment.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.acl</name>
-  <value>world:anyone:rwcda</value>
-  <description>
-    A comma-separated list of ZooKeeper ACLs to apply to the znodes
-    used by automatic failover. These ACLs are specified in the same
-    format as used by the ZooKeeper CLI.
-
-    If the ACL itself contains secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.auth</name>
-  <value></value>
-  <description>
-    A comma-separated list of ZooKeeper authentications to add when
-    connecting to ZooKeeper. These are specified in the same format
-    as used by the &quot;addauth&quot; command in the ZK CLI. It is
-    important that the authentications specified here are sufficient
-    to access znodes with the ACL specified in ha.zookeeper.acl.
-
-    If the auths contain secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<!-- SSLFactory configuration -->
-
-<property>
-  <name>hadoop.ssl.keystores.factory.class</name>
-  <value>org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory</value>
-  <description>
-    The keystores factory to use for retrieving certificates.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.require.client.cert</name>
-  <value>false</value>
-  <description>Whether client certificates are required</description>
-</property>
-
-<property>
-  <name>hadoop.ssl.hostname.verifier</name>
-  <value>DEFAULT</value>
-  <description>
-    The hostname verifier to provide for HttpsURLConnections.
-    Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and
-    ALLOW_ALL
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.server.conf</name>
-  <value>ssl-server.xml</value>
-  <description>
-    Resource file from which ssl server keystore information will be extracted.
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.client.conf</name>
-  <value>ssl-client.xml</value>
-  <description>
-    Resource file from which ssl client keystore information will be extracted
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled</name>
-  <value>false</value>
-  <description>
-    Deprecated. Use dfs.http.policy and yarn.http.policy instead.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled.protocols</name>
-  <value>TLSv1</value>
-  <description>
-    Protocols supported by the ssl.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.jetty.logs.serve.aliases</name>
-  <value>true</value>
-  <description>
-    Enable/Disable aliases serving from jetty
-  </description>
-</property>
-
-<property>
-  <name>fs.permissions.umask-mode</name>
-  <value>022</value>
-  <description>
-    The umask used when creating files and directories.
-    Can be in octal or in symbolic. Examples are:
-    "022" (octal for u=rwx,g=r-x,o=r-x in symbolic),
-    or "u=rwx,g=rwx,o=" (symbolic for 007 in octal).
-  </description>
-</property>
-
-<!-- ha properties -->
-
-<property>
-  <name>ha.health-monitor.connect-retry-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to retry connecting to the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.check-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to check the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.sleep-after-disconnect.ms</name>
-  <value>1000</value>
-  <description>
-    How long to sleep after an unexpected RPC error.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.rpc-timeout.ms</name>
-  <value>45000</value>
-  <description>
-    Timeout for the actual monitorHealth() calls.
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.new-active.rpc-timeout.ms</name>
-  <value>60000</value>
-  <description>
-    Timeout that the FC waits for the new active to become active
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.rpc-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    Timeout that the FC waits for the old active to go to standby
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.connection.retries</name>
-  <value>1</value>
-  <description>
-    FC connection retries for graceful fencing
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.cli-check.rpc-timeout.ms</name>
-  <value>20000</value>
-  <description>
-    Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.fallback-to-simple-auth-allowed</name>
-  <value>false</value>
-  <description>
-    When a client is configured to attempt a secure connection, but attempts to
-    connect to an insecure server, that server may instruct the client to
-    switch to SASL SIMPLE (unsecure) authentication. This setting controls
-    whether or not the client will accept this instruction from the server.
-    When false (the default), the client will not allow the fallback to SIMPLE
-    authentication, and will abort the connection.
-  </description>
-</property>
-
-<property>
-  <name>fs.client.resolve.remote.symlinks</name>
-  <value>true</value>
-  <description>
-      Whether to resolve symlinks when accessing a remote Hadoop filesystem.
-      Setting this to false causes an exception to be thrown upon encountering
-      a symlink. This setting does not apply to local filesystems, which
-      automatically resolve local symlinks.
-  </description>
-</property>
-
-<property>
-  <name>nfs.exports.allowed.hosts</name>
-  <value>* rw</value>
-  <description>
-    By default, the export can be mounted by any client. The value string
-    contains machine name and access privilege, separated by whitespace
-    characters. The machine name format can be a single host, a Java regular
-    expression, or an IPv4 address. The access privilege uses rw or ro to
-    specify read/write or read-only access of the machines to exports. If the
-    access privilege is not provided, the default is read-only. Entries are separated by ";".
-    For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;".
-    Only the NFS gateway needs to restart after this property is updated.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.user.group.static.mapping.overrides</name>
-  <value>dr.who=;</value>
-  <description>
-    Static mapping of user to groups. This will override the groups if
-    available in the system for the specified user. In otherwords, groups
-    look-up will not happen for these users, instead groups mapped in this
-    configuration will be used.
-    Mapping should be in this format.
-    user1=group1,group2;user2=;user3=group2;
-    Default, "dr.who=;" will consider "dr.who" as user without groups.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.quantile.enable</name>
-  <value>false</value>
-  <description>
-    Setting this property to true and rpc.metrics.percentiles.intervals
-    to a comma-separated list of the granularity in seconds, the
-    50/75/90/95/99th percentile latency for rpc queue/processing time in
-    milliseconds are added to rpc metrics.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.percentiles.intervals</name>
-  <value></value>
-  <description>
-    A comma-separated list of the granularity in seconds for the metrics which
-    describe the 50/75/90/95/99th percentile latency for rpc queue/processing
-    time. The metrics are outputted if rpc.metrics.quantile.enable is set to
-    true.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE</name>
-  <value></value>
-  <description>
-    The prefix for a given crypto codec, contains a comma-separated
-    list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE).
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.aes.ctr.nopadding</name>
-  <value>org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec</value>
-  <description>
-    Comma-separated list of crypto codec implementations for AES/CTR/NoPadding.
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.cipher.suite</name>
-  <value>AES/CTR/NoPadding</value>
-  <description>
-    Cipher suite for crypto codec.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.jce.provider</name>
-  <value></value>
-  <description>
-    The JCE provider name used in CryptoCodec.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.buffer.size</name>
-  <value>8192</value>
-  <description>
-    The buffer size used by CryptoInputStream and CryptoOutputStream.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.java.secure.random.algorithm</name>
-  <value>SHA1PRNG</value>
-  <description>
-    The java secure random algorithm.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.secure.random.impl</name>
-  <value></value>
-  <description>
-    Implementation of secure random.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.random.device.file.path</name>
-  <value>/dev/urandom</value>
-  <description>
-    OS security random device file path.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.key.provider.path</name>
-  <description>
-    The KeyProvider to use when managing zone keys, and interacting with
-    encryption keys when reading and writing to an encryption zone.
-  </description>
-</property>
-
-<property>
-  <name>fs.har.impl.disable.cache</name>
-  <value>true</value>
-  <description>Don't cache 'har' filesystem instances.</description>
-</property>
-
-<!--- KMSClientProvider configurations -->
-<property>
-  <name>hadoop.security.kms.client.authentication.retry-count</name>
-  <value>1</value>
-  <description>
-    Number of time to retry connecting to KMS on authentication failure
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.size</name>
-  <value>500</value>
-  <description>
-    Size of the EncryptedKeyVersion cache Queue for each key
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.low-watermark</name>
-  <value>0.3f</value>
-  <description>
-    If size of the EncryptedKeyVersion cache Queue falls below the
-    low watermark, this cache queue will be scheduled for a refill
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.num.refill.threads</name>
-  <value>2</value>
-  <description>
-    Number of threads to use for refilling depleted EncryptedKeyVersion
-    cache Queues
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.expiry</name>
-  <value>43200000</value>
-  <description>
-    Cache expiry time for a Key, after which the cache Queue for this
-    key will be dropped. Default = 12hrs
-  </description>
-</property>
-
- <property>
-  <name>ipc.server.max.connections</name>
-  <value>0</value>
-  <description>The maximum number of concurrent connections a server is allowed
-    to accept. If this limit is exceeded, incoming connections will first fill
-    the listen queue and then may go to an OS-specific listen overflow queue.
-    The client may fail or timeout, but the server can avoid running out of file
-    descriptors using this feature. 0 means no limit.
-  </description>
-</property>
-
-
-  <!-- YARN registry -->
-
-  <property>
-    <description>
-      Is the registry enabled in the YARN Resource Manager?
-
-      If true, the YARN RM will, as needed.
-      create the user and system paths, and purge
-      service records when containers, application attempts
-      and applications complete.
-
-      If false, the paths must be created by other means,
-      and no automatic cleanup of service records will take place.
-    </description>
-    <name>hadoop.registry.rm.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      The root zookeeper node for the registry
-    </description>
-    <name>hadoop.registry.zk.root</name>
-    <value>/registry</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper session timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.session.timeout.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.connection.timeout.ms</name>
-    <value>15000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection retry count before failing
-    </description>
-    <name>hadoop.registry.zk.retry.times</name>
-    <value>5</value>
-  </property>
-
-  <property>
-    <description>
-    </description>
-    <name>hadoop.registry.zk.retry.interval.ms</name>
-    <value>1000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper retry limit in milliseconds, during
-      exponential backoff.
-
-      This places a limit even
-      if the retry times and interval limit, combined
-      with the backoff policy, result in a long retry
-      period
-    </description>
-    <name>hadoop.registry.zk.retry.ceiling.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      List of hostname:port pairs defining the
-      zookeeper quorum binding for the registry
-    </description>
-    <name>hadoop.registry.zk.quorum</name>
-    <value>localhost:2181</value>
-  </property>
-
-  <property>
-    <description>
-      Key to set if the registry is secure. Turning it on
-      changes the permissions policy from "open access"
-      to restrictions on kerberos with the option of
-      a user adding one or more auth key pairs down their
-      own tree.
-    </description>
-    <name>hadoop.registry.secure</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      A comma separated list of Zookeeper ACL identifiers with
-      system access to the registry in a secure cluster.
-
-      These are given full access to all entries.
-
-      If there is an "@" at the end of a SASL entry it
-      instructs the registry client to append the default kerberos domain.
-    </description>
-    <name>hadoop.registry.system.acls</name>
-    <value>sasl:yarn@, sasl:mapred@, sasl:hdfs@</value>
-  </property>
-
-  <property>
-    <description>
-      The kerberos realm: used to set the realm of
-      system principals which do not declare their realm,
-      and any other accounts that need the value.
-
-      If empty, the default realm of the running process
-      is used.
-
-      If neither are known and the realm is needed, then the registry
-      service/client will fail.
-    </description>
-    <name>hadoop.registry.kerberos.realm</name>
-    <value></value>
-  </property>
-
-  <property>
-    <description>
-      Key to define the JAAS context. Used in secure
-      mode
-    </description>
-    <name>hadoop.registry.jaas.context</name>
-    <value>Client</value>
-  </property>
-
-  <property>
-    <description>
-      Enable hdfs shell commands to display warnings if (fs.defaultFS) property
-      is not set.
-    </description>
-    <name>hadoop.shell.missing.defaultFs.warning</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>hadoop.shell.safely.delete.limit.num.files</name>
-    <value>100</value>
-    <description>Used by -safely option of hadoop fs shell -rm command to avoid
-      accidental deletion of large directories. When enabled, the -rm command
-      requires confirmation if the number of files to be deleted is greater than
-      this limit.  The default limit is 100 files. The warning is disabled if
-      the limit is 0 or the -safely is not specified in -rm command.
-    </description>
-  </property>
-
-  <property>
-    <name>fs.client.htrace.sampler.classes</name>
-    <value></value>
-    <description>The class names of the HTrace Samplers to use for Hadoop
-      filesystem clients.
-    </description>
-  </property>
-
-  <property>
-    <name>hadoop.htrace.span.receiver.classes</name>
-    <value></value>
-    <description>The class names of the Span Receivers to use for Hadoop.
-    </description>
-  </property>
-
-  <property>
-    <name>hadoop.caller.context.enabled</name>
-    <value>false</value>
-    <description>When the feature is enabled, additional fields are written into
-      name-node audit log records for auditing coarse granularity operations.
-    </description>
-  </property>
-  <property>
-    <name>hadoop.caller.context.max.size</name>
-    <value>128</value>
-    <description>The maximum bytes a caller context string can have. If the
-      passed caller context is longer than this maximum bytes, client will
-      truncate it before sending to server. Note that the server may have a
-      different maximum size, and will truncate the caller context to the
-      maximum size it allows.
-    </description>
-  </property>
-  <property>
-    <name>hadoop.caller.context.signature.max.size</name>
-    <value>40</value>
-    <description>
-      The caller's signature (optional) is for offline validation. If the
-      signature exceeds the maximum allowed bytes in server, the caller context
-      will be abandoned, in which case the caller context will not be recorded
-      in audit logs.
-    </description>
-  </property>
-
+	<property>
+		<name>hadoop.security.group.mapping</name>
+		<value>this.does.not.exist.class</value>
+	</property>
 </configuration>
diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java
index ebf7948..1cf35f4 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -646,21 +646,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
 		}
 	}
 
-	static{
-		//print deprecation warning if hadoop-site.xml is found in classpath
-		ClassLoader cL = Thread.currentThread().getContextClassLoader();
-		if (cL == null) {
-			cL = Configuration.class.getClassLoader();
-		}
-		if(cL.getResource("hadoop-site.xml")!=null) {
-			LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
-				"Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
-				+ "mapred-site.xml and hdfs-site.xml to override properties of " +
-				"core-default-shaded.xml, mapred-default.xml and hdfs-default.xml " +
-				"respectively");
-		}
+	static {
 		addDefaultResource("core-default-shaded.xml");
-		addDefaultResource("core-site.xml");
+		addDefaultResource("core-default-testing.xml");
 	}
 
 	private Properties properties;
diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-default-testing.xml
similarity index 100%
copy from flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml
copy to flink-filesystems/flink-s3-fs-presto/src/test/resources/core-default-testing.xml
diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml b/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml
index 85c1f0d..bddcdc8 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml
+++ b/flink-filesystems/flink-s3-fs-presto/src/test/resources/core-site.xml
@@ -1,1978 +1,31 @@
 <?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
 <!--
-   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
+  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.
+  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.
 -->
 
-<!-- Do not modify this file directly.  Instead, copy entries that you -->
-<!-- wish to modify from this file into core-site.xml and change them -->
-<!-- there.  If core-site.xml does not already exist, create it.      -->
-
-<configuration>
-
-<!--- global properties -->
-
-<property>
-  <name>hadoop.common.configuration.version</name>
-  <value>0.23.0</value>
-  <description>version of this configuration file</description>
-</property>
-
-<property>
-  <name>hadoop.tmp.dir</name>
-  <value>/tmp/hadoop-${user.name}</value>
-  <description>A base for other temporary directories.</description>
-</property>
-
-<property>
-  <name>io.native.lib.available</name>
-  <value>true</value>
-  <description>Controls whether to use native libraries for bz2 and zlib
-    compression codecs or not. The property does not control any other native
-    libraries.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.filter.initializers</name>
-  <value>org.apache.hadoop.http.lib.StaticUserWebFilter</value>
-  <description>A comma separated list of class names. Each class in the list 
-  must extend org.apache.hadoop.http.FilterInitializer. The corresponding 
-  Filter will be initialized. Then, the Filter will be applied to all user 
-  facing jsp and servlet web pages.  The ordering of the list defines the 
-  ordering of the filters.</description>
-</property>
-
-<!--- security properties -->
-
-<property>
-  <name>hadoop.security.authorization</name>
-  <value>false</value>
-  <description>Is service-level authorization enabled?</description>
-</property>
-
-<property>
-  <name>hadoop.security.instrumentation.requires.admin</name>
-  <value>false</value>
-  <description>
-    Indicates if administrator ACLs are required to access
-    instrumentation servlets (JMX, METRICS, CONF, STACKS).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.authentication</name>
-  <value>simple</value>
-  <description>Possible values are simple (no authentication), and kerberos
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping</name>
-  <value>org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback</value>
-  <description>
-    Class for user to group mapping (get groups for a given user) for ACL. 
-    The default implementation,
-    org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback, 
-    will determine if the Java Native Interface (JNI) is available. If JNI is 
-    available the implementation will use the API within hadoop to resolve a 
-    list of groups for a user. If JNI is not available then the shell 
-    implementation, ShellBasedUnixGroupsMapping, is used.  This implementation 
-    shells out to the Linux/Unix environment with the 
-    <code>bash -c groups</code> command to resolve a list of groups for a user.
-  </description>
-</property>
-
-<!-- 
-=== Multiple group mapping providers configuration sample === 
-  This sample illustrates a typical use case for CompositeGroupsMapping where
-Hadoop authentication uses MIT Kerberos which trusts an AD realm. In this case, service
-principals such as hdfs, mapred, hbase, hive, oozie and etc can be placed in In MIT Kerberos,
-but end users are just from the trusted AD. For the service principals, ShellBasedUnixGroupsMapping
-provider can be used to query their groups for efficiency, and for end users, LdapGroupsMapping 
-provider can be used. This avoids to add group entries in AD for service principals when only using 
-LdapGroupsMapping provider.
-  In case multiple ADs are involved and trusted by the MIT Kerberos in this use case, LdapGroupsMapping
-provider can be used more times with different AD specific configurations. This sample also shows how
-to do that. Here are the necessary configurations.
-
-<property>
-  <name>hadoop.security.group.mapping</name>
-  <value>org.apache.hadoop.security.CompositeGroupsMapping</value>
-  <description>
-    Class for user to group mapping (get groups for a given user) for ACL, which 
-    makes use of other multiple providers to provide the service.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers</name>
-  <value>shell4services,ad4usersX,ad4usersY</value>
-  <description>
-    Comma separated of names of other providers to provide user to group mapping. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers.combined</name>
-  <value>true</value>
-  <description>
-    true or false to indicate whether groups from the providers are combined or not. The default value is true
-    If true, then all the providers will be tried to get groups and all the groups are combined to return as
-    the final results. Otherwise, providers are tried one by one in the configured list order, and if any
-    groups are retrieved from any provider, then the groups will be returned without trying the left ones.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.provider.shell4services</name>
-  <value>org.apache.hadoop.security.ShellBasedUnixGroupsMapping</value>
-  <description>
-    Class for group mapping provider named by 'shell4services'. The name can then be referenced 
-    by hadoop.security.group.mapping.providers property.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.provider.ad4usersX</name>
-  <value>org.apache.hadoop.security.LdapGroupsMapping</value>
-  <description>
-    Class for group mapping provider named by 'ad4usersX'. The name can then be referenced 
-    by hadoop.security.group.mapping.providers property.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.provider.ad4usersY</name>
-  <value>org.apache.hadoop.security.LdapGroupsMapping</value>
-  <description>
-    Class for group mapping provider named by 'ad4usersY'. The name can then be referenced 
-    by hadoop.security.group.mapping.providers property.
-  </description>
-</property>
-
-<property>
-<name>hadoop.security.group.mapping.provider.ad4usersX.ldap.url</name>
-<value>ldap://ad-host-for-users-X:389</value>
-  <description>
-    ldap url for the provider named by 'ad4usersX'. Note this property comes from 
-    'hadoop.security.group.mapping.ldap.url'.
-  </description>
-</property>
-
-<property>
-<name>hadoop.security.group.mapping.provider.ad4usersY.ldap.url</name>
-<value>ldap://ad-host-for-users-Y:389</value>
-  <description>
-    ldap url for the provider named by 'ad4usersY'. Note this property comes from 
-    'hadoop.security.group.mapping.ldap.url'.
-  </description>
-</property>
-
-You also need to configure other properties like
-  hadoop.security.group.mapping.ldap.bind.password.file and etc.
-for ldap providers in the same way as above does.
-
+<!--
+A configuration as it may accidentally be in the classpath when Hadoop is
+in the classpath. Used for tests checking that the presence of such a config
+does not pollute the settings.
 -->
- 
-<property>
-  <name>hadoop.security.groups.cache.secs</name>
-  <value>300</value>
-  <description>
-    This is the config controlling the validity of the entries in the cache
-    containing the user->group mapping. When this duration has expired,
-    then the implementation of the group mapping provider is invoked to get
-    the groups of the user and then cached back.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.negative-cache.secs</name>
-  <value>30</value>
-  <description>
-    Expiration time for entries in the the negative user-to-group mapping
-    caching, in seconds. This is useful when invalid users are retrying
-    frequently. It is suggested to set a small value for this expiration, since
-    a transient error in group lookup could temporarily lock out a legitimate
-    user.
-
-    Set this to zero or negative value to disable negative user-to-group caching.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.warn.after.ms</name>
-  <value>5000</value>
-  <description>
-    If looking up a single user to group takes longer than this amount of
-    milliseconds, we will log a warning message.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.url</name>
-  <value></value>
-  <description>
-    The URL of the LDAP server to use for resolving user groups when using
-    the LdapGroupsMapping user to group mapping.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl</name>
-  <value>false</value>
-  <description>
-    Whether or not to use SSL when connecting to the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore</name>
-  <value></value>
-  <description>
-    File path to the SSL keystore that contains the SSL certificate required
-    by the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the LDAP SSL keystore.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.user</name>
-  <value></value>
-  <description>
-    The distinguished name of the user to bind as when connecting to the LDAP
-    server. This may be left blank if the LDAP server supports anonymous binds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the bind user.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.base</name>
-  <value></value>
-  <description>
-    The search base for the LDAP connection. This is a distinguished name,
-    and will typically be the root of the LDAP directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.user</name>
-  <value>(&amp;(objectClass=user)(sAMAccountName={0}))</value>
-  <description>
-    An additional filter to use when searching for LDAP users. The default will
-    usually be appropriate for Active Directory installations. If connecting to
-    an LDAP server with a non-AD schema, this should be replaced with
-    (&amp;(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to
-    denote where the username fits into the filter.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.group</name>
-  <value>(objectClass=group)</value>
-  <description>
-    An additional filter to use when searching for LDAP groups. This should be
-    changed when resolving groups against a non-Active Directory installation.
-    posixGroups are currently not a supported group class.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.member</name>
-  <value>member</value>
-  <description>
-    The attribute of the group object that identifies the users that are
-    members of the group. The default will usually be appropriate for
-    any LDAP installation.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.group.name</name>
-  <value>cn</value>
-  <description>
-    The attribute of the group object that identifies the group name. The
-    default will usually be appropriate for all LDAP systems.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.directory.search.timeout</name>
-  <value>10000</value>
-  <description>
-    The attribute applied to the LDAP SearchControl properties to set a
-    maximum time limit when searching and awaiting a result.
-    Set to 0 if infinite wait period is desired.
-    Default is 10 seconds. Units in milliseconds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.service.user.name.key</name>
-  <value></value>
-  <description>
-    For those cases where the same RPC protocol is implemented by multiple
-    servers, this configuration is required for specifying the principal
-    name to use for the service when the client wishes to make an RPC call.
-  </description>
-</property>
-
-
-<property>
-    <name>hadoop.security.uid.cache.secs</name>
-    <value>14400</value>
-    <description>
-        This is the config controlling the validity of the entries in the cache
-        containing the userId to userName and groupId to groupName used by
-        NativeIO getFstat().
-    </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.protection</name>
-  <value>authentication</value>
-  <description>A comma-separated list of protection values for secured sasl 
-      connections. Possible values are authentication, integrity and privacy.
-      authentication means authentication only and no integrity or privacy; 
-      integrity implies authentication and integrity are enabled; and privacy 
-      implies all of authentication, integrity and privacy are enabled.
-      hadoop.security.saslproperties.resolver.class can be used to override
-      the hadoop.rpc.protection for a connection at the server side.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.saslproperties.resolver.class</name>
-  <value></value>
-  <description>SaslPropertiesResolver used to resolve the QOP used for a 
-      connection. If not specified, the full set of values specified in 
-      hadoop.rpc.protection is used while determining the QOP used for the 
-      connection. If a class is specified, then the QOP values returned by 
-      the class will be used while determining the QOP used for the connection.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.work.around.non.threadsafe.getpwuid</name>
-  <value>false</value>
-  <description>Some operating systems or authentication modules are known to
-  have broken implementations of getpwuid_r and getpwgid_r, such that these
-  calls are not thread-safe. Symptoms of this problem include JVM crashes
-  with a stack trace inside these functions. If your system exhibits this
-  issue, enable this configuration parameter to include a lock around the
-  calls as a workaround.
-
-  An incomplete list of some systems known to have this issue is available
-  at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations
-  </description>
-</property>
-
-<property>
-  <name>hadoop.kerberos.kinit.command</name>
-  <value>kinit</value>
-  <description>Used to periodically renew Kerberos credentials when provided
-  to Hadoop. The default setting assumes that kinit is in the PATH of users
-  running the Hadoop client. Change this to the absolute path to kinit if this
-  is not the case.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.auth_to_local</name>
-  <value></value>
-  <description>Maps kerberos principals to local user names</description>
-</property>
-
-<!-- i/o properties -->
-<property>
-  <name>io.file.buffer.size</name>
-  <value>4096</value>
-  <description>The size of buffer for use in sequence files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-  
-<property>
-  <name>io.bytes.per.checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  io.file.buffer.size.</description>
-</property>
-
-<property>
-  <name>io.skip.checksum.errors</name>
-  <value>false</value>
-  <description>If true, when a checksum error is encountered while
-  reading a sequence file, entries are skipped, instead of throwing an
-  exception.</description>
-</property>
-
-<property>
-  <name>io.compression.codecs</name>
-  <value></value>
-  <description>A comma-separated list of the compression codec classes that can
-  be used for compression/decompression. In addition to any classes specified
-  with this property (which take precedence), codec classes on the classpath
-  are discovered using a Java ServiceLoader.</description>
-</property>
-
-<property>
-  <name>io.compression.codec.bzip2.library</name>
-  <value>system-native</value>
-  <description>The native-code library to be used for compression and
-  decompression by the bzip2 codec.  This library could be specified
-  either by by name or the full pathname.  In the former case, the
-  library is located by the dynamic linker, usually searching the
-  directories specified in the environment variable LD_LIBRARY_PATH.
-  
-  The value of "system-native" indicates that the default system
-  library should be used.  To indicate that the algorithm should
-  operate entirely in Java, specify "java-builtin".</description>
-</property>
-
-<property>
-  <name>io.serializations</name>
-  <value>org.apache.hadoop.io.serializer.WritableSerialization,org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization,org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value>
-  <description>A list of serialization classes that can be used for
-  obtaining serializers and deserializers.</description>
-</property>
-
-<property>
-  <name>io.seqfile.local.dir</name>
-  <value>${hadoop.tmp.dir}/io/local</value>
-  <description>The local directory where sequence file stores intermediate
-  data files during merge.  May be a comma-separated list of
-  directories on different devices in order to spread disk i/o.
-  Directories that do not exist are ignored.
-  </description>
-</property>
-
-<property>
-  <name>io.map.index.skip</name>
-  <value>0</value>
-  <description>Number of index entries to skip between each entry.
-  Zero by default. Setting this to values larger than zero can
-  facilitate opening large MapFiles using less memory.</description>
-</property>
-
-<property>
-  <name>io.map.index.interval</name>
-  <value>128</value>
-  <description>
-    MapFile consist of two files - data file (tuples) and index file
-    (keys). For every io.map.index.interval records written in the
-    data file, an entry (record-key, data-file-position) is written
-    in the index file. This is to allow for doing binary search later
-    within the index file to look up records by their keys and get their
-    closest positions in the data file.
-  </description>
-</property>
-
-<!-- file system properties -->
-
-<property>
-  <name>fs.defaultFS</name>
-  <value>file:///</value>
-  <description>The name of the default file system.  A URI whose
-  scheme and authority determine the FileSystem implementation.  The
-  uri's scheme determines the config property (fs.SCHEME.impl) naming
-  the FileSystem implementation class.  The uri's authority is used to
-  determine the host, port, etc. for a filesystem.</description>
-</property>
-
-<property>
-  <name>fs.default.name</name>
-  <value>file:///</value>
-  <description>Deprecated. Use (fs.defaultFS) property
-  instead</description>
-</property>
-
-<property>
-  <name>fs.trash.interval</name>
-  <value>0</value>
-  <description>Number of minutes after which the checkpoint
-  gets deleted.  If zero, the trash feature is disabled.
-  This option may be configured both on the server and the
-  client. If trash is disabled server side then the client
-  side configuration is checked. If trash is enabled on the
-  server side then the value configured on the server is
-  used and the client configuration value is ignored.
-  </description>
-</property>
-
-<property>
-  <name>fs.trash.checkpoint.interval</name>
-  <value>0</value>
-  <description>Number of minutes between trash checkpoints.
-  Should be smaller or equal to fs.trash.interval. If zero,
-  the value is set to the value of fs.trash.interval.
-  Every time the checkpointer runs it creates a new checkpoint 
-  out of current and removes checkpoints created more than 
-  fs.trash.interval minutes ago.
-  </description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.file.impl</name>
-  <value>org.apache.hadoop.fs.local.LocalFs</value>
-  <description>The AbstractFileSystem for file: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.har.impl</name>
-  <value>org.apache.hadoop.fs.HarFs</value>
-  <description>The AbstractFileSystem for har: uris.</description>
-</property> 
-
-<property>
-  <name>fs.AbstractFileSystem.hdfs.impl</name>
-  <value>org.apache.hadoop.fs.Hdfs</value>
-  <description>The FileSystem for hdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.viewfs.impl</name>
-  <value>org.apache.hadoop.fs.viewfs.ViewFs</value>
-  <description>The AbstractFileSystem for view file system for viewfs: uris
-  (ie client side mount table:).</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.ftp.impl</name>
-  <value>org.apache.hadoop.fs.ftp.FtpFs</value>
-  <description>The FileSystem for Ftp: uris.</description>
-</property>
-
-<property>
-  <name>fs.ftp.host</name>
-  <value>0.0.0.0</value>
-  <description>FTP filesystem connects to this server</description>
-</property>
-
-<property>
-  <name>fs.ftp.host.port</name>
-  <value>21</value>
-  <description>
-    FTP filesystem connects to fs.ftp.host on this port
-  </description>
-</property>
-
-<property>
-  <name>fs.df.interval</name>
-  <value>60000</value>
-  <description>Disk usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.du.interval</name>
-  <value>600000</value>
-  <description>File space usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.s3.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when writing files to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3</value>
-  <description>Determines where on the local filesystem the S3 filesystem
-  should store files before sending them to S3
-  (or after retrieving them from S3).
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.maxRetries</name>
-  <value>4</value>
-  <description>The maximum number of retries for reading or writing files to S3, 
-  before we signal failure to the application.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.sleepTimeSeconds</name>
-  <value>10</value>
-  <description>The number of seconds to sleep between each S3 retry.
-  </description>
-</property>
-
-<property>
-  <name>fs.swift.impl</name>
-  <value>org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem</value>
-  <description>The implementation class of the OpenStack Swift Filesystem</description>
-</property>
-
-<property>
-  <name>fs.automatic.close</name>
-  <value>true</value>
-  <description>By default, FileSystem instances are automatically closed at program
-  exit using a JVM shutdown hook. Setting this property to false disables this
-  behavior. This is an advanced option that should only be used by server applications
-  requiring a more carefully orchestrated shutdown sequence.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when reading files using the native S3
-  filesystem (s3n: URIs).</description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.enabled</name>
-  <value>false</value>
-  <description>Setting this property to true enables multiple uploads to
-  native S3 filesystem. When uploading a file, it is split into blocks
-  if the size is larger than fs.s3n.multipart.uploads.block.size.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.block.size</name>
-  <value>67108864</value>
-  <description>The block size for multipart uploads to native S3 filesystem.
-  Default size is 64MB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.copy.block.size</name>
-  <value>5368709120</value>
-  <description>The block size for multipart copy in native S3 filesystem.
-  Default size is 5GB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.server-side-encryption-algorithm</name>
-  <value></value>
-  <description>Specify a server-side encryption algorithm for S3.
-  The default is NULL, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.awsAccessKeyId</name>
-  <description>AWS access key ID. Omit for Role-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.awsSecretAccessKey</name>
-  <description>AWS secret key. Omit for Role-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.maximum</name>
-  <value>15</value>
-  <description>Controls the maximum number of simultaneous connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.ssl.enabled</name>
-  <value>true</value>
-  <description>Enables or disables SSL connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.endpoint</name>
-  <description>AWS S3 endpoint to connect to. An up-to-date list is
-    provided in the AWS Documentation: regions and endpoints. Without this
-    property, the standard region (s3.amazonaws.com) is assumed.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.host</name>
-  <description>Hostname of the (optional) proxy server for S3 connections.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.port</name>
-  <description>Proxy server port. If this property is not set
-    but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with
-    the value of fs.s3a.connection.ssl.enabled).</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.username</name>
-  <description>Username for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.password</name>
-  <description>Password for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.domain</name>
-  <description>Domain for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.workstation</name>
-  <description>Workstation for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.attempts.maximum</name>
-  <value>10</value>
-  <description>How many times we should retry commands on transient errors.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.establish.timeout</name>
-  <value>5000</value>
-  <description>Socket connection setup timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.timeout</name>
-  <value>50000</value>
-  <description>Socket connection timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.paging.maximum</name>
-  <value>5000</value>
-  <description>How many keys to request from S3 when doing 
-     directory listings at a time.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.max</name>
-  <value>256</value>
-  <description> Maximum number of concurrent active (part)uploads,
-    which each use a thread from the threadpool.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.core</name>
-  <value>15</value>
-  <description>Number of core threads in the threadpool.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.keepalivetime</name>
-  <value>60</value>
-  <description>Number of seconds a thread can be idle before being
-    terminated.</description>
-</property>
-
-<property>
-  <name>fs.s3a.max.total.tasks</name>
-  <value>1000</value>
-  <description>Number of (part)uploads allowed to the queue before
-    blocking additional uploads.</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.size</name>
-  <value>104857600</value>
-  <description>How big (in bytes) to split upload or copy operations up into.</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.threshold</name>
-  <value>2147483647</value>
-  <description>Threshold before uploads or copies use parallel multipart operations.</description>
-</property>
-
-<property>
-  <name>fs.s3a.acl.default</name>
-  <description>Set a canned ACL for newly created and copied objects. Value may be private, 
-     public-read, public-read-write, authenticated-read, log-delivery-write, 
-     bucket-owner-read, or bucket-owner-full-control.</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge</name>
-  <value>false</value>
-  <description>True if you want to purge existing multipart uploads that may not have been
-     completed/aborted correctly</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge.age</name>
-  <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge</description>
-</property>
-
-<property>
-  <name>fs.s3a.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3a</value>
-  <description>Comma separated list of directories that will be used to buffer file 
-    uploads to.</description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload</name>
-  <value>false</value>
-  <description>Upload directly from memory instead of buffering to
-    disk first. Memory usage and parallelism can be controlled as up to
-    fs.s3a.multipart.size memory is consumed for each (part)upload actively
-    uploading (fs.s3a.threads.max) or queueing (fs.s3a.max.total.tasks)</description>
-</property>
-
-  <property>
-  <name>fs.s3a.fast.buffer.size</name>
-  <value>1048576</value>
-  <description>Size of initial memory buffer in bytes allocated for an
-    upload. No effect if fs.s3a.fast.upload is false.</description>
-</property>
-
-<property>
-  <name>fs.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-  <description>The implementation class of the S3A Filesystem</description>
-</property>
-
-<property>
-  <name>io.seqfile.compress.blocksize</name>
-  <value>1000000</value>
-  <description>The minimum block size for compression in block compressed 
-          SequenceFiles.
-  </description>
-</property>
-
-<property>
-  <name>io.seqfile.lazydecompress</name>
-  <value>true</value>
-  <description>Should values of block-compressed SequenceFiles be decompressed
-          only when necessary.
-  </description>
-</property>
-
-<property>
-  <name>io.seqfile.sorter.recordlimit</name>
-  <value>1000000</value>
-  <description>The limit on number of records to be kept in memory in a spill 
-          in SequenceFiles.Sorter
-  </description>
-</property>
-
- <property>
-  <name>io.mapfile.bloom.size</name>
-  <value>1048576</value>
-  <description>The size of BloomFilter-s used in BloomMapFile. Each time this many
-  keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter).
-  Larger values minimize the number of filters, which slightly increases the performance,
-  but may waste too much space if the total number of keys is usually much smaller
-  than this number.
-  </description>
-</property>
-
-<property>
-  <name>io.mapfile.bloom.error.rate</name>
-  <value>0.005</value>
-  <description>The rate of false positives in BloomFilter-s used in BloomMapFile.
-  As this value decreases, the size of BloomFilter-s increases exponentially. This
-  value is the probability of encountering false positives (default is 0.5%).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.util.hash.type</name>
-  <value>murmur</value>
-  <description>The default implementation of Hash. Currently this can take one of the
-  two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash.
-  </description>
-</property>
-
-
-<!-- ipc properties -->
-
-<property>
-  <name>ipc.client.idlethreshold</name>
-  <value>4000</value>
-  <description>Defines the threshold number of connections after which
-               connections will be inspected for idleness.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.kill.max</name>
-  <value>10</value>
-  <description>Defines the maximum number of clients to disconnect in one go.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connection.maxidletime</name>
-  <value>10000</value>
-  <description>The maximum time in msec after which a client will bring down the
-               connection to the server.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries</name>
-  <value>10</value>
-  <description>Indicates the number of retries a client will make to establish
-               a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.retry.interval</name>
-  <value>1000</value>
-  <description>Indicates the number of milliseconds a client will wait for
-    before retrying to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.timeout</name>
-  <value>20000</value>
-  <description>Indicates the number of milliseconds a client will wait for the 
-               socket to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries.on.timeouts</name>
-  <value>45</value>
-  <description>Indicates the number of retries a client will make on socket timeout
-               to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.ping</name>
-  <value>true</value>
-  <description>Send a ping to the server when timeout on reading the response,
-  if set to true. If no failure is detected, the client retries until at least
-  a byte is read.
-  </description>
-</property>
-
-<property>
-  <name>ipc.ping.interval</name>
-  <value>60000</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  The client will send ping when the interval is passed without receiving bytes,
-  if ipc.client.ping is set to true.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.rpc-timeout.ms</name>
-  <value>0</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  Currently this timeout works only when ipc.client.ping is set to true
-  because it uses the same facilities with IPC ping.
-  The timeout overrides the ipc.ping.interval and client will throw exception
-  instead of sending ping when the interval is passed.
-  </description>
-</property>
-
-<property>
-  <name>ipc.server.listen.queue.size</name>
-  <value>128</value>
-  <description>Indicates the length of the listen queue for servers accepting
-               client connections.
-  </description>
-</property>
-
-<property>
-  <name>ipc.maximum.data.length</name>
-  <value>67108864</value>
-  <description>This indicates the maximum IPC message length (bytes) that can be
-    accepted by the server. Messages larger than this value are rejected by
-    server immediately. This setting should rarely need to be changed. It merits
-    investigating whether the cause of long RPC messages can be fixed instead,
-    e.g. by splitting into smaller messages.
-  </description>
-</property>
-
-<!-- Proxy Configuration -->
-
-<property>
-  <name>hadoop.security.impersonation.provider.class</name>
-  <value></value>
-  <description>A class which implements ImpersonationProvider interface, used to 
-       authorize whether one user can impersonate a specific user. 
-       If not specified, the DefaultImpersonationProvider will be used. 
-       If a class is specified, then that class will be used to determine 
-       the impersonation capability.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.default</name>
-  <value>org.apache.hadoop.net.StandardSocketFactory</value>
-  <description> Default SocketFactory to use. This parameter is expected to be
-    formatted as "package.FactoryClassName".
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.ClientProtocol</name>
-  <value></value>
-  <description> SocketFactory to use to connect to a DFS. If null or empty, use
-    hadoop.rpc.socket.class.default. This socket factory is also used by
-    DFSClient to create sockets to DataNodes.
-  </description>
-</property>
-
-
-
-<property>
-  <name>hadoop.socks.server</name>
-  <value></value>
-  <description> Address (host:port) of the SOCKS server to be used by the
-    SocksSocketFactory.
-  </description>
-</property>
-
-<!-- Topology Configuration -->
-<property>
-  <name>net.topology.node.switch.mapping.impl</name>
-  <value>org.apache.hadoop.net.ScriptBasedMapping</value>
-  <description> The default implementation of the DNSToSwitchMapping. It
-    invokes a script specified in net.topology.script.file.name to resolve
-    node names. If the value for net.topology.script.file.name is not set, the
-    default value of DEFAULT_RACK is returned for all node names.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.impl</name>
-  <value>org.apache.hadoop.net.NetworkTopology</value>
-  <description> The default implementation of NetworkTopology which is classic three layer one.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.file.name</name>
-  <value></value>
-  <description> The script name that should be invoked to resolve DNS names to
-    NetworkTopology names. Example: the script would take host.foo.bar as an
-    argument, and return /rack1 as the output.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.number.args</name>
-  <value>100</value>
-  <description> The max number of args that the script configured with 
-    net.topology.script.file.name should be run with. Each arg is an
-    IP address.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.table.file.name</name>
-  <value></value>
-  <description> The file name for a topology file, which is used when the
-    net.topology.node.switch.mapping.impl property is set to
-    org.apache.hadoop.net.TableMapping. The file format is a two column text
-    file, with columns separated by whitespace. The first column is a DNS or
-    IP address and the second column specifies the rack where the address maps.
-    If no entry corresponding to a host in the cluster is found, then 
-    /default-rack is assumed.
-  </description>
-</property>
-
-<!-- Local file system -->
-<property>
-  <name>file.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>file.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  file.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>file.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>file.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>file.replication</name>
-  <value>1</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3 File System -->
-
-<property>
-  <name>s3.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3native File System -->
-
-<property>
-  <name>s3native.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3native.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3native.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3native.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3native.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3native.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- FTP file system -->
-<property>
-  <name>ftp.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>ftp.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  ftp.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>ftp.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>ftp.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>ftp.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- Tfile -->
-
-<property>
-  <name>tfile.io.chunk.size</name>
-  <value>1048576</value>
-  <description>
-    Value chunk size in bytes. Default  to
-    1MB. Values of the length less than the chunk size is
-    guaranteed to have known value length in read time (See also
-    TFile.Reader.Scanner.Entry.isValueLengthKnown()).
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.output.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataOutputStream in bytes.
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.input.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataInputStream in bytes.
-  </description>
-</property>
-
-<!-- HTTP web-consoles Authentication -->
-
-<property>
-  <name>hadoop.http.authentication.type</name>
-  <value>simple</value>
-  <description>
-    Defines authentication used for Oozie HTTP endpoint.
-    Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.token.validity</name>
-  <value>36000</value>
-  <description>
-    Indicates how long (in seconds) an authentication token is valid before it has
-    to be renewed.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.signature.secret.file</name>
-  <value>${user.home}/hadoop-http-auth-signature-secret</value>
-  <description>
-    The signature secret for signing the authentication tokens.
-    The same secret should be used for JT/NN/DN/TT configurations.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.cookie.domain</name>
-  <value></value>
-  <description>
-    The domain to use for the HTTP cookie that stores the authentication token.
-    In order to authentiation to work correctly across all Hadoop nodes web-consoles
-    the domain must be correctly set.
-    IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings.
-    For this setting to work properly all nodes in the cluster must be configured
-    to generate URLs with hostname.domain names on it.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.simple.anonymous.allowed</name>
-  <value>true</value>
-  <description>
-    Indicates if anonymous requests are allowed when using 'simple' authentication.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.principal</name>
-  <value>HTTP/_HOST@LOCALHOST</value>
-  <description>
-    Indicates the Kerberos principal to be used for HTTP endpoint.
-    The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.keytab</name>
-  <value>${user.home}/hadoop.keytab</value>
-  <description>
-    Location of the keytab file with the credentials for the principal.
-    Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop.
-  </description>
-</property>
-
-<!-- HTTP CORS support -->
-<property>
-  <description>Enable/disable the cross-origin (CORS) filter.</description>
-  <name>hadoop.http.cross-origin.enabled</name>
-  <value>false</value>
-</property>
-
-<property>
-  <description>Comma separated list of origins that are allowed for web
-    services needing cross-origin (CORS) support. Wildcards (*) and patterns
-    allowed</description>
-  <name>hadoop.http.cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
-
-<property>
-  <description>Comma separated list of methods that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
-
-<property>
-  <description>Comma separated list of headers that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
-
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-    for web services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.methods</name>
-  <value></value>
-  <description>
-    List of fencing methods to use for service fencing. May contain
-    builtin methods (eg shell and sshfence) or user-defined method.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.connect-timeout</name>
-  <value>30000</value>
-  <description>
-    SSH connection timeout, in milliseconds, to use with the builtin
-    sshfence fencer.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.private-key-files</name>
-  <value></value>
-  <description>
-    The SSH private key files to use with the builtin sshfence fencer.
-  </description>
-</property>
-
-
-<!-- Static Web User Filter properties. -->
-<property>
-  <description>
-    The user name to filter as, on static web filters
-    while rendering content. An example use is the HDFS
-    web UI (user to be used for browsing files).
-  </description>
-  <name>hadoop.http.staticuser.user</name>
-  <value>dr.who</value>
-</property>
-
-<property>
-  <name>ha.zookeeper.quorum</name>
-  <description>
-    A list of ZooKeeper server addresses, separated by commas, that are
-    to be used by the ZKFailoverController in automatic failover.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.session-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    The session timeout to use when the ZKFC connects to ZooKeeper.
-    Setting this value to a lower value implies that server crashes
-    will be detected more quickly, but risks triggering failover too
-    aggressively in the case of a transient error or network blip.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.parent-znode</name>
-  <value>/hadoop-ha</value>
-  <description>
-    The ZooKeeper znode under which the ZK failover controller stores
-    its information. Note that the nameservice ID is automatically
-    appended to this znode, so it is not normally necessary to
-    configure this, even in a federated environment.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.acl</name>
-  <value>world:anyone:rwcda</value>
-  <description>
-    A comma-separated list of ZooKeeper ACLs to apply to the znodes
-    used by automatic failover. These ACLs are specified in the same
-    format as used by the ZooKeeper CLI.
-
-    If the ACL itself contains secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.auth</name>
-  <value></value>
-  <description>
-    A comma-separated list of ZooKeeper authentications to add when
-    connecting to ZooKeeper. These are specified in the same format
-    as used by the &quot;addauth&quot; command in the ZK CLI. It is
-    important that the authentications specified here are sufficient
-    to access znodes with the ACL specified in ha.zookeeper.acl.
-
-    If the auths contain secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<!-- SSLFactory configuration -->
-
-<property>
-  <name>hadoop.ssl.keystores.factory.class</name>
-  <value>org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory</value>
-  <description>
-    The keystores factory to use for retrieving certificates.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.require.client.cert</name>
-  <value>false</value>
-  <description>Whether client certificates are required</description>
-</property>
-
-<property>
-  <name>hadoop.ssl.hostname.verifier</name>
-  <value>DEFAULT</value>
-  <description>
-    The hostname verifier to provide for HttpsURLConnections.
-    Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and
-    ALLOW_ALL
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.server.conf</name>
-  <value>ssl-server.xml</value>
-  <description>
-    Resource file from which ssl server keystore information will be extracted.
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.client.conf</name>
-  <value>ssl-client.xml</value>
-  <description>
-    Resource file from which ssl client keystore information will be extracted
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled</name>
-  <value>false</value>
-  <description>
-    Deprecated. Use dfs.http.policy and yarn.http.policy instead.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled.protocols</name>
-  <value>TLSv1</value>
-  <description>
-    Protocols supported by the ssl.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.jetty.logs.serve.aliases</name>
-  <value>true</value>
-  <description>
-    Enable/Disable aliases serving from jetty
-  </description>
-</property>
-
-<property>
-  <name>fs.permissions.umask-mode</name>
-  <value>022</value>
-  <description>
-    The umask used when creating files and directories.
-    Can be in octal or in symbolic. Examples are:
-    "022" (octal for u=rwx,g=r-x,o=r-x in symbolic),
-    or "u=rwx,g=rwx,o=" (symbolic for 007 in octal).
-  </description>
-</property>
-
-<!-- ha properties -->
-
-<property>
-  <name>ha.health-monitor.connect-retry-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to retry connecting to the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.check-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to check the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.sleep-after-disconnect.ms</name>
-  <value>1000</value>
-  <description>
-    How long to sleep after an unexpected RPC error.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.rpc-timeout.ms</name>
-  <value>45000</value>
-  <description>
-    Timeout for the actual monitorHealth() calls.
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.new-active.rpc-timeout.ms</name>
-  <value>60000</value>
-  <description>
-    Timeout that the FC waits for the new active to become active
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.rpc-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    Timeout that the FC waits for the old active to go to standby
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.connection.retries</name>
-  <value>1</value>
-  <description>
-    FC connection retries for graceful fencing
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.cli-check.rpc-timeout.ms</name>
-  <value>20000</value>
-  <description>
-    Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.fallback-to-simple-auth-allowed</name>
-  <value>false</value>
-  <description>
-    When a client is configured to attempt a secure connection, but attempts to
-    connect to an insecure server, that server may instruct the client to
-    switch to SASL SIMPLE (unsecure) authentication. This setting controls
-    whether or not the client will accept this instruction from the server.
-    When false (the default), the client will not allow the fallback to SIMPLE
-    authentication, and will abort the connection.
-  </description>
-</property>
-
-<property>
-  <name>fs.client.resolve.remote.symlinks</name>
-  <value>true</value>
-  <description>
-      Whether to resolve symlinks when accessing a remote Hadoop filesystem.
-      Setting this to false causes an exception to be thrown upon encountering
-      a symlink. This setting does not apply to local filesystems, which
-      automatically resolve local symlinks.
-  </description>
-</property>
-
-<property>
-  <name>nfs.exports.allowed.hosts</name>
-  <value>* rw</value>
-  <description>
-    By default, the export can be mounted by any client. The value string 
-    contains machine name and access privilege, separated by whitespace 
-    characters. The machine name format can be a single host, a Java regular 
-    expression, or an IPv4 address. The access privilege uses rw or ro to 
-    specify read/write or read-only access of the machines to exports. If the 
-    access privilege is not provided, the default is read-only. Entries are separated by ";".
-    For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;".
-    Only the NFS gateway needs to restart after this property is updated. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.user.group.static.mapping.overrides</name>
-  <value>dr.who=;</value>
-  <description>
-    Static mapping of user to groups. This will override the groups if
-    available in the system for the specified user. In otherwords, groups
-    look-up will not happen for these users, instead groups mapped in this
-    configuration will be used.
-    Mapping should be in this format.
-    user1=group1,group2;user2=;user3=group2;
-    Default, "dr.who=;" will consider "dr.who" as user without groups.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.quantile.enable</name>
-  <value>false</value>
-  <description>
-    Setting this property to true and rpc.metrics.percentiles.intervals
-    to a comma-separated list of the granularity in seconds, the
-    50/75/90/95/99th percentile latency for rpc queue/processing time in
-    milliseconds are added to rpc metrics.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.percentiles.intervals</name>
-  <value></value>
-  <description>
-    A comma-separated list of the granularity in seconds for the metrics which
-    describe the 50/75/90/95/99th percentile latency for rpc queue/processing
-    time. The metrics are outputted if rpc.metrics.quantile.enable is set to
-    true.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE</name>
-  <value></value>
-  <description>
-    The prefix for a given crypto codec, contains a comma-separated
-    list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE).
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.aes.ctr.nopadding</name>
-  <value>org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec,org.apache.hadoop.crypto.JceAesCtrCryptoCodec</value>
-  <description>
-    Comma-separated list of crypto codec implementations for AES/CTR/NoPadding. 
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.cipher.suite</name>
-  <value>AES/CTR/NoPadding</value>
-  <description>
-    Cipher suite for crypto codec.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.jce.provider</name>
-  <value></value>
-  <description>
-    The JCE provider name used in CryptoCodec. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.buffer.size</name>
-  <value>8192</value>
-  <description>
-    The buffer size used by CryptoInputStream and CryptoOutputStream. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.java.secure.random.algorithm</name>
-  <value>SHA1PRNG</value>
-  <description>
-    The java secure random algorithm. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.secure.random.impl</name>
-  <value></value>
-  <description>
-    Implementation of secure random. 
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.random.device.file.path</name>
-  <value>/dev/urandom</value>
-  <description>
-    OS security random device file path.
-  </description>
-</property>
-
-<property>
-  <name>fs.har.impl.disable.cache</name>
-  <value>true</value>
-  <description>Don't cache 'har' filesystem instances.</description>
-</property>
-
-<!--- KMSClientProvider configurations -->
-<property>
-  <name>hadoop.security.kms.client.authentication.retry-count</name>
-  <value>1</value>
-  <description>
-    Number of time to retry connecting to KMS on authentication failure
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.size</name>
-  <value>500</value>
-  <description>
-    Size of the EncryptedKeyVersion cache Queue for each key
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.low-watermark</name>
-  <value>0.3f</value>
-  <description>
-    If size of the EncryptedKeyVersion cache Queue falls below the
-    low watermark, this cache queue will be scheduled for a refill
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.num.refill.threads</name>
-  <value>2</value>
-  <description>
-    Number of threads to use for refilling depleted EncryptedKeyVersion
-    cache Queues
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.expiry</name>
-  <value>43200000</value>
-  <description>
-    Cache expiry time for a Key, after which the cache Queue for this
-    key will be dropped. Default = 12hrs
-  </description>
-</property>
-
-<property>
-  <name>hadoop.htrace.spanreceiver.classes</name>
-  <value></value>
-  <description>
-    A comma separated list of the fully-qualified class name of classes 
-    implementing SpanReceiver. The tracing system works by collecting 
-    information in structs called 'Spans'. It is up to you to choose 
-    how you want to receive this information by implementing the 
-    SpanReceiver interface.
-  </description>
-</property>
-
- <property>
-  <name>ipc.server.max.connections</name>
-  <value>0</value>
-  <description>The maximum number of concurrent connections a server is allowed
-    to accept. If this limit is exceeded, incoming connections will first fill
-    the listen queue and then may go to an OS-specific listen overflow queue. 
-    The client may fail or timeout, but the server can avoid running out of file
-    descriptors using this feature. 0 means no limit.
-  </description>
-</property>
-
-
-  <!-- YARN registry -->
-
-  <property>
-    <description>
-      Is the registry enabled in the YARN Resource Manager?
-
-      If true, the YARN RM will, as needed.
-      create the user and system paths, and purge
-      service records when containers, application attempts
-      and applications complete.
-
-      If false, the paths must be created by other means,
-      and no automatic cleanup of service records will take place.
-    </description>
-    <name>hadoop.registry.rm.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      The root zookeeper node for the registry
-    </description>
-    <name>hadoop.registry.zk.root</name>
-    <value>/registry</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper session timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.session.timeout.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.connection.timeout.ms</name>
-    <value>15000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection retry count before failing
-    </description>
-    <name>hadoop.registry.zk.retry.times</name>
-    <value>5</value>
-  </property>
-
-  <property>
-    <description>
-    </description>
-    <name>hadoop.registry.zk.retry.interval.ms</name>
-    <value>1000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper retry limit in milliseconds, during
-      exponential backoff.
-
-      This places a limit even
-      if the retry times and interval limit, combined
-      with the backoff policy, result in a long retry
-      period
-    </description>
-    <name>hadoop.registry.zk.retry.ceiling.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      List of hostname:port pairs defining the
-      zookeeper quorum binding for the registry
-    </description>
-    <name>hadoop.registry.zk.quorum</name>
-    <value>localhost:2181</value>
-  </property>
-
-  <property>
-    <description>
-      Key to set if the registry is secure. Turning it on
-      changes the permissions policy from "open access"
-      to restrictions on kerberos with the option of
-      a user adding one or more auth key pairs down their
-      own tree.
-    </description>
-    <name>hadoop.registry.secure</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      A comma separated list of Zookeeper ACL identifiers with
-      system access to the registry in a secure cluster.
-
-      These are given full access to all entries.
-
-      If there is an "@" at the end of a SASL entry it
-      instructs the registry client to append the default kerberos domain.
-    </description>
-    <name>hadoop.registry.system.acls</name>
-    <value>sasl:yarn@, sasl:mapred@, sasl:hdfs@</value>
-  </property>
-
-  <property>
-    <description>
-      The kerberos realm: used to set the realm of
-      system principals which do not declare their realm,
-      and any other accounts that need the value.
-
-      If empty, the default realm of the running process
-      is used.
-
-      If neither are known and the realm is needed, then the registry
-      service/client will fail.
-    </description>
-    <name>hadoop.registry.kerberos.realm</name>
-    <value></value>
-  </property>
-
-  <property>
-    <description>
-      Key to define the JAAS context. Used in secure
-      mode
-    </description>
-    <name>hadoop.registry.jaas.context</name>
-    <value>Client</value>
-  </property>
 
+<configuration>
+	<property>
+		<name>hadoop.security.group.mapping</name>
+		<value>this.does.not.exist.class</value>
+	</property>
 </configuration>
diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java b/flink-filesystems/flink-swift-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
index 51e4359..dd950ae 100644
--- a/flink-filesystems/flink-swift-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
+++ b/flink-filesystems/flink-swift-fs-hadoop/src/main/java/org/apache/hadoop/conf/Configuration.java
@@ -620,21 +620,9 @@ public class Configuration implements Iterable<Entry<String,String>>,
     }
   }
 
-  static{
-    //print deprecation warning if hadoop-site.xml is found in classpath
-    ClassLoader cL = Thread.currentThread().getContextClassLoader();
-    if (cL == null) {
-      cL = Configuration.class.getClassLoader();
-    }
-    if(cL.getResource("hadoop-site.xml")!=null) {
-      LOG.warn("DEPRECATED: hadoop-site.xml found in the classpath. " +
-          "Usage of hadoop-site.xml is deprecated. Instead use core-site.xml, "
-          + "mapred-site.xml and hdfs-site.xml to override properties of " +
-          "core-default-shaded.xml, mapred-default.xml and hdfs-default.xml " +
-          "respectively");
-    }
+  static {
     addDefaultResource("core-default-shaded.xml");
-    addDefaultResource("core-site.xml");
+    addDefaultResource("core-default-testing.xml");
   }
 
   private Properties properties;
diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml b/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-default-testing.xml
similarity index 100%
copy from flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml
copy to flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-default-testing.xml
diff --git a/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml b/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml
index b875e97..bddcdc8 100644
--- a/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml
+++ b/flink-filesystems/flink-swift-fs-hadoop/src/test/resources/core-site.xml
@@ -1,2312 +1,31 @@
 <?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-
 <!--
-   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
+  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.
+  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.
 -->
 
-<!-- Do not modify this file directly.  Instead, copy entries that you -->
-<!-- wish to modify from this file into core-site.xml and change them -->
-<!-- there.  If core-site.xml does not already exist, create it.      -->
+<!--
+A configuration as it may accidentally be in the classpath when Hadoop is
+in the classpath. Used for tests checking that the presence of such a config
+does not pollute the settings.
+-->
 
 <configuration>
-
-<!--- global properties -->
-
-<property>
-  <name>hadoop.common.configuration.version</name>
-  <value>0.23.0</value>
-  <description>version of this configuration file</description>
-</property>
-
-<property>
-  <name>hadoop.tmp.dir</name>
-  <value>/tmp/hadoop-${user.name}</value>
-  <description>A base for other temporary directories.</description>
-</property>
-
-<property>
-  <name>io.native.lib.available</name>
-  <value>true</value>
-  <description>Controls whether to use native libraries for bz2 and zlib
-    compression codecs or not. The property does not control any other native
-    libraries.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.filter.initializers</name>
-  <value>org.apache.hadoop.http.lib.StaticUserWebFilter</value>
-  <description>A comma separated list of class names. Each class in the list
-  must extend org.apache.hadoop.http.FilterInitializer. The corresponding
-  Filter will be initialized. Then, the Filter will be applied to all user
-  facing jsp and servlet web pages.  The ordering of the list defines the
-  ordering of the filters.</description>
-</property>
-
-<!--- security properties -->
-
-<property>
-  <name>hadoop.security.authorization</name>
-  <value>false</value>
-  <description>Is service-level authorization enabled?</description>
-</property>
-
-<property>
-  <name>hadoop.security.instrumentation.requires.admin</name>
-  <value>false</value>
-  <description>
-    Indicates if administrator ACLs are required to access
-    instrumentation servlets (JMX, METRICS, CONF, STACKS).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.authentication</name>
-  <value>simple</value>
-  <description>Possible values are simple (no authentication), and kerberos
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping</name>
-  <value>org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback</value>
-  <description>
-    Class for user to group mapping (get groups for a given user) for ACL.
-    The default implementation,
-    org.apache.hadoop.security.JniBasedUnixGroupsMappingWithFallback,
-    will determine if the Java Native Interface (JNI) is available. If JNI is
-    available the implementation will use the API within hadoop to resolve a
-    list of groups for a user. If JNI is not available then the shell
-    implementation, ShellBasedUnixGroupsMapping, is used.  This implementation
-    shells out to the Linux/Unix environment with the
-    <code>bash -c groups</code> command to resolve a list of groups for a user.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.interface</name>
-  <description>
-    The name of the Network Interface from which the service should determine
-    its host name for Kerberos login. e.g. eth2. In a multi-homed environment,
-    the setting can be used to affect the _HOST subsitution in the service
-    Kerberos principal. If this configuration value is not set, the service
-    will use its default hostname as returned by
-    InetAddress.getLocalHost().getCanonicalHostName().
-
-    Most clusters will not require this setting.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.nameserver</name>
-  <description>
-    The host name or IP address of the name server (DNS) which a service Node
-    should use to determine its own host name for Kerberos Login. Requires
-    hadoop.security.dns.interface.
-
-    Most clusters will not require this setting.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.log-slow-lookups.enabled</name>
-  <value>false</value>
-  <description>
-    Time name lookups (via SecurityUtil) and log them if they exceed the
-    configured threshold.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.dns.log-slow-lookups.threshold.ms</name>
-  <value>1000</value>
-  <description>
-    If slow lookup logging is enabled, this threshold is used to decide if a
-    lookup is considered slow enough to be logged.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.secs</name>
-  <value>300</value>
-  <description>
-    This is the config controlling the validity of the entries in the cache
-    containing the user->group mapping. When this duration has expired,
-    then the implementation of the group mapping provider is invoked to get
-    the groups of the user and then cached back.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.negative-cache.secs</name>
-  <value>30</value>
-  <description>
-    Expiration time for entries in the the negative user-to-group mapping
-    caching, in seconds. This is useful when invalid users are retrying
-    frequently. It is suggested to set a small value for this expiration, since
-    a transient error in group lookup could temporarily lock out a legitimate
-    user.
-
-    Set this to zero or negative value to disable negative user-to-group caching.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.warn.after.ms</name>
-  <value>5000</value>
-  <description>
-    If looking up a single user to group takes longer than this amount of
-    milliseconds, we will log a warning message.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.background.reload</name>
-  <value>false</value>
-  <description>
-    Whether to reload expired user->group mappings using a background thread
-    pool. If set to true, a pool of
-    hadoop.security.groups.cache.background.reload.threads is created to
-    update the cache in the background.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.groups.cache.background.reload.threads</name>
-  <value>3</value>
-  <description>
-    Only relevant if hadoop.security.groups.cache.background.reload is true.
-    Controls the number of concurrent background user->group cache entry
-    refreshes. Pending refresh requests beyond this value are queued and
-    processed when a thread is free.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.connection.timeout.ms</name>
-  <value>60000</value>
-  <description>
-    This property is the connection timeout (in milliseconds) for LDAP
-    operations. If the LDAP provider doesn't establish a connection within the
-    specified period, it will abort the connect attempt. Non-positive value
-    means no LDAP connection timeout is specified in which case it waits for the
-    connection to establish until the underlying network times out.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.read.timeout.ms</name>
-  <value>60000</value>
-  <description>
-    This property is the read timeout (in milliseconds) for LDAP
-    operations. If the LDAP provider doesn't get a LDAP response within the
-    specified period, it will abort the read attempt. Non-positive value
-    means no read timeout is specified in which case it waits for the response
-    infinitely.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.url</name>
-  <value></value>
-  <description>
-    The URL of the LDAP server to use for resolving user groups when using
-    the LdapGroupsMapping user to group mapping.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl</name>
-  <value>false</value>
-  <description>
-    Whether or not to use SSL when connecting to the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore</name>
-  <value></value>
-  <description>
-    File path to the SSL keystore that contains the SSL certificate required
-    by the LDAP server.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.ssl.keystore.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the LDAP SSL keystore.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.user</name>
-  <value></value>
-  <description>
-    The distinguished name of the user to bind as when connecting to the LDAP
-    server. This may be left blank if the LDAP server supports anonymous binds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.bind.password.file</name>
-  <value></value>
-  <description>
-    The path to a file containing the password of the bind user.
-
-    IMPORTANT: This file should be readable only by the Unix user running
-    the daemons.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.base</name>
-  <value></value>
-  <description>
-    The search base for the LDAP connection. This is a distinguished name,
-    and will typically be the root of the LDAP directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.user</name>
-  <value>(&amp;(objectClass=user)(sAMAccountName={0}))</value>
-  <description>
-    An additional filter to use when searching for LDAP users. The default will
-    usually be appropriate for Active Directory installations. If connecting to
-    an LDAP server with a non-AD schema, this should be replaced with
-    (&amp;(objectClass=inetOrgPerson)(uid={0}). {0} is a special string used to
-    denote where the username fits into the filter.
-
-    If the LDAP server supports posixGroups, Hadoop can enable the feature by
-    setting the value of this property to "posixAccount" and the value of
-    the hadoop.security.group.mapping.ldap.search.filter.group property to
-    "posixGroup".
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.filter.group</name>
-  <value>(objectClass=group)</value>
-  <description>
-    An additional filter to use when searching for LDAP groups. This should be
-    changed when resolving groups against a non-Active Directory installation.
-
-    See the description of hadoop.security.group.mapping.ldap.search.filter.user
-    to enable posixGroups support.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.member</name>
-  <value>member</value>
-  <description>
-    The attribute of the group object that identifies the users that are
-    members of the group. The default will usually be appropriate for
-    any LDAP installation.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.search.attr.group.name</name>
-  <value>cn</value>
-  <description>
-    The attribute of the group object that identifies the group name. The
-    default will usually be appropriate for all LDAP systems.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.posix.attr.uid.name</name>
-  <value>uidNumber</value>
-  <description>
-    The attribute of posixAccount to use when groups for membership.
-    Mostly useful for schemas wherein groups have memberUids that use an
-    attribute other than uidNumber.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.posix.attr.gid.name</name>
-  <value>gidNumber</value>
-  <description>
-    The attribute of posixAccount indicating the group id.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.ldap.directory.search.timeout</name>
-  <value>10000</value>
-  <description>
-    The attribute applied to the LDAP SearchControl properties to set a
-    maximum time limit when searching and awaiting a result.
-    Set to 0 if infinite wait period is desired.
-    Default is 10 seconds. Units in milliseconds.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers</name>
-  <value></value>
-  <description>
-    Comma separated of names of other providers to provide user to group
-    mapping. Used by CompositeGroupsMapping.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.group.mapping.providers.combined</name>
-  <value>true</value>
-  <description>
-    true or false to indicate whether groups from the providers are combined or
-    not. The default value is true. If true, then all the providers will be
-    tried to get groups and all the groups are combined to return as the final
-    results. Otherwise, providers are tried one by one in the configured list
-    order, and if any groups are retrieved from any provider, then the groups
-    will be returned without trying the left ones.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.service.user.name.key</name>
-  <value></value>
-  <description>
-    For those cases where the same RPC protocol is implemented by multiple
-    servers, this configuration is required for specifying the principal
-    name to use for the service when the client wishes to make an RPC call.
-  </description>
-</property>
-
-
-<property>
-    <name>hadoop.security.uid.cache.secs</name>
-    <value>14400</value>
-    <description>
-        This is the config controlling the validity of the entries in the cache
-        containing the userId to userName and groupId to groupName used by
-        NativeIO getFstat().
-    </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.protection</name>
-  <value>authentication</value>
-  <description>A comma-separated list of protection values for secured sasl
-      connections. Possible values are authentication, integrity and privacy.
-      authentication means authentication only and no integrity or privacy;
-      integrity implies authentication and integrity are enabled; and privacy
-      implies all of authentication, integrity and privacy are enabled.
-      hadoop.security.saslproperties.resolver.class can be used to override
-      the hadoop.rpc.protection for a connection at the server side.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.saslproperties.resolver.class</name>
-  <value></value>
-  <description>SaslPropertiesResolver used to resolve the QOP used for a
-      connection. If not specified, the full set of values specified in
-      hadoop.rpc.protection is used while determining the QOP used for the
-      connection. If a class is specified, then the QOP values returned by
-      the class will be used while determining the QOP used for the connection.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.sensitive-config-keys</name>
-  <value>secret$,password$,ssl.keystore.pass$,fs.s3.*[Ss]ecret.?[Kk]ey,fs.azure.account.key.*,dfs.webhdfs.oauth2.[a-z]+.token,hadoop.security.sensitive-config-keys</value>
-  <description>A comma-separated list of regular expressions to match against
-      configuration keys that should be redacted where appropriate, for
-      example, when logging modified properties during a reconfiguration,
-      private credentials should not be logged.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.workaround.non.threadsafe.getpwuid</name>
-  <value>true</value>
-  <description>Some operating systems or authentication modules are known to
-  have broken implementations of getpwuid_r and getpwgid_r, such that these
-  calls are not thread-safe. Symptoms of this problem include JVM crashes
-  with a stack trace inside these functions. If your system exhibits this
-  issue, enable this configuration parameter to include a lock around the
-  calls as a workaround.
-
-  An incomplete list of some systems known to have this issue is available
-  at http://wiki.apache.org/hadoop/KnownBrokenPwuidImplementations
-  </description>
-</property>
-
-<property>
-  <name>hadoop.kerberos.kinit.command</name>
-  <value>kinit</value>
-  <description>Used to periodically renew Kerberos credentials when provided
-  to Hadoop. The default setting assumes that kinit is in the PATH of users
-  running the Hadoop client. Change this to the absolute path to kinit if this
-  is not the case.
-  </description>
-</property>
-
-<property>
-    <name>hadoop.kerberos.min.seconds.before.relogin</name>
-    <value>60</value>
-    <description>The minimum time between relogin attempts for Kerberos, in
-    seconds.
-    </description>
-</property>
-
-<property>
-  <name>hadoop.security.auth_to_local</name>
-  <value></value>
-  <description>Maps kerberos principals to local user names</description>
-</property>
-
-<!-- i/o properties -->
-<property>
-  <name>io.file.buffer.size</name>
-  <value>4096</value>
-  <description>The size of buffer for use in sequence files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>io.bytes.per.checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  io.file.buffer.size.</description>
-</property>
-
-<property>
-  <name>io.skip.checksum.errors</name>
-  <value>false</value>
-  <description>If true, when a checksum error is encountered while
-  reading a sequence file, entries are skipped, instead of throwing an
-  exception.</description>
-</property>
-
-<property>
-  <name>io.compression.codecs</name>
-  <value></value>
-  <description>A comma-separated list of the compression codec classes that can
-  be used for compression/decompression. In addition to any classes specified
-  with this property (which take precedence), codec classes on the classpath
-  are discovered using a Java ServiceLoader.</description>
-</property>
-
-<property>
-  <name>io.compression.codec.bzip2.library</name>
-  <value>system-native</value>
-  <description>The native-code library to be used for compression and
-  decompression by the bzip2 codec.  This library could be specified
-  either by by name or the full pathname.  In the former case, the
-  library is located by the dynamic linker, usually searching the
-  directories specified in the environment variable LD_LIBRARY_PATH.
-
-  The value of "system-native" indicates that the default system
-  library should be used.  To indicate that the algorithm should
-  operate entirely in Java, specify "java-builtin".</description>
-</property>
-
-<property>
-  <name>io.serializations</name>
-  <value>org.apache.hadoop.io.serializer.WritableSerialization, org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization, org.apache.hadoop.io.serializer.avro.AvroReflectSerialization</value>
-  <description>A list of serialization classes that can be used for
-  obtaining serializers and deserializers.</description>
-</property>
-
-<property>
-  <name>io.seqfile.local.dir</name>
-  <value>${hadoop.tmp.dir}/io/local</value>
-  <description>The local directory where sequence file stores intermediate
-  data files during merge.  May be a comma-separated list of
-  directories on different devices in order to spread disk i/o.
-  Directories that do not exist are ignored.
-  </description>
-</property>
-
-<property>
-  <name>io.map.index.skip</name>
-  <value>0</value>
-  <description>Number of index entries to skip between each entry.
-  Zero by default. Setting this to values larger than zero can
-  facilitate opening large MapFiles using less memory.</description>
-</property>
-
-<property>
-  <name>io.map.index.interval</name>
-  <value>128</value>
-  <description>
-    MapFile consist of two files - data file (tuples) and index file
-    (keys). For every io.map.index.interval records written in the
-    data file, an entry (record-key, data-file-position) is written
-    in the index file. This is to allow for doing binary search later
-    within the index file to look up records by their keys and get their
-    closest positions in the data file.
-  </description>
-</property>
-
-<!-- file system properties -->
-
-<property>
-  <name>fs.defaultFS</name>
-  <value>file:///</value>
-  <description>The name of the default file system.  A URI whose
-  scheme and authority determine the FileSystem implementation.  The
-  uri's scheme determines the config property (fs.SCHEME.impl) naming
-  the FileSystem implementation class.  The uri's authority is used to
-  determine the host, port, etc. for a filesystem.</description>
-</property>
-
-<property>
-  <name>fs.default.name</name>
-  <value>file:///</value>
-  <description>Deprecated. Use (fs.defaultFS) property
-  instead</description>
-</property>
-
-<property>
-  <name>fs.trash.interval</name>
-  <value>0</value>
-  <description>Number of minutes after which the checkpoint
-  gets deleted.  If zero, the trash feature is disabled.
-  This option may be configured both on the server and the
-  client. If trash is disabled server side then the client
-  side configuration is checked. If trash is enabled on the
-  server side then the value configured on the server is
-  used and the client configuration value is ignored.
-  </description>
-</property>
-
-<property>
-  <name>fs.trash.checkpoint.interval</name>
-  <value>0</value>
-  <description>Number of minutes between trash checkpoints.
-  Should be smaller or equal to fs.trash.interval. If zero,
-  the value is set to the value of fs.trash.interval.
-  Every time the checkpointer runs it creates a new checkpoint
-  out of current and removes checkpoints created more than
-  fs.trash.interval minutes ago.
-  </description>
-</property>
-
-<property>
-  <name>fs.protected.directories</name>
-  <value></value>
-  <description>A comma-separated list of directories which cannot
-    be deleted even by the superuser unless they are empty. This
-    setting can be used to guard important system directories
-    against accidental deletion due to administrator error.
-  </description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.file.impl</name>
-  <value>org.apache.hadoop.fs.local.LocalFs</value>
-  <description>The AbstractFileSystem for file: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.har.impl</name>
-  <value>org.apache.hadoop.fs.HarFs</value>
-  <description>The AbstractFileSystem for har: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.hdfs.impl</name>
-  <value>org.apache.hadoop.fs.Hdfs</value>
-  <description>The FileSystem for hdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.viewfs.impl</name>
-  <value>org.apache.hadoop.fs.viewfs.ViewFs</value>
-  <description>The AbstractFileSystem for view file system for viewfs: uris
-  (ie client side mount table:).</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.ftp.impl</name>
-  <value>org.apache.hadoop.fs.ftp.FtpFs</value>
-  <description>The FileSystem for Ftp: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.webhdfs.impl</name>
-  <value>org.apache.hadoop.fs.WebHdfs</value>
-  <description>The FileSystem for webhdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.swebhdfs.impl</name>
-  <value>org.apache.hadoop.fs.SWebHdfs</value>
-  <description>The FileSystem for swebhdfs: uris.</description>
-</property>
-
-<property>
-  <name>fs.ftp.host</name>
-  <value>0.0.0.0</value>
-  <description>FTP filesystem connects to this server</description>
-</property>
-
-<property>
-  <name>fs.ftp.host.port</name>
-  <value>21</value>
-  <description>
-    FTP filesystem connects to fs.ftp.host on this port
-  </description>
-</property>
-
-<property>
-  <name>fs.df.interval</name>
-  <value>60000</value>
-  <description>Disk usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.du.interval</name>
-  <value>600000</value>
-  <description>File space usage statistics refresh interval in msec.</description>
-</property>
-
-<property>
-  <name>fs.s3.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when writing files to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3</value>
-  <description>Determines where on the local filesystem the s3:/s3n: filesystem
-  should store files before sending them to S3
-  (or after retrieving them from S3).
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.maxRetries</name>
-  <value>4</value>
-  <description>The maximum number of retries for reading or writing files to S3,
-  before we signal failure to the application.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3.sleepTimeSeconds</name>
-  <value>10</value>
-  <description>The number of seconds to sleep between each S3 retry.
-  </description>
-</property>
-
-<property>
-  <name>fs.automatic.close</name>
-  <value>true</value>
-  <description>By default, FileSystem instances are automatically closed at program
-  exit using a JVM shutdown hook. Setting this property to false disables this
-  behavior. This is an advanced option that should only be used by server applications
-  requiring a more carefully orchestrated shutdown sequence.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.block.size</name>
-  <value>67108864</value>
-  <description>Block size to use when reading files using the native S3
-  filesystem (s3n: URIs).</description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.enabled</name>
-  <value>false</value>
-  <description>Setting this property to true enables multiple uploads to
-  native S3 filesystem. When uploading a file, it is split into blocks
-  if the size is larger than fs.s3n.multipart.uploads.block.size.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.uploads.block.size</name>
-  <value>67108864</value>
-  <description>The block size for multipart uploads to native S3 filesystem.
-  Default size is 64MB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.multipart.copy.block.size</name>
-  <value>5368709120</value>
-  <description>The block size for multipart copy in native S3 filesystem.
-  Default size is 5GB.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3n.server-side-encryption-algorithm</name>
-  <value></value>
-  <description>Specify a server-side encryption algorithm for S3.
-  Unset by default, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.access.key</name>
-  <description>AWS access key ID used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.secret.key</name>
-  <description>AWS secret key used by S3A file system. Omit for IAM role-based or provider-based authentication.</description>
-</property>
-
-<property>
-  <name>fs.s3a.aws.credentials.provider</name>
-  <description>
-	  Comma-separated class names of credential provider classes which implement
-	  com.amazonaws.auth.AWSCredentialsProvider.
-
-	  These are loaded and queried in sequence for a valid set of credentials.
-	  Each listed class must implement one of the following means of
-	  construction, which are attempted in order:
-	  1. a public constructor accepting java.net.URI and
-	  org.apache.hadoop.conf.Configuration,
-	  2. a public static method named getInstance that accepts no
-	  arguments and returns an instance of
-	  com.amazonaws.auth.AWSCredentialsProvider, or
-	  3. a public default constructor.
-
-	  Specifying
-	  org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider
-	  allows
-	  anonymous access to a publicly accessible S3 bucket without any credentials.
-	  Please note that allowing anonymous access to an S3 bucket compromises
-	  security and therefore is unsuitable for most use cases. It can be useful
-	  for accessing public data sets without requiring AWS credentials.
-
-	  If unspecified, then the default list of credential provider classes,
-	  queried in sequence, is:
-	  1. org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider:
-	  supports static
-	  configuration of AWS access key ID and secret access key. See also
-	  fs.s3a.access.key and fs.s3a.secret.key.
-	  2. com.amazonaws.auth.EnvironmentVariableCredentialsProvider: supports
-	  configuration of AWS access key ID and secret access key in
-	  environment variables named AWS_ACCESS_KEY_ID and
-	  AWS_SECRET_ACCESS_KEY, as documented in the AWS SDK.
-	  3.
-	  org.apache.hadoop.fs.s3a.SharedInstanceProfileCredentialsProvider:
-	  a shared instance of
-	  com.amazonaws.auth.InstanceProfileCredentialsProvider from the AWS
-	  SDK, which supports use of instance profile credentials if running
-	  in an EC2 VM. Using this shared instance potentially reduces load
-	  on the EC2 instance metadata service for multi-threaded
-	  applications.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.session.token</name>
-  <description>Session token, when using org.apache.hadoop.fs.s3a.TemporaryAWSCredentialsProvider
-    as one of the providers.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.security.credential.provider.path</name>
-  <value />
-  <description>
-    Optional comma separated list of credential providers, a list
-    which is prepended to that set in hadoop.security.credential.provider.path
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.maximum</name>
-  <value>15</value>
-  <description>Controls the maximum number of simultaneous connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.ssl.enabled</name>
-  <value>true</value>
-  <description>Enables or disables SSL connections to S3.</description>
-</property>
-
-<property>
-  <name>fs.s3a.endpoint</name>
-  <description>AWS S3 endpoint to connect to. An up-to-date list is
-    provided in the AWS Documentation: regions and endpoints. Without this
-    property, the standard region (s3.amazonaws.com) is assumed.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.path.style.access</name>
-  <value>false</value>
-  <description>Enable S3 path style access ie disabling the default virtual hosting behaviour.
-    Useful for S3A-compliant storage providers as it removes the need to set up DNS for virtual hosting.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.host</name>
-  <description>Hostname of the (optional) proxy server for S3 connections.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.port</name>
-  <description>Proxy server port. If this property is not set
-    but fs.s3a.proxy.host is, port 80 or 443 is assumed (consistent with
-    the value of fs.s3a.connection.ssl.enabled).</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.username</name>
-  <description>Username for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.password</name>
-  <description>Password for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.domain</name>
-  <description>Domain for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.proxy.workstation</name>
-  <description>Workstation for authenticating with proxy server.</description>
-</property>
-
-<property>
-  <name>fs.s3a.attempts.maximum</name>
-  <value>20</value>
-  <description>How many times we should retry commands on transient errors.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.establish.timeout</name>
-  <value>5000</value>
-  <description>Socket connection setup timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.connection.timeout</name>
-  <value>200000</value>
-  <description>Socket connection timeout in milliseconds.</description>
-</property>
-
-<property>
-  <name>fs.s3a.socket.send.buffer</name>
-  <value>8192</value>
-  <description>Socket send buffer hint to amazon connector. Represented in bytes.</description>
-</property>
-
-<property>
-  <name>fs.s3a.socket.recv.buffer</name>
-  <value>8192</value>
-  <description>Socket receive buffer hint to amazon connector. Represented in bytes.</description>
-</property>
-
-<property>
-  <name>fs.s3a.paging.maximum</name>
-  <value>5000</value>
-  <description>How many keys to request from S3 when doing
-     directory listings at a time.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.max</name>
-  <value>10</value>
-  <description>The total number of threads available in the filesystem for data
-    uploads *or any other queued filesystem operation*.</description>
-</property>
-
-<property>
-  <name>fs.s3a.threads.keepalivetime</name>
-  <value>60</value>
-  <description>Number of seconds a thread can be idle before being
-    terminated.</description>
-</property>
-
-<property>
-  <name>fs.s3a.max.total.tasks</name>
-  <value>5</value>
-  <description>The number of operations which can be queued for execution</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.size</name>
-  <value>100M</value>
-  <description>How big (in bytes) to split upload or copy operations up into.
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.threshold</name>
-  <value>2147483647</value>
-  <description>How big (in bytes) to split upload or copy operations up into.
-    This also controls the partition size in renamed files, as rename() involves
-    copying the source file(s).
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multiobjectdelete.enable</name>
-  <value>true</value>
-  <description>When enabled, multiple single-object delete requests are replaced by
-    a single 'delete multiple objects'-request, reducing the number of requests.
-    Beware: legacy S3-compatible object stores might not support this request.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.acl.default</name>
-  <description>Set a canned ACL for newly created and copied objects. Value may be Private,
-      PublicRead, PublicReadWrite, AuthenticatedRead, LogDeliveryWrite, BucketOwnerRead,
-      or BucketOwnerFullControl.</description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge</name>
-  <value>false</value>
-  <description>True if you want to purge existing multipart uploads that may not have been
-    completed/aborted correctly. The corresponding purge age is defined in
-    fs.s3a.multipart.purge.age.
-    If set, when the filesystem is instantiated then all outstanding uploads
-    older than the purge age will be terminated -across the entire bucket.
-    This will impact multipart uploads by other applications and users. so should
-    be used sparingly, with an age value chosen to stop failed uploads, without
-    breaking ongoing operations.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.multipart.purge.age</name>
-  <value>86400</value>
-  <description>Minimum age in seconds of multipart uploads to purge.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.server-side-encryption-algorithm</name>
-  <description>Specify a server-side encryption algorithm for s3a: file system.
-    Unset by default, and the only other currently allowable value is AES256.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.signing-algorithm</name>
-  <description>Override the default signing algorithm so legacy
-    implementations can still be used</description>
-</property>
-
-<property>
-  <name>fs.s3a.block.size</name>
-  <value>32M</value>
-  <description>Block size to use when reading files using s3a: file system.
-    A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.buffer.dir</name>
-  <value>${hadoop.tmp.dir}/s3a</value>
-  <description>Comma separated list of directories that will be used to buffer file
-    uploads to.</description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload</name>
-  <value>false</value>
-  <description>
-    Use the incremental block-based fast upload mechanism with
-    the buffering mechanism set in fs.s3a.fast.upload.buffer.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload.buffer</name>
-  <value>disk</value>
-  <description>
-    The buffering mechanism to use when using S3A fast upload
-    (fs.s3a.fast.upload=true). Values: disk, array, bytebuffer.
-    This configuration option has no effect if fs.s3a.fast.upload is false.
-
-    "disk" will use the directories listed in fs.s3a.buffer.dir as
-    the location(s) to save data prior to being uploaded.
-
-    "array" uses arrays in the JVM heap
-
-    "bytebuffer" uses off-heap memory within the JVM.
-
-    Both "array" and "bytebuffer" will consume memory in a single stream up to the number
-    of blocks set by:
-
-        fs.s3a.multipart.size * fs.s3a.fast.upload.active.blocks.
-
-    If using either of these mechanisms, keep this value low
-
-    The total number of threads performing work across all threads is set by
-    fs.s3a.threads.max, with fs.s3a.max.total.tasks values setting the number of queued
-    work items.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.fast.upload.active.blocks</name>
-  <value>4</value>
-  <description>
-    Maximum Number of blocks a single output stream can have
-    active (uploading, or queued to the central FileSystem
-    instance's pool of queued operations.
-
-    This stops a single stream overloading the shared thread pool.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.readahead.range</name>
-  <value>64K</value>
-  <description>Bytes to read ahead during a seek() before closing and
-  re-opening the S3 HTTP connection. This option will be overridden if
-  any call to setReadahead() is made to an open stream.
-  A suffix from the set {K,M,G,T,P} may be used to scale the numeric value.
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.user.agent.prefix</name>
-  <value></value>
-  <description>
-    Sets a custom value that will be prepended to the User-Agent header sent in
-    HTTP requests to the S3 back-end by S3AFileSystem.  The User-Agent header
-    always includes the Hadoop version number followed by a string generated by
-    the AWS SDK.  An example is "User-Agent: Hadoop 2.8.0, aws-sdk-java/1.10.6".
-    If this optional property is set, then its value is prepended to create a
-    customized User-Agent.  For example, if this configuration property was set
-    to "MyApp", then an example of the resulting User-Agent would be
-    "User-Agent: MyApp, Hadoop 2.8.0, aws-sdk-java/1.10.6".
-  </description>
-</property>
-
-<property>
-  <name>fs.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3AFileSystem</value>
-  <description>The implementation class of the S3A Filesystem</description>
-</property>
-
-<property>
-  <name>fs.AbstractFileSystem.s3a.impl</name>
-  <value>org.apache.hadoop.fs.s3a.S3A</value>
-  <description>The implementation class of the S3A AbstractFileSystem.</description>
-</property>
-
-<property>
-  <name>io.seqfile.compress.blocksize</name>
-  <value>1000000</value>
-  <description>The minimum block size for compression in block compressed
-          SequenceFiles.
-  </description>
-</property>
-
- <property>
-  <name>io.mapfile.bloom.size</name>
-  <value>1048576</value>
-  <description>The size of BloomFilter-s used in BloomMapFile. Each time this many
-  keys is appended the next BloomFilter will be created (inside a DynamicBloomFilter).
-  Larger values minimize the number of filters, which slightly increases the performance,
-  but may waste too much space if the total number of keys is usually much smaller
-  than this number.
-  </description>
-</property>
-
-<property>
-  <name>io.mapfile.bloom.error.rate</name>
-  <value>0.005</value>
-  <description>The rate of false positives in BloomFilter-s used in BloomMapFile.
-  As this value decreases, the size of BloomFilter-s increases exponentially. This
-  value is the probability of encountering false positives (default is 0.5%).
-  </description>
-</property>
-
-<property>
-  <name>hadoop.util.hash.type</name>
-  <value>murmur</value>
-  <description>The default implementation of Hash. Currently this can take one of the
-  two values: 'murmur' to select MurmurHash and 'jenkins' to select JenkinsHash.
-  </description>
-</property>
-
-
-<!-- ipc properties -->
-
-<property>
-  <name>ipc.client.idlethreshold</name>
-  <value>4000</value>
-  <description>Defines the threshold number of connections after which
-               connections will be inspected for idleness.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.kill.max</name>
-  <value>10</value>
-  <description>Defines the maximum number of clients to disconnect in one go.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connection.maxidletime</name>
-  <value>10000</value>
-  <description>The maximum time in msec after which a client will bring down the
-               connection to the server.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries</name>
-  <value>10</value>
-  <description>Indicates the number of retries a client will make to establish
-               a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.retry.interval</name>
-  <value>1000</value>
-  <description>Indicates the number of milliseconds a client will wait for
-    before retrying to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.timeout</name>
-  <value>20000</value>
-  <description>Indicates the number of milliseconds a client will wait for the
-               socket to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.connect.max.retries.on.timeouts</name>
-  <value>45</value>
-  <description>Indicates the number of retries a client will make on socket timeout
-               to establish a server connection.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.tcpnodelay</name>
-  <value>true</value>
-  <description>Use TCP_NODELAY flag to bypass Nagle's algorithm transmission delays.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.low-latency</name>
-  <value>false</value>
-  <description>Use low-latency QoS markers for IPC connections.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.ping</name>
-  <value>true</value>
-  <description>Send a ping to the server when timeout on reading the response,
-  if set to true. If no failure is detected, the client retries until at least
-  a byte is read or the time given by ipc.client.rpc-timeout.ms is passed.
-  </description>
-</property>
-
-<property>
-  <name>ipc.ping.interval</name>
-  <value>60000</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  The client will send ping when the interval is passed without receiving bytes,
-  if ipc.client.ping is set to true.
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.rpc-timeout.ms</name>
-  <value>0</value>
-  <description>Timeout on waiting response from server, in milliseconds.
-  If ipc.client.ping is set to true and this rpc-timeout is greater than
-  the value of ipc.ping.interval, the effective value of the rpc-timeout is
-  rounded up to multiple of ipc.ping.interval.
-  </description>
-</property>
-
-<property>
-  <name>ipc.server.listen.queue.size</name>
-  <value>128</value>
-  <description>Indicates the length of the listen queue for servers accepting
-               client connections.
-  </description>
-</property>
-
-<property>
-    <name>ipc.server.log.slow.rpc</name>
-    <value>false</value>
-    <description>This setting is useful to troubleshoot performance issues for
-     various services. If this value is set to true then we log requests that
-     fall into 99th percentile as well as increment RpcSlowCalls counter.
-    </description>
-</property>
-
-<property>
-  <name>ipc.maximum.data.length</name>
-  <value>67108864</value>
-  <description>This indicates the maximum IPC message length (bytes) that can be
-    accepted by the server. Messages larger than this value are rejected by the
-    immediately to avoid possible OOMs. This setting should rarely need to be
-    changed.
-  </description>
-</property>
-
-<property>
-  <name>ipc.maximum.response.length</name>
-  <value>134217728</value>
-  <description>This indicates the maximum IPC message length (bytes) that can be
-    accepted by the client. Messages larger than this value are rejected
-    immediately to avoid possible OOMs. This setting should rarely need to be
-    changed.  Set to 0 to disable.
-  </description>
-</property>
-
-<!-- Proxy Configuration -->
-
-<property>
-  <name>hadoop.security.impersonation.provider.class</name>
-  <value></value>
-  <description>A class which implements ImpersonationProvider interface, used to
-       authorize whether one user can impersonate a specific user.
-       If not specified, the DefaultImpersonationProvider will be used.
-       If a class is specified, then that class will be used to determine
-       the impersonation capability.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.default</name>
-  <value>org.apache.hadoop.net.StandardSocketFactory</value>
-  <description> Default SocketFactory to use. This parameter is expected to be
-    formatted as "package.FactoryClassName".
-  </description>
-</property>
-
-<property>
-  <name>hadoop.rpc.socket.factory.class.ClientProtocol</name>
-  <value></value>
-  <description> SocketFactory to use to connect to a DFS. If null or empty, use
-    hadoop.rpc.socket.class.default. This socket factory is also used by
-    DFSClient to create sockets to DataNodes.
-  </description>
-</property>
-
-
-
-<property>
-  <name>hadoop.socks.server</name>
-  <value></value>
-  <description> Address (host:port) of the SOCKS server to be used by the
-    SocksSocketFactory.
-  </description>
-</property>
-
-<!-- Topology Configuration -->
-<property>
-  <name>net.topology.node.switch.mapping.impl</name>
-  <value>org.apache.hadoop.net.ScriptBasedMapping</value>
-  <description> The default implementation of the DNSToSwitchMapping. It
-    invokes a script specified in net.topology.script.file.name to resolve
-    node names. If the value for net.topology.script.file.name is not set, the
-    default value of DEFAULT_RACK is returned for all node names.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.impl</name>
-  <value>org.apache.hadoop.net.NetworkTopology</value>
-  <description> The default implementation of NetworkTopology which is classic three layer one.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.file.name</name>
-  <value></value>
-  <description> The script name that should be invoked to resolve DNS names to
-    NetworkTopology names. Example: the script would take host.foo.bar as an
-    argument, and return /rack1 as the output.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.script.number.args</name>
-  <value>100</value>
-  <description> The max number of args that the script configured with
-    net.topology.script.file.name should be run with. Each arg is an
-    IP address.
-  </description>
-</property>
-
-<property>
-  <name>net.topology.table.file.name</name>
-  <value></value>
-  <description> The file name for a topology file, which is used when the
-    net.topology.node.switch.mapping.impl property is set to
-    org.apache.hadoop.net.TableMapping. The file format is a two column text
-    file, with columns separated by whitespace. The first column is a DNS or
-    IP address and the second column specifies the rack where the address maps.
-    If no entry corresponding to a host in the cluster is found, then
-    /default-rack is assumed.
-  </description>
-</property>
-
-<!-- Local file system -->
-<property>
-  <name>file.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>file.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  file.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>file.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>file.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>file.replication</name>
-  <value>1</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3 File System -->
-
-<property>
-  <name>s3.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- s3native File System -->
-
-<property>
-  <name>s3native.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>s3native.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  s3native.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>s3native.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>s3native.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>s3native.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- FTP file system -->
-<property>
-  <name>ftp.stream-buffer-size</name>
-  <value>4096</value>
-  <description>The size of buffer to stream files.
-  The size of this buffer should probably be a multiple of hardware
-  page size (4096 on Intel x86), and it determines how much data is
-  buffered during read and write operations.</description>
-</property>
-
-<property>
-  <name>ftp.bytes-per-checksum</name>
-  <value>512</value>
-  <description>The number of bytes per checksum.  Must not be larger than
-  ftp.stream-buffer-size</description>
-</property>
-
-<property>
-  <name>ftp.client-write-packet-size</name>
-  <value>65536</value>
-  <description>Packet size for clients to write</description>
-</property>
-
-<property>
-  <name>ftp.blocksize</name>
-  <value>67108864</value>
-  <description>Block size</description>
-</property>
-
-<property>
-  <name>ftp.replication</name>
-  <value>3</value>
-  <description>Replication factor</description>
-</property>
-
-<!-- Tfile -->
-
-<property>
-  <name>tfile.io.chunk.size</name>
-  <value>1048576</value>
-  <description>
-    Value chunk size in bytes. Default  to
-    1MB. Values of the length less than the chunk size is
-    guaranteed to have known value length in read time (See also
-    TFile.Reader.Scanner.Entry.isValueLengthKnown()).
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.output.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataOutputStream in bytes.
-  </description>
-</property>
-
-<property>
-  <name>tfile.fs.input.buffer.size</name>
-  <value>262144</value>
-  <description>
-    Buffer size used for FSDataInputStream in bytes.
-  </description>
-</property>
-
-<!-- HTTP web-consoles Authentication -->
-
-<property>
-  <name>hadoop.http.authentication.type</name>
-  <value>simple</value>
-  <description>
-    Defines authentication used for Oozie HTTP endpoint.
-    Supported values are: simple | kerberos | #AUTHENTICATION_HANDLER_CLASSNAME#
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.token.validity</name>
-  <value>36000</value>
-  <description>
-    Indicates how long (in seconds) an authentication token is valid before it has
-    to be renewed.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.signature.secret.file</name>
-  <value>${user.home}/hadoop-http-auth-signature-secret</value>
-  <description>
-    The signature secret for signing the authentication tokens.
-    The same secret should be used for JT/NN/DN/TT configurations.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.cookie.domain</name>
-  <value></value>
-  <description>
-    The domain to use for the HTTP cookie that stores the authentication token.
-    In order to authentiation to work correctly across all Hadoop nodes web-consoles
-    the domain must be correctly set.
-    IMPORTANT: when using IP addresses, browsers ignore cookies with domain settings.
-    For this setting to work properly all nodes in the cluster must be configured
-    to generate URLs with hostname.domain names on it.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.simple.anonymous.allowed</name>
-  <value>true</value>
-  <description>
-    Indicates if anonymous requests are allowed when using 'simple' authentication.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.principal</name>
-  <value>HTTP/_HOST@LOCALHOST</value>
-  <description>
-    Indicates the Kerberos principal to be used for HTTP endpoint.
-    The principal MUST start with 'HTTP/' as per Kerberos HTTP SPNEGO specification.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.http.authentication.kerberos.keytab</name>
-  <value>${user.home}/hadoop.keytab</value>
-  <description>
-    Location of the keytab file with the credentials for the principal.
-    Referring to the same keytab file Oozie uses for its Kerberos credentials for Hadoop.
-  </description>
-</property>
-
-<!-- HTTP CORS support -->
-<property>
-  <description>Enable/disable the cross-origin (CORS) filter.</description>
-  <name>hadoop.http.cross-origin.enabled</name>
-  <value>false</value>
-</property>
-
-<property>
-  <description>Comma separated list of origins that are allowed for web
-    services needing cross-origin (CORS) support. Wildcards (*) and patterns
-    allowed</description>
-  <name>hadoop.http.cross-origin.allowed-origins</name>
-  <value>*</value>
-</property>
-
-<property>
-  <description>Comma separated list of methods that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-methods</name>
-  <value>GET,POST,HEAD</value>
-</property>
-
-<property>
-  <description>Comma separated list of headers that are allowed for web
-    services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.allowed-headers</name>
-  <value>X-Requested-With,Content-Type,Accept,Origin</value>
-</property>
-
-<property>
-  <description>The number of seconds a pre-flighted request can be cached
-    for web services needing cross-origin (CORS) support.</description>
-  <name>hadoop.http.cross-origin.max-age</name>
-  <value>1800</value>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.methods</name>
-  <value></value>
-  <description>
-    List of fencing methods to use for service fencing. May contain
-    builtin methods (eg shell and sshfence) or user-defined method.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.connect-timeout</name>
-  <value>30000</value>
-  <description>
-    SSH connection timeout, in milliseconds, to use with the builtin
-    sshfence fencer.
-  </description>
-</property>
-
-<property>
-  <name>dfs.ha.fencing.ssh.private-key-files</name>
-  <value></value>
-  <description>
-    The SSH private key files to use with the builtin sshfence fencer.
-  </description>
-</property>
-
-
-<!-- Static Web User Filter properties. -->
-<property>
-  <description>
-    The user name to filter as, on static web filters
-    while rendering content. An example use is the HDFS
-    web UI (user to be used for browsing files).
-  </description>
-  <name>hadoop.http.staticuser.user</name>
-  <value>dr.who</value>
-</property>
-
-<property>
-  <name>ha.zookeeper.quorum</name>
-  <description>
-    A list of ZooKeeper server addresses, separated by commas, that are
-    to be used by the ZKFailoverController in automatic failover.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.session-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    The session timeout to use when the ZKFC connects to ZooKeeper.
-    Setting this value to a lower value implies that server crashes
-    will be detected more quickly, but risks triggering failover too
-    aggressively in the case of a transient error or network blip.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.parent-znode</name>
-  <value>/hadoop-ha</value>
-  <description>
-    The ZooKeeper znode under which the ZK failover controller stores
-    its information. Note that the nameservice ID is automatically
-    appended to this znode, so it is not normally necessary to
-    configure this, even in a federated environment.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.acl</name>
-  <value>world:anyone:rwcda</value>
-  <description>
-    A comma-separated list of ZooKeeper ACLs to apply to the znodes
-    used by automatic failover. These ACLs are specified in the same
-    format as used by the ZooKeeper CLI.
-
-    If the ACL itself contains secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<property>
-  <name>ha.zookeeper.auth</name>
-  <value></value>
-  <description>
-    A comma-separated list of ZooKeeper authentications to add when
-    connecting to ZooKeeper. These are specified in the same format
-    as used by the &quot;addauth&quot; command in the ZK CLI. It is
-    important that the authentications specified here are sufficient
-    to access znodes with the ACL specified in ha.zookeeper.acl.
-
-    If the auths contain secrets, you may instead specify a
-    path to a file, prefixed with the '@' symbol, and the value of
-    this configuration will be loaded from within.
-  </description>
-</property>
-
-<!-- SSLFactory configuration -->
-
-<property>
-  <name>hadoop.ssl.keystores.factory.class</name>
-  <value>org.apache.hadoop.security.ssl.FileBasedKeyStoresFactory</value>
-  <description>
-    The keystores factory to use for retrieving certificates.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.require.client.cert</name>
-  <value>false</value>
-  <description>Whether client certificates are required</description>
-</property>
-
-<property>
-  <name>hadoop.ssl.hostname.verifier</name>
-  <value>DEFAULT</value>
-  <description>
-    The hostname verifier to provide for HttpsURLConnections.
-    Valid values are: DEFAULT, STRICT, STRICT_I6, DEFAULT_AND_LOCALHOST and
-    ALLOW_ALL
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.server.conf</name>
-  <value>ssl-server.xml</value>
-  <description>
-    Resource file from which ssl server keystore information will be extracted.
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.client.conf</name>
-  <value>ssl-client.xml</value>
-  <description>
-    Resource file from which ssl client keystore information will be extracted
-    This file is looked up in the classpath, typically it should be in Hadoop
-    conf/ directory.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled</name>
-  <value>false</value>
-  <description>
-    Deprecated. Use dfs.http.policy and yarn.http.policy instead.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.ssl.enabled.protocols</name>
-  <value>TLSv1</value>
-  <description>
-    Protocols supported by the ssl.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.jetty.logs.serve.aliases</name>
-  <value>true</value>
-  <description>
-    Enable/Disable aliases serving from jetty
-  </description>
-</property>
-
-<property>
-  <name>fs.permissions.umask-mode</name>
-  <value>022</value>
-  <description>
-    The umask used when creating files and directories.
-    Can be in octal or in symbolic. Examples are:
-    "022" (octal for u=rwx,g=r-x,o=r-x in symbolic),
-    or "u=rwx,g=rwx,o=" (symbolic for 007 in octal).
-  </description>
-</property>
-
-<!-- ha properties -->
-
-<property>
-  <name>ha.health-monitor.connect-retry-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to retry connecting to the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.check-interval.ms</name>
-  <value>1000</value>
-  <description>
-    How often to check the service.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.sleep-after-disconnect.ms</name>
-  <value>1000</value>
-  <description>
-    How long to sleep after an unexpected RPC error.
-  </description>
-</property>
-
-<property>
-  <name>ha.health-monitor.rpc-timeout.ms</name>
-  <value>45000</value>
-  <description>
-    Timeout for the actual monitorHealth() calls.
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.new-active.rpc-timeout.ms</name>
-  <value>60000</value>
-  <description>
-    Timeout that the FC waits for the new active to become active
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.rpc-timeout.ms</name>
-  <value>5000</value>
-  <description>
-    Timeout that the FC waits for the old active to go to standby
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.graceful-fence.connection.retries</name>
-  <value>1</value>
-  <description>
-    FC connection retries for graceful fencing
-  </description>
-</property>
-
-<property>
-  <name>ha.failover-controller.cli-check.rpc-timeout.ms</name>
-  <value>20000</value>
-  <description>
-    Timeout that the CLI (manual) FC waits for monitorHealth, getServiceState
-  </description>
-</property>
-
-<property>
-  <name>ipc.client.fallback-to-simple-auth-allowed</name>
-  <value>false</value>
-  <description>
-    When a client is configured to attempt a secure connection, but attempts to
-    connect to an insecure server, that server may instruct the client to
-    switch to SASL SIMPLE (unsecure) authentication. This setting controls
-    whether or not the client will accept this instruction from the server.
-    When false (the default), the client will not allow the fallback to SIMPLE
-    authentication, and will abort the connection.
-  </description>
-</property>
-
-<property>
-  <name>fs.client.resolve.remote.symlinks</name>
-  <value>true</value>
-  <description>
-      Whether to resolve symlinks when accessing a remote Hadoop filesystem.
-      Setting this to false causes an exception to be thrown upon encountering
-      a symlink. This setting does not apply to local filesystems, which
-      automatically resolve local symlinks.
-  </description>
-</property>
-
-<property>
-  <name>nfs.exports.allowed.hosts</name>
-  <value>* rw</value>
-  <description>
-    By default, the export can be mounted by any client. The value string
-    contains machine name and access privilege, separated by whitespace
-    characters. The machine name format can be a single host, a Java regular
-    expression, or an IPv4 address. The access privilege uses rw or ro to
-    specify read/write or read-only access of the machines to exports. If the
-    access privilege is not provided, the default is read-only. Entries are separated by ";".
-    For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;".
-    Only the NFS gateway needs to restart after this property is updated.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.user.group.static.mapping.overrides</name>
-  <value>dr.who=;</value>
-  <description>
-    Static mapping of user to groups. This will override the groups if
-    available in the system for the specified user. In otherwords, groups
-    look-up will not happen for these users, instead groups mapped in this
-    configuration will be used.
-    Mapping should be in this format.
-    user1=group1,group2;user2=;user3=group2;
-    Default, "dr.who=;" will consider "dr.who" as user without groups.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.quantile.enable</name>
-  <value>false</value>
-  <description>
-    Setting this property to true and rpc.metrics.percentiles.intervals
-    to a comma-separated list of the granularity in seconds, the
-    50/75/90/95/99th percentile latency for rpc queue/processing time in
-    milliseconds are added to rpc metrics.
-  </description>
-</property>
-
-<property>
-  <name>rpc.metrics.percentiles.intervals</name>
-  <value></value>
-  <description>
-    A comma-separated list of the granularity in seconds for the metrics which
-    describe the 50/75/90/95/99th percentile latency for rpc queue/processing
-    time. The metrics are outputted if rpc.metrics.quantile.enable is set to
-    true.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE</name>
-  <value></value>
-  <description>
-    The prefix for a given crypto codec, contains a comma-separated
-    list of implementation classes for a given crypto codec (eg EXAMPLECIPHERSUITE).
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.codec.classes.aes.ctr.nopadding</name>
-  <value>org.apache.hadoop.crypto.OpensslAesCtrCryptoCodec, org.apache.hadoop.crypto.JceAesCtrCryptoCodec</value>
-  <description>
-    Comma-separated list of crypto codec implementations for AES/CTR/NoPadding.
-    The first implementation will be used if available, others are fallbacks.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.cipher.suite</name>
-  <value>AES/CTR/NoPadding</value>
-  <description>
-    Cipher suite for crypto codec.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.jce.provider</name>
-  <value></value>
-  <description>
-    The JCE provider name used in CryptoCodec.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.crypto.buffer.size</name>
-  <value>8192</value>
-  <description>
-    The buffer size used by CryptoInputStream and CryptoOutputStream.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.java.secure.random.algorithm</name>
-  <value>SHA1PRNG</value>
-  <description>
-    The java secure random algorithm.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.secure.random.impl</name>
-  <value></value>
-  <description>
-    Implementation of secure random.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.random.device.file.path</name>
-  <value>/dev/urandom</value>
-  <description>
-    OS security random device file path.
-  </description>
-</property>
-
-<property>
-  <name>hadoop.security.key.provider.path</name>
-  <description>
-    The KeyProvider to use when managing zone keys, and interacting with
-    encryption keys when reading and writing to an encryption zone.
-  </description>
-</property>
-
-<property>
-  <name>fs.har.impl.disable.cache</name>
-  <value>true</value>
-  <description>Don't cache 'har' filesystem instances.</description>
-</property>
-
-<!--- KMSClientProvider configurations -->
-<property>
-  <name>hadoop.security.kms.client.authentication.retry-count</name>
-  <value>1</value>
-  <description>
-    Number of time to retry connecting to KMS on authentication failure
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.size</name>
-  <value>500</value>
-  <description>
-    Size of the EncryptedKeyVersion cache Queue for each key
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.low-watermark</name>
-  <value>0.3f</value>
-  <description>
-    If size of the EncryptedKeyVersion cache Queue falls below the
-    low watermark, this cache queue will be scheduled for a refill
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.num.refill.threads</name>
-  <value>2</value>
-  <description>
-    Number of threads to use for refilling depleted EncryptedKeyVersion
-    cache Queues
-  </description>
-</property>
-<property>
-  <name>hadoop.security.kms.client.encrypted.key.cache.expiry</name>
-  <value>43200000</value>
-  <description>
-    Cache expiry time for a Key, after which the cache Queue for this
-    key will be dropped. Default = 12hrs
-  </description>
-</property>
-
- <property>
-  <name>ipc.server.max.connections</name>
-  <value>0</value>
-  <description>The maximum number of concurrent connections a server is allowed
-    to accept. If this limit is exceeded, incoming connections will first fill
-    the listen queue and then may go to an OS-specific listen overflow queue.
-    The client may fail or timeout, but the server can avoid running out of file
-    descriptors using this feature. 0 means no limit.
-  </description>
-</property>
-
-
-  <!-- YARN registry -->
-
-  <property>
-    <description>
-      Is the registry enabled in the YARN Resource Manager?
-
-      If true, the YARN RM will, as needed.
-      create the user and system paths, and purge
-      service records when containers, application attempts
-      and applications complete.
-
-      If false, the paths must be created by other means,
-      and no automatic cleanup of service records will take place.
-    </description>
-    <name>hadoop.registry.rm.enabled</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      The root zookeeper node for the registry
-    </description>
-    <name>hadoop.registry.zk.root</name>
-    <value>/registry</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper session timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.session.timeout.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection timeout in milliseconds
-    </description>
-    <name>hadoop.registry.zk.connection.timeout.ms</name>
-    <value>15000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper connection retry count before failing
-    </description>
-    <name>hadoop.registry.zk.retry.times</name>
-    <value>5</value>
-  </property>
-
-  <property>
-    <description>
-    </description>
-    <name>hadoop.registry.zk.retry.interval.ms</name>
-    <value>1000</value>
-  </property>
-
-  <property>
-    <description>
-      Zookeeper retry limit in milliseconds, during
-      exponential backoff.
-
-      This places a limit even
-      if the retry times and interval limit, combined
-      with the backoff policy, result in a long retry
-      period
-    </description>
-    <name>hadoop.registry.zk.retry.ceiling.ms</name>
-    <value>60000</value>
-  </property>
-
-  <property>
-    <description>
-      List of hostname:port pairs defining the
-      zookeeper quorum binding for the registry
-    </description>
-    <name>hadoop.registry.zk.quorum</name>
-    <value>localhost:2181</value>
-  </property>
-
-  <property>
-    <description>
-      Key to set if the registry is secure. Turning it on
-      changes the permissions policy from "open access"
-      to restrictions on kerberos with the option of
-      a user adding one or more auth key pairs down their
-      own tree.
-    </description>
-    <name>hadoop.registry.secure</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <description>
-      A comma separated list of Zookeeper ACL identifiers with
-      system access to the registry in a secure cluster.
-
-      These are given full access to all entries.
-
-      If there is an "@" at the end of a SASL entry it
-      instructs the registry client to append the default kerberos domain.
-    </description>
-    <name>hadoop.registry.system.acls</name>
-    <value>sasl:yarn@, sasl:mapred@, sasl:hdfs@</value>
-  </property>
-
-  <property>
-    <description>
-      The kerberos realm: used to set the realm of
-      system principals which do not declare their realm,
-      and any other accounts that need the value.
-
-      If empty, the default realm of the running process
-      is used.
-
-      If neither are known and the realm is needed, then the registry
-      service/client will fail.
-    </description>
-    <name>hadoop.registry.kerberos.realm</name>
-    <value></value>
-  </property>
-
-  <property>
-    <description>
-      Key to define the JAAS context. Used in secure
-      mode
-    </description>
-    <name>hadoop.registry.jaas.context</name>
-    <value>Client</value>
-  </property>
-
-  <property>
-    <description>
-      Enable hdfs shell commands to display warnings if (fs.defaultFS) property
-      is not set.
-    </description>
-    <name>hadoop.shell.missing.defaultFs.warning</name>
-    <value>false</value>
-  </property>
-
-  <property>
-    <name>hadoop.shell.safely.delete.limit.num.files</name>
-    <value>100</value>
-    <description>Used by -safely option of hadoop fs shell -rm command to avoid
-      accidental deletion of large directories. When enabled, the -rm command
-      requires confirmation if the number of files to be deleted is greater than
-      this limit.  The default limit is 100 files. The warning is disabled if
-      the limit is 0 or the -safely is not specified in -rm command.
-    </description>
-  </property>
-
-  <property>
-    <name>fs.client.htrace.sampler.classes</name>
-    <value></value>
-    <description>The class names of the HTrace Samplers to use for Hadoop
-      filesystem clients.
-    </description>
-  </property>
-
-  <property>
-    <name>hadoop.htrace.span.receiver.classes</name>
-    <value></value>
-    <description>The class names of the Span Receivers to use for Hadoop.
-    </description>
-  </property>
-
-  <property>
-    <name>hadoop.caller.context.enabled</name>
-    <value>false</value>
-    <description>When the feature is enabled, additional fields are written into
-      name-node audit log records for auditing coarse granularity operations.
-    </description>
-  </property>
-  <property>
-    <name>hadoop.caller.context.max.size</name>
-    <value>128</value>
-    <description>The maximum bytes a caller context string can have. If the
-      passed caller context is longer than this maximum bytes, client will
-      truncate it before sending to server. Note that the server may have a
-      different maximum size, and will truncate the caller context to the
-      maximum size it allows.
-    </description>
-  </property>
-  <property>
-    <name>hadoop.caller.context.signature.max.size</name>
-    <value>40</value>
-    <description>
-      The caller's signature (optional) is for offline validation. If the
-      signature exceeds the maximum allowed bytes in server, the caller context
-      will be abandoned, in which case the caller context will not be recorded
-      in audit logs.
-    </description>
-  </property>
-
+	<property>
+		<name>hadoop.security.group.mapping</name>
+		<value>this.does.not.exist.class</value>
+	</property>
 </configuration>


[flink] 02/06: [hotfix] Add JavaDocs to CheckpointMetadataOutputStream

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit e08f557e4f3d6a8645abfe6ab3c9188b0abbfc79
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Sun Sep 23 22:37:56 2018 +0200

    [hotfix] Add JavaDocs to CheckpointMetadataOutputStream
---
 .../runtime/state/CheckpointMetadataOutputStream.java     | 15 +++++++++++++++
 1 file changed, 15 insertions(+)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointMetadataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointMetadataOutputStream.java
index 4180e88..70aff7e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointMetadataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/CheckpointMetadataOutputStream.java
@@ -22,8 +22,23 @@ import org.apache.flink.core.fs.FSDataOutputStream;
 
 import java.io.IOException;
 
+/**
+ * An output stream for checkpoint metadata.
+ *
+ * <p>This stream is similar to the {@link CheckpointStreamFactory.CheckpointStateOutputStream},
+ * but for metadata files rather thancdata files.
+ *
+ * <p>This stream always creates a file, regardless of the amount of data written.
+ */
 public abstract class CheckpointMetadataOutputStream extends FSDataOutputStream {
 
+	/**
+	 * Closes the stream after all metadata was written and finalizes the checkpoint location.
+	 *
+	 * @return An object representing a finalized checkpoint storage location.
+	 *
+	 * @throws IOException Thrown, if the stream cannot be closed or the finalization fails.
+	 */
 	public abstract CompletedCheckpointStorageLocation closeAndFinalizeCheckpoint() throws IOException;
 
 	/**


[flink] 05/06: [FLINK-9061] [s3] Make S3 Presto file system entropy injecting

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit a2b2041bf3885ee16d0333b4f153a12d435edff4
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Mon Sep 24 22:11:09 2018 +0200

    [FLINK-9061] [s3] Make S3 Presto file system entropy injecting
---
 .../flink/fs/s3presto/S3FileSystemFactory.java     | 103 ++++++++++++++++++---
 .../flink/fs/s3presto/S3PrestoFileSystem.java      |  94 +++++++++++++++++++
 .../flink/fs/s3presto/PrestoS3EntropyTest.java     |  47 ++++++++++
 3 files changed, 231 insertions(+), 13 deletions(-)

diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
index a04f9c9..6cf0267 100644
--- a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
+++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3FileSystemFactory.java
@@ -19,12 +19,20 @@
 package org.apache.flink.fs.s3presto;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.runtime.fs.hdfs.AbstractFileSystemFactory;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystemFactory;
 import org.apache.flink.runtime.fs.hdfs.HadoopConfigLoader;
 import org.apache.flink.util.FlinkRuntimeException;
 
 import com.facebook.presto.hive.PrestoS3FileSystem;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.Collections;
@@ -34,7 +42,35 @@ import java.util.Set;
 /**
  * Simple factory for the S3 file system.
  */
-public class S3FileSystemFactory extends AbstractFileSystemFactory {
+public class S3FileSystemFactory implements FileSystemFactory {
+
+	/**
+	 * The substring to be replaced by random entropy in checkpoint paths.
+	 */
+	public static final ConfigOption<String> ENTROPY_INJECT_KEY_OPTION = ConfigOptions
+			.key("s3.entropy.key")
+			.noDefaultValue()
+			.withDescription(
+					"This option can be used to improve performance due to sharding issues on Amazon S3. " +
+					"For file creations with entropy injection, this key will be replaced by random " +
+					"alphanumeric characters. For other file creations, the key will be filtered out.");
+
+	/**
+	 * The number of entropy characters, in case entropy injection is configured.
+	 */
+	public static final ConfigOption<Integer> ENTROPY_INJECT_LENGTH_OPTION = ConfigOptions
+			.key("s3.entropy.length")
+			.defaultValue(4)
+			.withDescription(
+					"When '" + ENTROPY_INJECT_KEY_OPTION.key() + "' is set, this option defines the number of " +
+					"random characters to replace the entropy key with.");
+
+	// ------------------------------------------------------------------------
+
+	private static final Logger LOG = LoggerFactory.getLogger(S3FileSystemFactory.class);
+
+	private static final String INVALID_ENTROPY_KEY_CHARS = "^.*[~#@*+%{}<>\\[\\]|\"\\\\].*$";
+
 	private static final Set<String> PACKAGE_PREFIXES_TO_SHADE =
 		new HashSet<>(Collections.singletonList("com.amazonaws."));
 
@@ -50,28 +86,69 @@ public class S3FileSystemFactory extends AbstractFileSystemFactory {
 			{ "presto.s3.secret.key", "presto.s3.secret-key" }
 	};
 
-	public S3FileSystemFactory() {
-		super("Presto S3 File System", createHadoopConfigLoader());
-	}
+	// ------------------------------------------------------------------------
+
+	private final HadoopConfigLoader hadoopConfigLoader = createHadoopConfigLoader();
+
+	private Configuration flinkConfig;
 
 	@Override
 	public String getScheme() {
 		return "s3";
 	}
 
+	@Override
+	public void configure(Configuration config) {
+		flinkConfig = config;
+		hadoopConfigLoader.setFlinkConfig(config);
+	}
+
+	@Override
+	public FileSystem create(URI fsUri) throws IOException {
+		Configuration flinkConfig = this.flinkConfig;
+
+		if (flinkConfig == null) {
+			LOG.warn("Creating S3 Presto FileSystem without configuring the factory. All behavior will be default.");
+			flinkConfig = new Configuration();
+		}
+
+		LOG.debug("Creating S3 file system backed by PrestoS3FileSystem");
+
+		try {
+			org.apache.hadoop.conf.Configuration hadoopConfig = hadoopConfigLoader.getOrLoadHadoopConfig();
+			PrestoS3FileSystem fs = new PrestoS3FileSystem();
+			fs.initialize(createInitUri(fsUri), hadoopConfig);
+
+			// load the entropy injection settings
+			String entropyInjectionKey = flinkConfig.getString(ENTROPY_INJECT_KEY_OPTION);
+			int numEntropyChars = -1;
+			if (entropyInjectionKey != null) {
+				if (entropyInjectionKey.matches(INVALID_ENTROPY_KEY_CHARS)) {
+					throw new IllegalConfigurationException("Invalid character in value for " +
+							ENTROPY_INJECT_KEY_OPTION.key() + " : " + entropyInjectionKey);
+				}
+				numEntropyChars = flinkConfig.getInteger(ENTROPY_INJECT_LENGTH_OPTION);
+				if (numEntropyChars <= 0) {
+					throw new IllegalConfigurationException(
+							ENTROPY_INJECT_LENGTH_OPTION.key() + " must configure a value > 0");
+				}
+			}
+
+			return new S3PrestoFileSystem(fs, entropyInjectionKey, numEntropyChars);
+		} catch (IOException ioe) {
+			throw ioe;
+		} catch (Exception e) {
+			throw new IOException(e.getMessage(), e);
+		}
+	}
+
 	@VisibleForTesting
 	static HadoopConfigLoader createHadoopConfigLoader() {
 		return new HadoopConfigLoader(FLINK_CONFIG_PREFIXES, MIRRORED_CONFIG_KEYS,
 			"presto.s3.", PACKAGE_PREFIXES_TO_SHADE, CONFIG_KEYS_TO_SHADE, FLINK_SHADING_PREFIX);
 	}
 
-	@Override
-	protected org.apache.hadoop.fs.FileSystem createHadoopFileSystem() {
-		return new PrestoS3FileSystem();
-	}
-
-	@Override
-	protected URI getInitURI(URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig) {
+	private static URI createInitUri(URI fsUri) {
 		final String scheme = fsUri.getScheme();
 		final String authority = fsUri.getAuthority();
 		final URI initUri;
@@ -88,7 +165,7 @@ public class S3FileSystemFactory extends AbstractFileSystemFactory {
 		return initUri;
 	}
 
-	private URI createURI(String str) {
+	private static URI createURI(String str) {
 		try {
 			return new URI(str);
 		} catch (URISyntaxException e) {
diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java
new file mode 100644
index 0000000..b901063
--- /dev/null
+++ b/flink-filesystems/flink-s3-fs-presto/src/main/java/org/apache/flink/fs/s3presto/S3PrestoFileSystem.java
@@ -0,0 +1,94 @@
+/*
+ * 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.fs.s3presto;
+
+import org.apache.flink.core.fs.EntropyInjectingFileSystem;
+import org.apache.flink.core.fs.FileSystemKind;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+import org.apache.flink.util.StringUtils;
+
+import com.facebook.presto.hive.PrestoS3FileSystem;
+
+import javax.annotation.Nullable;
+
+import java.util.concurrent.ThreadLocalRandom;
+
+/**
+ * Implementation of the Flink {@link org.apache.flink.core.fs.FileSystem} interface for S3.
+ * This class implements the common behavior implemented directly by Flink and delegates
+ * common calls to an implementation of Hadoop's filesystem abstraction.
+ */
+public class S3PrestoFileSystem extends HadoopFileSystem implements EntropyInjectingFileSystem {
+
+	@Nullable
+	private final String entropyInjectionKey;
+
+	private final int entropyLength;
+
+	/**
+	 * Creates a S3PrestoFileSystem based on the given Presto S3 file system.
+	 * The given Hadoop file system object is expected to be initialized already.
+	 */
+	public S3PrestoFileSystem(PrestoS3FileSystem prestoFs) {
+		this(prestoFs, null, -1);
+	}
+
+	/**
+	 * Creates a FlinkS3FileSystem based on the given Hadoop S3 file system.
+	 * The given Hadoop file system object is expected to be initialized already.
+	 *
+	 * <p>This constructor additionally configures the entropy injection for the file system.
+	 *
+	 * @param prestoFs The Presto S3 File System that will be used under the hood.
+	 * @param entropyInjectionKey The substring that will be replaced by entropy or removed.
+	 * @param entropyLength The number of random alphanumeric characters to inject as entropy.
+	 */
+	public S3PrestoFileSystem(
+			PrestoS3FileSystem prestoFs,
+			@Nullable String entropyInjectionKey,
+			int entropyLength) {
+
+		super(prestoFs);
+
+		if (entropyInjectionKey != null && entropyLength <= 0) {
+			throw new IllegalArgumentException("Entropy length must be >= 0 when entropy injection key is set");
+		}
+
+		this.entropyInjectionKey = entropyInjectionKey;
+		this.entropyLength = entropyLength;
+	}
+
+	// ------------------------------------------------------------------------
+
+	@Nullable
+	@Override
+	public String getEntropyInjectionKey() {
+		return entropyInjectionKey;
+	}
+
+	@Override
+	public String generateEntropy() {
+		return StringUtils.generateRandomAlphanumericString(ThreadLocalRandom.current(), entropyLength);
+	}
+
+	@Override
+	public FileSystemKind getKind() {
+		return FileSystemKind.OBJECT_STORE;
+	}
+}
diff --git a/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3EntropyTest.java b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3EntropyTest.java
new file mode 100644
index 0000000..02a0bd5
--- /dev/null
+++ b/flink-filesystems/flink-s3-fs-presto/src/test/java/org/apache/flink/fs/s3presto/PrestoS3EntropyTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.fs.s3presto;
+
+import org.apache.flink.configuration.Configuration;
+
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests that the file system factory picks up the entropy configuration properly.
+ */
+public class PrestoS3EntropyTest {
+
+	@Test
+	public void testEntropyInjectionConfig() throws Exception {
+		final Configuration conf = new Configuration();
+		conf.setString("s3.entropy.key", "__entropy__");
+		conf.setInteger("s3.entropy.length", 7);
+
+		S3FileSystemFactory factory = new S3FileSystemFactory();
+		factory.configure(conf);
+
+		S3PrestoFileSystem fs = (S3PrestoFileSystem) factory.create(new URI("s3://test"));
+		assertEquals("__entropy__", fs.getEntropyInjectionKey());
+		assertEquals(7, fs.generateEntropy().length());
+	}
+}


[flink] 04/06: [FLINK-9061] [checkpoints] FsStatebackend optionally injects entropy into state data file paths

Posted by se...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

sewen pushed a commit to branch release-1.6
in repository https://gitbox.apache.org/repos/asf/flink.git

commit badb72d6a5b3c1862a70caaa6ce5715cf4c0efee
Author: Stephan Ewen <se...@apache.org>
AuthorDate: Mon Sep 24 20:45:48 2018 +0200

    [FLINK-9061] [checkpoints] FsStatebackend optionally injects entropy into state data file paths
---
 .../FsCheckpointMetadataOutputStream.java          |   3 +-
 .../state/filesystem/FsCheckpointStorage.java      |  17 ++-
 .../filesystem/FsCheckpointStorageLocation.java    |   6 +-
 .../filesystem/FsCheckpointStreamFactory.java      |  13 ++-
 .../filesystem/FsStateBackendEntropyTest.java      | 123 +++++++++++++++++++++
 5 files changed, 153 insertions(+), 9 deletions(-)

diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java
index f18d578..95f78b8 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointMetadataOutputStream.java
@@ -133,7 +133,8 @@ public final class FsCheckpointMetadataOutputStream extends CheckpointMetadataOu
 					FileStateHandle metaDataHandle = new FileStateHandle(metadataFilePath, size);
 
 					return new FsCompletedCheckpointStorageLocation(
-							fileSystem, exclusiveCheckpointDir, metaDataHandle, exclusiveCheckpointDir.toString());
+							fileSystem, exclusiveCheckpointDir, metaDataHandle,
+							metaDataHandle.getFilePath().getParent().toString());
 				}
 				catch (Exception e) {
 					try {
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
index af80af7..1549e01 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorage.java
@@ -32,6 +32,7 @@ import javax.annotation.Nullable;
 import java.io.IOException;
 
 import static org.apache.flink.util.Preconditions.checkArgument;
+import static org.apache.flink.util.Preconditions.checkNotNull;
 
 /**
  * An implementation of durable checkpoint storage to file systems.
@@ -54,11 +55,25 @@ public class FsCheckpointStorage extends AbstractFsCheckpointStorage {
 			JobID jobId,
 			int fileSizeThreshold) throws IOException {
 
+		this(checkpointBaseDirectory.getFileSystem(),
+				checkpointBaseDirectory,
+				defaultSavepointDirectory,
+				jobId,
+				fileSizeThreshold);
+	}
+
+	public FsCheckpointStorage(
+			FileSystem fs,
+			Path checkpointBaseDirectory,
+			@Nullable Path defaultSavepointDirectory,
+			JobID jobId,
+			int fileSizeThreshold) throws IOException {
+
 		super(jobId, defaultSavepointDirectory);
 
 		checkArgument(fileSizeThreshold >= 0);
 
-		this.fileSystem = checkpointBaseDirectory.getFileSystem();
+		this.fileSystem = checkNotNull(fs);
 		this.checkpointsDirectory = getCheckpointDirectoryForJob(checkpointBaseDirectory, jobId);
 		this.sharedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_SHARED_STATE_DIR);
 		this.taskOwnedStateDirectory = new Path(checkpointsDirectory, CHECKPOINT_TASK_OWNED_STATE_DIR);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
index 5637b40..360ae2b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStorageLocation.java
@@ -19,6 +19,7 @@
 package org.apache.flink.runtime.state.filesystem;
 
 import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.core.fs.EntropyInjector;
 import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
@@ -67,7 +68,10 @@ public class FsCheckpointStorageLocation extends FsCheckpointStreamFactory imple
 		this.taskOwnedStateDirectory = checkNotNull(taskOwnedStateDir);
 		this.reference = checkNotNull(reference);
 
-		this.metadataFilePath = new Path(checkpointDir, AbstractFsCheckpointStorage.METADATA_FILE_NAME);
+		// the metadata file should not have entropy in its path
+		Path metadataDir = EntropyInjector.removeEntropyMarkerIfPresent(fileSystem, checkpointDir);
+
+		this.metadataFilePath = new Path(metadataDir, AbstractFsCheckpointStorage.METADATA_FILE_NAME);
 		this.fileStateSizeThreshold = fileStateSizeThreshold;
 	}
 
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
index 609ef69..665e7b3 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsCheckpointStreamFactory.java
@@ -18,8 +18,11 @@
 
 package org.apache.flink.runtime.state.filesystem;
 
+import org.apache.flink.core.fs.EntropyInjector;
 import org.apache.flink.core.fs.FSDataOutputStream;
 import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.FileSystem.WriteMode;
+import org.apache.flink.core.fs.OutputStreamAndPath;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.state.CheckpointStreamFactory;
 import org.apache.flink.runtime.state.CheckpointedStateScope;
@@ -344,12 +347,10 @@ public class FsCheckpointStreamFactory implements CheckpointStreamFactory {
 			Exception latestException = null;
 			for (int attempt = 0; attempt < 10; attempt++) {
 				try {
-					Path statePath = createStatePath();
-					FSDataOutputStream outStream = fs.create(statePath, FileSystem.WriteMode.NO_OVERWRITE);
-
-					// success, managed to open the stream
-					this.statePath = statePath;
-					this.outStream = outStream;
+					OutputStreamAndPath streamAndPath = EntropyInjector.createEntropyAware(
+							fs, createStatePath(), WriteMode.NO_OVERWRITE);
+					this.outStream = streamAndPath.stream();
+					this.statePath = streamAndPath.path();
 					return;
 				}
 				catch (Exception e) {
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
new file mode 100644
index 0000000..12a4c42
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/filesystem/FsStateBackendEntropyTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.core.fs.EntropyInjectingFileSystem;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.fs.local.LocalFileSystem;
+import org.apache.flink.runtime.state.CheckpointMetadataOutputStream;
+import org.apache.flink.runtime.state.CheckpointStreamFactory.CheckpointStateOutputStream;
+import org.apache.flink.runtime.state.CheckpointedStateScope;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.hamcrest.Matchers.containsString;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.startsWith;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+
+/**
+ * Tests verifying that the FsStateBackend passes the entropy injection option
+ * to the FileSystem for state payload files, but not for metadata files.
+ */
+public class FsStateBackendEntropyTest {
+
+	static final String ENTROPY_MARKER = "__ENTROPY__";
+	static final String RESOLVED_MARKER = "+RESOLVED+";
+
+	@Rule
+	public final TemporaryFolder tmp = new TemporaryFolder();
+
+	@Test
+	public void testEntropyInjection() throws Exception {
+		final FileSystem fs = new TestEntropyAwareFs();
+
+		final Path checkpointDir = new Path(Path.fromLocalFile(tmp.newFolder()), ENTROPY_MARKER + "/checkpoints");
+		final String checkpointDirStr = checkpointDir.toString();
+
+		FsCheckpointStorage storage = new FsCheckpointStorage(
+				fs, checkpointDir, null, new JobID(), 1024);
+
+		FsCheckpointStorageLocation location = (FsCheckpointStorageLocation)
+				storage.initializeLocationForCheckpoint(96562);
+
+		assertThat(location.getCheckpointDirectory().toString(), startsWith(checkpointDirStr));
+		assertThat(location.getSharedStateDirectory().toString(), startsWith(checkpointDirStr));
+		assertThat(location.getTaskOwnedStateDirectory().toString(), startsWith(checkpointDirStr));
+		assertThat(location.getMetadataFilePath().toString(), not(containsString(ENTROPY_MARKER)));
+
+		// check entropy in task-owned state
+		try (CheckpointStateOutputStream stream = storage.createTaskOwnedStateStream()) {
+			stream.flush();
+			FileStateHandle handle = (FileStateHandle) stream.closeAndGetHandle();
+
+			assertNotNull(handle);
+			assertThat(handle.getFilePath().toString(), not(containsString(ENTROPY_MARKER)));
+			assertThat(handle.getFilePath().toString(), containsString(RESOLVED_MARKER));
+		}
+
+		// check entropy in the exclusive/shared state
+		try (CheckpointStateOutputStream stream =
+				location.createCheckpointStateOutputStream(CheckpointedStateScope.EXCLUSIVE)) {
+
+			stream.flush();
+			FileStateHandle handle = (FileStateHandle) stream.closeAndGetHandle();
+
+			assertNotNull(handle);
+			assertThat(handle.getFilePath().toString(), not(containsString(ENTROPY_MARKER)));
+			assertThat(handle.getFilePath().toString(), containsString(RESOLVED_MARKER));
+		}
+
+		// check that there is no entropy in the metadata
+		// check entropy in the exclusive/shared state
+		try (CheckpointMetadataOutputStream stream = location.createMetadataOutputStream()) {
+			stream.flush();
+			FsCompletedCheckpointStorageLocation handle =
+					(FsCompletedCheckpointStorageLocation) stream.closeAndFinalizeCheckpoint();
+
+			assertNotNull(handle);
+
+			// metadata files have no entropy
+			assertThat(handle.getMetadataHandle().getFilePath().toString(), not(containsString(ENTROPY_MARKER)));
+			assertThat(handle.getMetadataHandle().getFilePath().toString(), not(containsString(RESOLVED_MARKER)));
+
+			// external location is the same as metadata, without the file name
+			assertEquals(handle.getMetadataHandle().getFilePath().getParent().toString(), handle.getExternalPointer());
+		}
+	}
+
+	private static class TestEntropyAwareFs extends LocalFileSystem implements EntropyInjectingFileSystem {
+
+		@Override
+		public String getEntropyInjectionKey() {
+			return ENTROPY_MARKER;
+		}
+
+		@Override
+		public String generateEntropy() {
+			return RESOLVED_MARKER;
+		}
+	}
+}