|
17 | 17 | */
|
18 | 18 | package org.apache.hbase.hbck1;
|
19 | 19 |
|
| 20 | +import java.io.IOException; |
| 21 | +import java.lang.reflect.InvocationTargetException; |
| 22 | +import java.lang.reflect.Method; |
20 | 23 | import java.util.ArrayList;
|
21 | 24 | import java.util.HashMap;
|
22 | 25 | import java.util.HashSet;
|
23 | 26 | import java.util.List;
|
24 | 27 | import java.util.Map;
|
25 | 28 | import java.util.Set;
|
26 | 29 | import org.apache.hadoop.conf.Configuration;
|
| 30 | +import org.apache.hadoop.fs.FileSystem; |
27 | 31 | import org.apache.hadoop.hbase.ServerName;
|
28 | 32 | import org.apache.hadoop.hbase.replication.ReplicationException;
|
29 | 33 | import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
|
@@ -55,11 +59,34 @@ public class ReplicationChecker {
|
55 | 59 |
|
56 | 60 | public ReplicationChecker(Configuration conf, ZKWatcher zkw,
|
57 | 61 | HBaseFsck.ErrorReporter errorReporter) {
|
58 |
| - this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf); |
| 62 | + this.peerStorage = getReplicationPeerStorage(conf, zkw); |
59 | 63 | this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
|
60 | 64 | this.errorReporter = errorReporter;
|
61 | 65 | }
|
62 | 66 |
|
| 67 | + private ReplicationPeerStorage getReplicationPeerStorage(Configuration conf, ZKWatcher zkw) |
| 68 | + throws AssertionError { |
| 69 | + ReplicationPeerStorage peerStorage; |
| 70 | + try { |
| 71 | + // Case HBase >= 2.6.0: Invoke the method that requires three parameters |
| 72 | + Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage", |
| 73 | + FileSystem.class, ZKWatcher.class, Configuration.class); |
| 74 | + FileSystem fileSystem = FileSystem.get(conf); |
| 75 | + peerStorage = (ReplicationPeerStorage) method.invoke(null, fileSystem, zkw, conf); |
| 76 | + } catch (IOException | NoSuchMethodException | IllegalAccessException |
| 77 | + | InvocationTargetException e1) { |
| 78 | + // Case HBase < 2.6.0: Fall back to the method that requires only two parameters |
| 79 | + try { |
| 80 | + Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage", |
| 81 | + ZKWatcher.class, Configuration.class); |
| 82 | + peerStorage = (ReplicationPeerStorage) method.invoke(null, zkw, conf); |
| 83 | + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e2) { |
| 84 | + throw new AssertionError("should not happen", e2); |
| 85 | + } |
| 86 | + } |
| 87 | + return peerStorage; |
| 88 | + } |
| 89 | + |
63 | 90 | public boolean hasUnDeletedQueues() {
|
64 | 91 | return errorReporter.getErrorList()
|
65 | 92 | .contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE);
|
|
0 commit comments