You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by kt...@apache.org on 2013/12/20 05:13:39 UTC

[1/5] git commit: ACCUMULO-2072 fixed generation of wal recovery paths

Updated Branches:
  refs/heads/master 260b6bf9f -> 322ee056c


ACCUMULO-2072 fixed generation of wal recovery paths


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

Branch: refs/heads/master
Commit: 0b95612faf0ea6f340a1da3f8f04dc6b6e2bb2b0
Parents: 0d34428
Author: Keith Turner <kt...@apache.org>
Authored: Thu Dec 19 22:57:08 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Dec 19 23:00:02 2013 -0500

----------------------------------------------------------------------
 .../server/master/recovery/RecoveryPath.java    | 56 ++++++++++++++++++++
 .../master/recovery/RecoveryManager.java        | 14 +++--
 .../apache/accumulo/tserver/TabletServer.java   |  5 +-
 3 files changed, 65 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0b95612f/server/base/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryPath.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryPath.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryPath.java
new file mode 100644
index 0000000..1da945d
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/RecoveryPath.java
@@ -0,0 +1,56 @@
+/*
+ * 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.master.recovery;
+
+import java.io.IOException;
+
+import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.fs.VolumeManager.FileType;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * 
+ */
+public class RecoveryPath {
+
+  // given a wal path, transform it to a recovery path
+  public static Path getRecoveryPath(VolumeManager fs, Path walPath) throws IOException {
+    if (walPath.depth() >= 3 && walPath.toUri().getScheme() != null) {
+      // its a fully qualified path
+      String uuid = walPath.getName();
+      // drop uuid
+      walPath = walPath.getParent();
+      // drop server
+      walPath = walPath.getParent();
+  
+      if (!walPath.getName().equals(FileType.WAL.getDirectory()))
+        throw new IllegalArgumentException("Bad path " + walPath);
+  
+      // drop wal
+      walPath = walPath.getParent();
+  
+      walPath = new Path(walPath, FileType.RECOVERY.getDirectory());
+      walPath = new Path(walPath, uuid);
+
+      return walPath;
+    }
+  
+    throw new IllegalArgumentException("Bad path " + walPath);
+  
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0b95612f/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
----------------------------------------------------------------------
diff --git a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index dbe73ac..789d482 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -35,10 +35,10 @@ import org.apache.accumulo.core.data.KeyExtent;
 import org.apache.accumulo.core.util.NamingThreadFactory;
 import org.apache.accumulo.core.zookeeper.ZooUtil;
 import org.apache.accumulo.master.Master;
-import org.apache.accumulo.server.ServerConstants;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.master.recovery.HadoopLogCloser;
 import org.apache.accumulo.server.master.recovery.LogCloser;
+import org.apache.accumulo.server.master.recovery.RecoveryPath;
 import org.apache.accumulo.server.zookeeper.DistributedWorkQueue;
 import org.apache.accumulo.server.zookeeper.ZooCache;
 import org.apache.hadoop.fs.Path;
@@ -126,13 +126,11 @@ public class RecoveryManager {
     ;
     for (Collection<String> logs : walogs) {
       for (String walog : logs) {
-        String hostFilename[] = walog.split("/", 2);
-        String host = hostFilename[0];
-        String filename = hostFilename[1];
-        String parts[] = filename.split("/");
+
+        String parts[] = walog.split("/");
         String sortId = parts[parts.length - 1];
-        String dest = master.getFileSystem().choose(ServerConstants.getRecoveryDirs()) + "/" + sortId;
-        filename = master.getFileSystem().getFullPath(FileType.WAL, walog).toString();
+        String filename = master.getFileSystem().getFullPath(FileType.WAL, walog).toString();
+        String dest = RecoveryPath.getRecoveryPath(master.getFileSystem(), new Path(filename)).toString();
         log.debug("Recovering " + filename + " to " + dest);
 
         boolean sortQueued;
@@ -167,7 +165,7 @@ public class RecoveryManager {
               delay = Math.min(2 * delay, 1000 * 60 * 5l);
             }
 
-            log.info("Starting recovery of " + filename + " (in : " + (delay / 1000) + "s) created for " + host + ", tablet " + extent + " holds a reference");
+            log.info("Starting recovery of " + filename + " (in : " + (delay / 1000) + "s), tablet " + extent + " holds a reference");
 
             executor.schedule(new LogSortTask(closer, filename, dest, sortId), delay, TimeUnit.MILLISECONDS);
             closeTasksQueued.add(sortId);

http://git-wip-us.apache.org/repos/asf/accumulo/blob/0b95612f/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
index 9dabee7..e3f508f 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
@@ -157,6 +157,7 @@ import org.apache.accumulo.server.fs.FileRef;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.fs.VolumeManager.FileType;
 import org.apache.accumulo.server.fs.VolumeManagerImpl;
+import org.apache.accumulo.server.master.recovery.RecoveryPath;
 import org.apache.accumulo.server.master.state.Assignment;
 import org.apache.accumulo.server.master.state.DistributedStoreException;
 import org.apache.accumulo.server.master.state.TServerInstance;
@@ -3661,8 +3662,8 @@ public class TabletServer extends AbstractMetricsImpl implements org.apache.accu
     for (LogEntry entry : sorted) {
       Path recovery = null;
       for (String log : entry.logSet) {
-        String[] parts = log.split("/"); // "host:port/filename"
-        Path finished = new Path(fs.getFullPath(FileType.RECOVERY, parts[parts.length - 1]), "finished");
+        Path finished = RecoveryPath.getRecoveryPath(fs, fs.getFullPath(FileType.WAL, log));
+        finished = new Path(finished, "finished");
         TabletServer.log.info("Looking for " + finished);
         if (fs.exists(finished)) {
           recovery = finished.getParent();


[3/5] git commit: ACCUMULO-2044 fix HdfsZooInstance issue w/ multiple namenodes

Posted by kt...@apache.org.
ACCUMULO-2044 fix HdfsZooInstance issue w/ multiple namenodes


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

Branch: refs/heads/master
Commit: cddf04f72cf7490c5e7a096d742ef877d485798f
Parents: 0893667
Author: Keith Turner <kt...@apache.org>
Authored: Wed Dec 18 00:10:56 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Dec 19 23:00:02 2013 -0500

----------------------------------------------------------------------
 .../accumulo/core/client/admin/TableOperationsImpl.java      | 4 +---
 .../org/apache/accumulo/core/client/impl/OfflineScanner.java | 8 +-------
 .../main/java/org/apache/accumulo/core/file/FileUtil.java    | 7 +++++++
 .../java/org/apache/accumulo/core/zookeeper/ZooUtil.java     | 4 +++-
 4 files changed, 12 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/cddf04f7/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
index 26192ef..76cda7d 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java
@@ -1152,13 +1152,11 @@ public class TableOperationsImpl extends TableOperationsHelper {
   @SuppressWarnings("deprecation")
   private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException {
     Path ret;
-    FileSystem fs;
+    FileSystem fs = FileUtil.getFileSystem(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance));
 
     if (dir.contains(":")) {
       ret = new Path(dir);
-      fs = ret.getFileSystem(CachedConfiguration.getInstance());
     } else {
-      fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance));
       ret = fs.makeQualified(new Path(dir));
     }
 

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cddf04f7/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
index 6e08710..0cb0409 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java
@@ -67,7 +67,6 @@ import org.apache.accumulo.core.util.UtilWaitThread;
 import org.apache.commons.lang.NotImplementedException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 
 class OfflineIterator implements Iterator<Entry<Key,Value>> {
@@ -296,8 +295,6 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     
     Configuration conf = CachedConfiguration.getInstance();
     
-    FileSystem defaultFs = FileUtil.getFileSystem(conf, ServerConfigurationUtil.getConfiguration(instance));
-    
     for (SortedKeyValueIterator<Key,Value> reader : readers) {
       ((FileSKVIterator) reader).close();
     }
@@ -306,10 +303,7 @@ class OfflineIterator implements Iterator<Entry<Key,Value>> {
     
     // TODO need to close files - ACCUMULO-1303
     for (String file : absFiles) {
-      FileSystem fs = defaultFs;
-      if (file.contains(":"))
-        fs = new Path(file).getFileSystem(conf);
-
+      FileSystem fs = FileUtil.getFileSystem(file, conf, ServerConfigurationUtil.getConfiguration(instance));
       FileSKVIterator reader = FileOperations.getInstance().openReader(file, false, fs, conf, acuTableConf, null, null);
       readers.add(reader);
     }

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cddf04f7/core/src/main/java/org/apache/accumulo/core/file/FileUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileUtil.java b/core/src/main/java/org/apache/accumulo/core/file/FileUtil.java
index 998aa9e..33e1835 100644
--- a/core/src/main/java/org/apache/accumulo/core/file/FileUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/file/FileUtil.java
@@ -544,6 +544,13 @@ public class FileUtil {
     return results;
   }
   
+  public static FileSystem getFileSystem(String path, Configuration conf, AccumuloConfiguration acuconf) throws IOException {
+    if (path.contains(":"))
+      return new Path(path).getFileSystem(conf);
+    else
+      return getFileSystem(conf, acuconf);
+  }
+
   public static FileSystem getFileSystem(Configuration conf, AccumuloConfiguration acuconf) throws IOException {
     String uri = acuconf.get(Property.INSTANCE_DFS_URI);
     if ("".equals(uri))

http://git-wip-us.apache.org/repos/asf/accumulo/blob/cddf04f7/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
index 17447e5..9206beb 100644
--- a/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
+++ b/core/src/main/java/org/apache/accumulo/core/zookeeper/ZooUtil.java
@@ -44,10 +44,12 @@ public class ZooUtil extends org.apache.accumulo.fate.zookeeper.ZooUtil {
   /**
    * Utility to support certain client side utilities to minimize command-line options.
    */
+
   public static String getInstanceIDFromHdfs(Path instanceDirectory) {
     try {
+
       @SuppressWarnings("deprecation")
-      FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
+      FileSystem fs = FileUtil.getFileSystem(instanceDirectory.toString(), CachedConfiguration.getInstance(), AccumuloConfiguration.getSiteConfiguration());
       FileStatus[] files = null;
       try {
         files = fs.listStatus(instanceDirectory);


[5/5] git commit: Merge remote-tracking branch 'origin/1.6.0-SNAPSHOT'

Posted by kt...@apache.org.
Merge remote-tracking branch 'origin/1.6.0-SNAPSHOT'


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

Branch: refs/heads/master
Commit: 322ee056cc61146da461d27d92f7867f8d9e19f8
Parents: 260b6bf 0b95612
Author: Keith Turner <kt...@apache.org>
Authored: Thu Dec 19 23:13:01 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Dec 19 23:13:01 2013 -0500

----------------------------------------------------------------------
 .../core/client/admin/TableOperationsImpl.java  |  4 +-
 .../core/client/impl/OfflineScanner.java        |  8 +--
 .../org/apache/accumulo/core/file/FileUtil.java |  7 +++
 .../apache/accumulo/core/zookeeper/ZooUtil.java |  4 +-
 .../apache/accumulo/server/fs/ViewFSUtils.java  | 51 ++++++++++++++++++
 .../accumulo/server/fs/VolumeManagerImpl.java   | 18 ++++++-
 .../server/master/recovery/HadoopLogCloser.java | 14 ++++-
 .../server/master/recovery/RecoveryPath.java    | 56 ++++++++++++++++++++
 .../master/recovery/RecoveryManager.java        | 14 +++--
 .../apache/accumulo/tserver/TabletServer.java   |  5 +-
 10 files changed, 158 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/322ee056/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java
----------------------------------------------------------------------


[4/5] git commit: ACCUMULO-2050 made it possible to recover logs on viewfs

Posted by kt...@apache.org.
ACCUMULO-2050 made it possible to recover logs on viewfs


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

Branch: refs/heads/master
Commit: b671fdaa7d8afcff202abe471a203230ba17d176
Parents: cddf04f
Author: Keith Turner <kt...@apache.org>
Authored: Thu Dec 19 21:26:59 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Dec 19 23:00:02 2013 -0500

----------------------------------------------------------------------
 .../apache/accumulo/server/fs/ViewFSUtils.java  | 51 ++++++++++++++++++++
 .../server/master/recovery/HadoopLogCloser.java | 14 +++++-
 2 files changed, 64 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/b671fdaa/server/base/src/main/java/org/apache/accumulo/server/fs/ViewFSUtils.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/ViewFSUtils.java b/server/base/src/main/java/org/apache/accumulo/server/fs/ViewFSUtils.java
new file mode 100644
index 0000000..ae7a8ae
--- /dev/null
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/ViewFSUtils.java
@@ -0,0 +1,51 @@
+/*
+ * 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.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * 
+ */
+public class ViewFSUtils {
+  public static boolean isViewFS(FileSystem fs) {
+    return fs.getClass().getName().equals("org.apache.hadoop.fs.viewfs.ViewFileSystem");
+  }
+
+  public static Path resolvePath(FileSystem fs, Path path) throws IOException {
+    // resolve path is new hadoop 2 so call it via reflection
+    try {
+      Method method = fs.getClass().getMethod("resolvePath", Path.class);
+      return (Path) method.invoke(fs, path);
+    } catch (IllegalArgumentException e) {
+      throw new IOException(e);
+    } catch (IllegalAccessException e) {
+      throw new IOException(e);
+    } catch (InvocationTargetException e) {
+      throw new IOException(e);
+    } catch (SecurityException e) {
+      throw new IOException(e);
+    } catch (NoSuchMethodException e) {
+      throw new IOException(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/accumulo/blob/b671fdaa/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
index b3c0934..0006bf9 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/recovery/HadoopLogCloser.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 
 import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
+import org.apache.accumulo.core.util.CachedConfiguration;
+import org.apache.accumulo.server.fs.ViewFSUtils;
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
@@ -35,6 +37,16 @@ public class HadoopLogCloser implements LogCloser {
   @Override
   public long close(AccumuloConfiguration conf, VolumeManager fs, Path source) throws IOException {
     FileSystem ns = fs.getFileSystemByPath(source);
+
+    // if path points to a viewfs path, then resolve to underlying filesystem
+    if (ViewFSUtils.isViewFS(ns)) {
+      Path newSource = ViewFSUtils.resolvePath(ns, source);
+      if (!newSource.equals(source) && newSource.toUri().getScheme() != null) {
+        ns = newSource.getFileSystem(CachedConfiguration.getInstance());
+        source = newSource;
+      }
+    }
+
     if (ns instanceof DistributedFileSystem) {
       DistributedFileSystem dfs = (DistributedFileSystem) ns;
       try {
@@ -53,7 +65,7 @@ public class HadoopLogCloser implements LogCloser {
     } else if (ns instanceof LocalFileSystem) {
       // ignore
     } else {
-      throw new IllegalStateException("Don't know how to recover a lease for " + fs.getClass().getName());
+      throw new IllegalStateException("Don't know how to recover a lease for " + ns.getClass().getName());
     }
     return 0;
   }


[2/5] git commit: ACCUMULO-2051 use new getDefaultReplication(Path) if available

Posted by kt...@apache.org.
ACCUMULO-2051 use new getDefaultReplication(Path) if available


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

Branch: refs/heads/master
Commit: 0d3442831650bd6c70625660e56fc43006ca36e2
Parents: b671fda
Author: Keith Turner <kt...@apache.org>
Authored: Thu Dec 19 21:44:10 2013 -0500
Committer: Keith Turner <kt...@apache.org>
Committed: Thu Dec 19 23:00:02 2013 -0500

----------------------------------------------------------------------
 .../accumulo/server/fs/VolumeManagerImpl.java     | 18 +++++++++++++++++-
 1 file changed, 17 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/accumulo/blob/0d344283/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
----------------------------------------------------------------------
diff --git a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
index 472b0c0..a89572d 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/fs/VolumeManagerImpl.java
@@ -19,6 +19,7 @@ package org.apache.accumulo.server.fs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.net.URI;
 import java.util.ArrayList;
@@ -325,8 +326,23 @@ public class VolumeManagerImpl implements VolumeManager {
 
   @Override
   public short getDefaultReplication(Path path) {
+    FileSystem fs = getFileSystemByPath(path);
+    try {
+      // try calling hadoop 2 method
+      Method method = fs.getClass().getMethod("getDefaultReplication", Path.class);
+      return ((Short) method.invoke(fs, path)).shortValue();
+    } catch (NoSuchMethodException e) {
+      // ignore
+    } catch (IllegalArgumentException e) {
+      throw new RuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e);
+    } catch (InvocationTargetException e) {
+      throw new RuntimeException(e);
+    }
+
     @SuppressWarnings("deprecation")
-    short rep = getFileSystemByPath(path).getDefaultReplication();
+    short rep = fs.getDefaultReplication();
     return rep;
   }