You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by el...@apache.org on 2013/10/24 21:16:04 UTC

git commit: ACCUMULO-1637 Update the HDFS configuration check in VolumnManagerImpl from the one in TabletServer.

Updated Branches:
  refs/heads/master 6337c285f -> ab6779c30


ACCUMULO-1637 Update the HDFS configuration check in VolumnManagerImpl
from the one in TabletServer.

Looks like I missed the movement of the configuration validation code in
1.6.0 from TabletServer to VolumeManagerImpl. Updated the check, and
fixed the unit test to invoke the correct code.


Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo
Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/ab6779c3
Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/ab6779c3
Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/ab6779c3

Branch: refs/heads/master
Commit: ab6779c30c0308b5905e894c74fce24f6aca7679
Parents: 6337c28
Author: Josh Elser <el...@apache.org>
Authored: Thu Oct 24 11:49:59 2013 -0700
Committer: Josh Elser <el...@apache.org>
Committed: Thu Oct 24 12:15:52 2013 -0700

----------------------------------------------------------------------
 .../accumulo/server/fs/VolumeManagerImpl.java   |  57 +++--
 .../server/tabletserver/TabletServer.java       |  55 -----
 .../server/fs/TabletServerSyncCheckTest.java    | 243 +++++++++++++++++++
 .../tabletserver/TabletServerSyncCheckTest.java | 241 ------------------
 4 files changed, 284 insertions(+), 312 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/ab6779c3/server/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 6c26c04..39afe75 100644
--- a/server/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.server.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.net.URI;
 import java.util.ArrayList;
@@ -26,6 +27,7 @@ import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.DefaultConfiguration;
@@ -48,6 +50,7 @@ import org.apache.hadoop.fs.LocalFileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.util.Progressable;
 import org.apache.log4j.Logger;
@@ -200,30 +203,52 @@ public class VolumeManagerImpl implements VolumeManager {
     return getFileSystemByPath(path).delete(path, true);
   }
 
-  private void ensureSyncIsEnabled() {
-    for (FileSystem fs : getFileSystems().values()) {
+  protected void ensureSyncIsEnabled() {
+    for (Entry<String,? extends FileSystem> entry : getFileSystems().entrySet()) {
+      final String volumeName = entry.getKey();
+      final FileSystem fs = entry.getValue();
+      
       if (fs instanceof DistributedFileSystem) {
-        if (!fs.getConf().getBoolean("dfs.durable.sync", false) && !fs.getConf().getBoolean("dfs.support.append", false)) {
-          String msg = "Must set dfs.durable.sync OR dfs.support.append to true.  Which one needs to be set depends on your version of HDFS.  See ACCUMULO-623. \n"
-              + "HADOOP RELEASE          VERSION           SYNC NAME             DEFAULT\n"
-              + "Apache Hadoop           0.20.205          dfs.support.append    false\n"
-              + "Apache Hadoop            0.23.x           dfs.support.append    true\n"
-              + "Apache Hadoop             1.0.x           dfs.support.append    false\n"
-              + "Apache Hadoop             1.1.x           dfs.durable.sync      true\n"
-              + "Apache Hadoop          2.0.0-2.0.2        dfs.support.append    true\n"
-              + "Cloudera CDH             3u0-3u3             ????               true\n"
-              + "Cloudera CDH               3u4            dfs.support.append    true\n"
-              + "Hortonworks HDP           `1.0            dfs.support.append    false\n"
-              + "Hortonworks HDP           `1.1            dfs.support.append    false";
+        final String DFS_DURABLE_SYNC = "dfs.durable.sync", DFS_SUPPORT_APPEND = "dfs.support.append";
+        final String ticketMessage = "See ACCUMULO-623 and ACCUMULO-1637 for more details.";
+        // Check to make sure that we have proper defaults configured
+        try {
+          // If the default is off (0.20.205.x or 1.0.x)
+          DFSConfigKeys configKeys = new DFSConfigKeys();
+          
+          // Can't use the final constant itself as Java will inline it at compile time
+          Field dfsSupportAppendDefaultField = configKeys.getClass().getField("DFS_SUPPORT_APPEND_DEFAULT");
+          boolean dfsSupportAppendDefaultValue = dfsSupportAppendDefaultField.getBoolean(configKeys);
+          
+          if (!dfsSupportAppendDefaultValue) {
+            // See if the user did the correct override
+            if (!fs.getConf().getBoolean(DFS_SUPPORT_APPEND, false)) {
+              String msg = "Accumulo requires that dfs.support.append to true. " + ticketMessage;
+              log.fatal(msg);
+              throw new RuntimeException(msg);
+            }
+          }
+        } catch (NoSuchFieldException e) {
+          // If we can't find DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT, the user is running
+          // 1.1.x or 1.2.x. This is ok, though, as, by default, these versions have append/sync enabled.
+        } catch (Exception e) {
+          log.warn("Error while checking for " + DFS_SUPPORT_APPEND + " on volume " + volumeName + ". The user should ensure that Hadoop is configured to properly supports append and sync. " + ticketMessage, e);
+        }
+        
+        // If either of these parameters are configured to be false, fail.
+        // This is a sign that someone is writing bad configuration.
+        if (!fs.getConf().getBoolean(DFS_SUPPORT_APPEND, true) || !fs.getConf().getBoolean(DFS_DURABLE_SYNC, true)) {
+          String msg = "Accumulo requires that " + DFS_SUPPORT_APPEND + " and " + DFS_DURABLE_SYNC + " not be configured as false. " + ticketMessage;
           log.fatal(msg);
-          System.exit(-1);
+          throw new RuntimeException(msg);
         }
+        
         try {
           // if this class exists
           Class.forName("org.apache.hadoop.fs.CreateFlag");
           // we're running hadoop 2.0, 1.1
           if (!fs.getConf().getBoolean("dfs.datanode.synconclose", false)) {
-            log.warn("dfs.datanode.synconclose set to false: data loss is possible on system reset or power loss");
+            log.warn("dfs.datanode.synconclose set to false: data loss is possible on system reset or power loss on volume " + volumeName);
           }
         } catch (ClassNotFoundException ex) {
           // hadoop 1.0

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ab6779c3/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
index 3aac4f4..a4009ad 100644
--- a/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
+++ b/server/src/main/java/org/apache/accumulo/server/tabletserver/TabletServer.java
@@ -3599,7 +3599,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       Instance instance = HdfsZooInstance.getInstance();
       ServerConfiguration conf = new ServerConfiguration(instance);
       Accumulo.init(fs, conf, "tserver");
-      ensureHdfsSyncIsEnabled(fs);
       TabletServer server = new TabletServer(conf, fs);
       server.config(hostname);
       Accumulo.enableTracing(hostname, "tserver");
@@ -3609,60 +3608,6 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
       System.exit(1);
     }
   }
-  
-  protected static void ensureHdfsSyncIsEnabled(VolumeManager volumes) {
-    for (Entry<String,? extends FileSystem> entry : volumes.getFileSystems().entrySet()) {
-      final String volumeName = entry.getKey();
-      final FileSystem fs = entry.getValue();
-      
-      if (fs instanceof DistributedFileSystem) {
-        final String DFS_DURABLE_SYNC = "dfs.durable.sync", DFS_SUPPORT_APPEND = "dfs.support.append";
-        final String ticketMessage = "See ACCUMULO-623 and ACCUMULO-1637 for more details.";
-        // Check to make sure that we have proper defaults configured
-        try {
-          // If the default is off (0.20.205.x or 1.0.x)
-          DFSConfigKeys configKeys = new DFSConfigKeys();
-          
-          // Can't use the final constant itself as Java will inline it at compile time
-          Field dfsSupportAppendDefaultField = configKeys.getClass().getField("DFS_SUPPORT_APPEND_DEFAULT");
-          boolean dfsSupportAppendDefaultValue = dfsSupportAppendDefaultField.getBoolean(configKeys);
-          
-          if (!dfsSupportAppendDefaultValue) {
-            // See if the user did the correct override
-            if (!fs.getConf().getBoolean(DFS_SUPPORT_APPEND, false)) {
-              String msg = "Accumulo requires that dfs.support.append to true. " + ticketMessage;
-              log.fatal(msg);
-              throw new RuntimeException(msg);
-            }
-          }
-        } catch (NoSuchFieldException e) {
-          // If we can't find DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT, the user is running
-          // 1.1.x or 1.2.x. This is ok, though, as, by default, these versions have append/sync enabled.
-        } catch (Exception e) {
-          log.warn("Error while checking for " + DFS_SUPPORT_APPEND + " on volume " + volumeName + ". The user should ensure that Hadoop is configured to properly supports append and sync. " + ticketMessage, e);
-        }
-        
-        // If either of these parameters are configured to be false, fail.
-        // This is a sign that someone is writing bad configuration.
-        if (!fs.getConf().getBoolean(DFS_SUPPORT_APPEND, true) || !fs.getConf().getBoolean(DFS_DURABLE_SYNC, true)) {
-          String msg = "Accumulo requires that " + DFS_SUPPORT_APPEND + " and " + DFS_DURABLE_SYNC + " not be configured as false. " + ticketMessage;
-          log.fatal(msg);
-          throw new RuntimeException(msg);
-        }
-        
-        try {
-          // if this class exists
-          Class.forName("org.apache.hadoop.fs.CreateFlag");
-          // we're running hadoop 2.0, 1.1
-          if (!fs.getConf().getBoolean("dfs.datanode.synconclose", false)) {
-            log.warn("dfs.datanode.synconclose set to false: data loss is possible on system reset or power loss on volume " + volumeName);
-          }
-        } catch (ClassNotFoundException ex) {
-          // hadoop 1.0
-        }
-      }
-    }
-  }
 
   public void minorCompactionFinished(CommitSession tablet, String newDatafile, int walogSeq) throws IOException {
     totalMinorCompactions++;

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ab6779c3/server/src/test/java/org/apache/accumulo/server/fs/TabletServerSyncCheckTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/accumulo/server/fs/TabletServerSyncCheckTest.java b/server/src/test/java/org/apache/accumulo/server/fs/TabletServerSyncCheckTest.java
new file mode 100644
index 0000000..2c215ec
--- /dev/null
+++ b/server/src/test/java/org/apache/accumulo/server/fs/TabletServerSyncCheckTest.java
@@ -0,0 +1,243 @@
+/*
+ * 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.accumulo.server.fs;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.accumulo.core.conf.ConfigurationCopy;
+import org.apache.accumulo.core.data.Key;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.Test;
+
+import com.google.common.collect.ImmutableMap;
+
+public class TabletServerSyncCheckTest {
+  private static final String DFS_DURABLE_SYNC = "dfs.durable.sync", DFS_SUPPORT_APPEND = "dfs.support.append";
+  
+  @Test(expected = RuntimeException.class)
+  public void testFailureOnExplicitSyncFalseConf() {
+    Configuration conf = new Configuration();
+    conf.set(DFS_DURABLE_SYNC, "false");
+    
+    FileSystem fs = new TestFileSystem(conf);
+    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("foo", fs));
+    
+    vm.ensureSyncIsEnabled();
+  }
+  
+  @Test(expected = RuntimeException.class)
+  public void testFailureOnSingleExplicitSyncFalseConf() {
+    Configuration conf1 = new Configuration(), conf2 = new Configuration();
+    conf1.set(DFS_DURABLE_SYNC, "false");
+    
+    FileSystem fs1 = new TestFileSystem(conf1);
+    FileSystem fs2 = new TestFileSystem(conf2);
+    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("bar", fs2, "foo", fs1));
+
+    vm.ensureSyncIsEnabled();
+  }
+  
+  @Test(expected = RuntimeException.class)
+  public void testFailureOnExplicitAppendFalseConf() {
+    Configuration conf = new Configuration();
+    conf.set(DFS_SUPPORT_APPEND, "false");
+    
+    FileSystem fs = new TestFileSystem(conf);
+    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("foo", fs));
+
+    vm.ensureSyncIsEnabled();
+  }
+  
+  private class TestFileSystem extends DistributedFileSystem {
+    protected final Configuration conf;
+    
+    public TestFileSystem(Configuration conf) {
+      this.conf = conf;
+    }
+    
+    @Override
+    public Configuration getConf() {
+      return conf;
+    }
+    
+  }
+  
+  private class TestVolumeManagerImpl extends VolumeManagerImpl {
+    
+    protected final Map<String,? extends FileSystem> volumes;
+
+    public TestVolumeManagerImpl(Map<String,? extends FileSystem> volumes) {
+      super(volumes, volumes.keySet().iterator().next(), new ConfigurationCopy(Collections.<String,String> emptyMap()));
+      this.volumes = volumes;
+    }
+
+    @Override
+    public void close() throws IOException {
+      
+    }
+
+    @Override
+    public boolean closePossiblyOpenFile(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FSDataOutputStream create(Path dest) throws IOException {
+      return null;
+    }
+
+    @Override
+    public FSDataOutputStream create(Path path, boolean b) throws IOException {
+      return null;
+    }
+
+    @Override
+    public FSDataOutputStream create(Path path, boolean b, int int1, short int2, long long1) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean createNewFile(Path writable) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FSDataOutputStream createSyncable(Path logPath, int buffersize, short replication, long blockSize) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean delete(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public boolean deleteRecursively(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public boolean exists(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FileStatus getFileStatus(Path path) throws IOException {
+      return null;
+    }
+
+    @Override
+    public FileSystem getFileSystemByPath(Path path) {
+      return null;
+    }
+
+    @Override
+    public Map<String,? extends FileSystem> getFileSystems() {
+      return volumes;
+    }
+
+    @Override
+    public Path matchingFileSystem(Path source, String[] options) {
+      return null;
+    }
+
+    @Override
+    public FileStatus[] listStatus(Path path) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean mkdirs(Path directory) throws IOException {
+      return false;
+    }
+
+    @Override
+    public FSDataInputStream open(Path path) throws IOException {
+      return null;
+    }
+
+    @Override
+    public boolean rename(Path path, Path newPath) throws IOException {
+      return false;
+    }
+
+    @Override
+    public boolean moveToTrash(Path sourcePath) throws IOException {
+      return false;
+    }
+
+    @Override
+    public short getDefaultReplication(Path logPath) {
+      return 0;
+    }
+
+    @Override
+    public boolean isFile(Path path) throws IOException {
+      return false;
+    }
+
+    @Override
+    public boolean isReady() throws IOException {
+      return false;
+    }
+
+    @Override
+    public FileSystem getDefaultVolume() {
+      return null;
+    }
+
+    @Override
+    public FileStatus[] globStatus(Path path) throws IOException {
+      return null;
+    }
+
+    @Override
+    public Path getFullPath(Key key) {
+      return null;
+    }
+
+    @Override
+    public Path getFullPath(String tableId, String path) {
+      return null;
+    }
+
+    @Override
+    public Path getFullPath(FileType fileType, String fileName) {
+      return null;
+    }
+
+    @Override
+    public ContentSummary getContentSummary(Path dir) throws IOException {
+      return null;
+    }
+
+    @Override
+    public String choose(String[] options) {
+      return null;
+    }
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/ab6779c3/server/src/test/java/org/apache/accumulo/server/tabletserver/TabletServerSyncCheckTest.java
----------------------------------------------------------------------
diff --git a/server/src/test/java/org/apache/accumulo/server/tabletserver/TabletServerSyncCheckTest.java b/server/src/test/java/org/apache/accumulo/server/tabletserver/TabletServerSyncCheckTest.java
deleted file mode 100644
index 9450364..0000000
--- a/server/src/test/java/org/apache/accumulo/server/tabletserver/TabletServerSyncCheckTest.java
+++ /dev/null
@@ -1,241 +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.accumulo.server.tabletserver;
-
-import java.io.IOException;
-import java.util.Map;
-
-import org.apache.accumulo.core.data.Key;
-import org.apache.accumulo.server.fs.VolumeManager;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableMap;
-
-public class TabletServerSyncCheckTest {
-  private static final String DFS_DURABLE_SYNC = "dfs.durable.sync", DFS_SUPPORT_APPEND = "dfs.support.append";
-  
-  @Test(expected = RuntimeException.class)
-  public void testFailureOnExplicitSyncFalseConf() {
-    Configuration conf = new Configuration();
-    conf.set(DFS_DURABLE_SYNC, "false");
-    
-    FileSystem fs = new TestFileSystem(conf);
-    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("foo", fs));
-    
-    TabletServer.ensureHdfsSyncIsEnabled(vm);
-  }
-  
-  @Test(expected = RuntimeException.class)
-  public void testFailureOnSingleExplicitSyncFalseConf() {
-    Configuration conf1 = new Configuration(), conf2 = new Configuration();
-    conf1.set(DFS_DURABLE_SYNC, "false");
-    
-    FileSystem fs1 = new TestFileSystem(conf1);
-    FileSystem fs2 = new TestFileSystem(conf2);
-    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("bar", fs2, "foo", fs1));
-    
-    TabletServer.ensureHdfsSyncIsEnabled(vm);
-  }
-  
-  @Test(expected = RuntimeException.class)
-  public void testFailureOnExplicitAppendFalseConf() {
-    Configuration conf = new Configuration();
-    conf.set(DFS_SUPPORT_APPEND, "false");
-    
-    FileSystem fs = new TestFileSystem(conf);
-    TestVolumeManagerImpl vm = new TestVolumeManagerImpl(ImmutableMap.of("foo", fs));
-    
-    TabletServer.ensureHdfsSyncIsEnabled(vm);
-  }
-  
-  private class TestFileSystem extends DistributedFileSystem {
-    protected final Configuration conf;
-    
-    public TestFileSystem(Configuration conf) {
-      this.conf = conf;
-    }
-    
-    @Override
-    public Configuration getConf() {
-      return conf;
-    }
-    
-  }
-  
-  private class TestVolumeManagerImpl implements VolumeManager {
-    
-    protected final Map<String,? extends FileSystem> volumes;
-
-    public TestVolumeManagerImpl(Map<String,? extends FileSystem> volumes) {
-      this.volumes = volumes;
-    }
-
-    @Override
-    public void close() throws IOException {
-      
-    }
-
-    @Override
-    public boolean closePossiblyOpenFile(Path path) throws IOException {
-      return false;
-    }
-
-    @Override
-    public FSDataOutputStream create(Path dest) throws IOException {
-      return null;
-    }
-
-    @Override
-    public FSDataOutputStream create(Path path, boolean b) throws IOException {
-      return null;
-    }
-
-    @Override
-    public FSDataOutputStream create(Path path, boolean b, int int1, short int2, long long1) throws IOException {
-      return null;
-    }
-
-    @Override
-    public boolean createNewFile(Path writable) throws IOException {
-      return false;
-    }
-
-    @Override
-    public FSDataOutputStream createSyncable(Path logPath, int buffersize, short replication, long blockSize) throws IOException {
-      return null;
-    }
-
-    @Override
-    public boolean delete(Path path) throws IOException {
-      return false;
-    }
-
-    @Override
-    public boolean deleteRecursively(Path path) throws IOException {
-      return false;
-    }
-
-    @Override
-    public boolean exists(Path path) throws IOException {
-      return false;
-    }
-
-    @Override
-    public FileStatus getFileStatus(Path path) throws IOException {
-      return null;
-    }
-
-    @Override
-    public FileSystem getFileSystemByPath(Path path) {
-      return null;
-    }
-
-    @Override
-    public Map<String,? extends FileSystem> getFileSystems() {
-      return volumes;
-    }
-
-    @Override
-    public Path matchingFileSystem(Path source, String[] options) {
-      return null;
-    }
-
-    @Override
-    public FileStatus[] listStatus(Path path) throws IOException {
-      return null;
-    }
-
-    @Override
-    public boolean mkdirs(Path directory) throws IOException {
-      return false;
-    }
-
-    @Override
-    public FSDataInputStream open(Path path) throws IOException {
-      return null;
-    }
-
-    @Override
-    public boolean rename(Path path, Path newPath) throws IOException {
-      return false;
-    }
-
-    @Override
-    public boolean moveToTrash(Path sourcePath) throws IOException {
-      return false;
-    }
-
-    @Override
-    public short getDefaultReplication(Path logPath) {
-      return 0;
-    }
-
-    @Override
-    public boolean isFile(Path path) throws IOException {
-      return false;
-    }
-
-    @Override
-    public boolean isReady() throws IOException {
-      return false;
-    }
-
-    @Override
-    public FileSystem getDefaultVolume() {
-      return null;
-    }
-
-    @Override
-    public FileStatus[] globStatus(Path path) throws IOException {
-      return null;
-    }
-
-    @Override
-    public Path getFullPath(Key key) {
-      return null;
-    }
-
-    @Override
-    public Path getFullPath(String tableId, String path) {
-      return null;
-    }
-
-    @Override
-    public Path getFullPath(FileType fileType, String fileName) throws IOException {
-      return null;
-    }
-
-    @Override
-    public ContentSummary getContentSummary(Path dir) throws IOException {
-      return null;
-    }
-
-    @Override
-    public String choose(String[] options) {
-      return null;
-    }
-    
-  }
-}