You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by cm...@apache.org on 2013/07/23 19:54:45 UTC
svn commit: r1506188 - in
/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop:
fs/ io/ io/nativeio/
Author: cmccabe
Date: Tue Jul 23 17:54:44 2013
New Revision: 1506188
URL: http://svn.apache.org/r1506188
Log:
HDFS-4817. Make HDFS advisory caching configurable on a per-file basis. (Colin Patrick McCabe)
Added:
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetReadahead.java
Modified:
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java
hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java?rev=1506188&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetDropBehind.java Tue Jul 23 17:54:44 2013
@@ -0,0 +1,41 @@
+/**
+ * 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.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface CanSetDropBehind {
+ /**
+ * Configure whether the stream should drop the cache.
+ *
+ * @param dropCache Whether to drop the cache. null means to use the
+ * default value.
+ * @throws IOException If there was an error changing the dropBehind
+ * setting.
+ * UnsupportedOperationException If this stream doesn't support
+ * setting the drop-behind.
+ */
+ public void setDropBehind(Boolean dropCache)
+ throws IOException, UnsupportedOperationException;
+}
Added: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetReadahead.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetReadahead.java?rev=1506188&view=auto
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetReadahead.java (added)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CanSetReadahead.java Tue Jul 23 17:54:44 2013
@@ -0,0 +1,40 @@
+/**
+ * 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.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public interface CanSetReadahead {
+ /**
+ * Set the readahead on this stream.
+ *
+ * @param readahead The readahead to use. null means to use the default.
+ * @throws IOException If there was an error changing the dropBehind
+ * setting.
+ * UnsupportedOperationException If this stream doesn't support
+ * setting readahead.
+ */
+ public void setReadahead(Long readahead)
+ throws IOException, UnsupportedOperationException;
+}
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java?rev=1506188&r1=1506187&r2=1506188&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java Tue Jul 23 17:54:44 2013
@@ -28,7 +28,8 @@ import org.apache.hadoop.classification.
@InterfaceAudience.Public
@InterfaceStability.Stable
public class FSDataInputStream extends DataInputStream
- implements Seekable, PositionedReadable, Closeable, ByteBufferReadable, HasFileDescriptor {
+ implements Seekable, PositionedReadable, Closeable,
+ ByteBufferReadable, HasFileDescriptor, CanSetDropBehind, CanSetReadahead {
public FSDataInputStream(InputStream in)
throws IOException {
@@ -143,4 +144,27 @@ public class FSDataInputStream extends D
return null;
}
}
+
+ @Override
+ public void setReadahead(Long readahead)
+ throws IOException, UnsupportedOperationException {
+ try {
+ ((CanSetReadahead)in).setReadahead(readahead);
+ } catch (ClassCastException e) {
+ throw new UnsupportedOperationException(
+ "this stream does not support setting the readahead " +
+ "caching strategy.");
+ }
+ }
+
+ @Override
+ public void setDropBehind(Boolean dropBehind)
+ throws IOException, UnsupportedOperationException {
+ try {
+ ((CanSetDropBehind)in).setDropBehind(dropBehind);
+ } catch (ClassCastException e) {
+ throw new UnsupportedOperationException("this stream does not " +
+ "support setting the drop-behind caching setting.");
+ }
+ }
}
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java?rev=1506188&r1=1506187&r2=1506188&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStream.java Tue Jul 23 17:54:44 2013
@@ -18,6 +18,10 @@
package org.apache.hadoop.fs;
import java.io.*;
+import java.io.DataOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@@ -26,8 +30,9 @@ import org.apache.hadoop.classification.
*/
@InterfaceAudience.Public
@InterfaceStability.Stable
-public class FSDataOutputStream extends DataOutputStream implements Syncable {
- private OutputStream wrappedStream;
+public class FSDataOutputStream extends DataOutputStream
+ implements Syncable, CanSetDropBehind {
+ private final OutputStream wrappedStream;
private static class PositionCache extends FilterOutputStream {
private FileSystem.Statistics statistics;
@@ -133,4 +138,14 @@ public class FSDataOutputStream extends
wrappedStream.flush();
}
}
+
+ @Override
+ public void setDropBehind(Boolean dropBehind) throws IOException {
+ try {
+ ((CanSetDropBehind)wrappedStream).setDropBehind(dropBehind);
+ } catch (ClassCastException e) {
+ throw new UnsupportedOperationException("the wrapped stream does " +
+ "not support setting the drop-behind caching setting.");
+ }
+ }
}
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java?rev=1506188&r1=1506187&r2=1506188&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java Tue Jul 23 17:54:44 2013
@@ -810,7 +810,8 @@ public class HarFileSystem extends Filte
/**
* Create an input stream that fakes all the reads/positions/seeking.
*/
- private static class HarFsInputStream extends FSInputStream {
+ private static class HarFsInputStream extends FSInputStream
+ implements CanSetDropBehind, CanSetReadahead {
private long position, start, end;
//The underlying data input stream that the
// underlying filesystem will return.
@@ -957,7 +958,18 @@ public class HarFileSystem extends Filte
public void readFully(long pos, byte[] b) throws IOException {
readFully(pos, b, 0, b.length);
}
-
+
+ @Override
+ public void setReadahead(Long readahead)
+ throws IOException, UnsupportedEncodingException {
+ underLyingStream.setReadahead(readahead);
+ }
+
+ @Override
+ public void setDropBehind(Boolean dropBehind)
+ throws IOException, UnsupportedEncodingException {
+ underLyingStream.setDropBehind(dropBehind);
+ }
}
/**
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java?rev=1506188&r1=1506187&r2=1506188&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java Tue Jul 23 17:54:44 2013
@@ -203,7 +203,7 @@ public class ReadaheadPool {
// It's also possible that we'll end up requesting readahead on some
// other FD, which may be wasted work, but won't cause a problem.
try {
- NativeIO.POSIX.posixFadviseIfPossible(fd, off, len,
+ NativeIO.POSIX.posixFadviseIfPossible(identifier, fd, off, len,
NativeIO.POSIX.POSIX_FADV_WILLNEED);
} catch (IOException ioe) {
if (canceled) {
Modified: hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java?rev=1506188&r1=1506187&r2=1506188&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java (original)
+++ hadoop/common/branches/branch-2/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java Tue Jul 23 17:54:44 2013
@@ -37,6 +37,8 @@ import org.apache.hadoop.util.Shell;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
+import com.google.common.annotations.VisibleForTesting;
+
/**
* JNI wrappers for various native IO-related calls not available in Java.
* These functions should generally be used alongside a fallback to another
@@ -92,6 +94,9 @@ public class NativeIO {
private static final Log LOG = LogFactory.getLog(NativeIO.class);
+ @VisibleForTesting
+ public static CacheTracker cacheTracker = null;
+
private static boolean nativeLoaded = false;
private static boolean fadvisePossible = true;
private static boolean syncFileRangePossible = true;
@@ -102,6 +107,10 @@ public class NativeIO {
private static long cacheTimeout = -1;
+ public static interface CacheTracker {
+ public void fadvise(String identifier, long offset, long len, int flags);
+ }
+
static {
if (NativeCodeLoader.isNativeCodeLoaded()) {
try {
@@ -178,9 +187,12 @@ public class NativeIO {
*
* @throws NativeIOException if there is an error with the syscall
*/
- public static void posixFadviseIfPossible(
+ public static void posixFadviseIfPossible(String identifier,
FileDescriptor fd, long offset, long len, int flags)
throws NativeIOException {
+ if (cacheTracker != null) {
+ cacheTracker.fadvise(identifier, offset, len, flags);
+ }
if (nativeLoaded && fadvisePossible) {
try {
posix_fadvise(fd, offset, len, flags);