Skip to content

Commit 557fd6e

Browse files
authored
HBASE-28375 HBase Operator Tools fails to compile with hbase 2.6.0 (#140)
Signed-off-by: Duo Zhang <[email protected]>
1 parent 4d4c928 commit 557fd6e

File tree

1 file changed

+28
-1
lines changed

1 file changed

+28
-1
lines changed

hbase-hbck2/src/main/java/org/apache/hbase/hbck1/ReplicationChecker.java

+28-1
Original file line numberDiff line numberDiff line change
@@ -17,13 +17,17 @@
1717
*/
1818
package org.apache.hbase.hbck1;
1919

20+
import java.io.IOException;
21+
import java.lang.reflect.InvocationTargetException;
22+
import java.lang.reflect.Method;
2023
import java.util.ArrayList;
2124
import java.util.HashMap;
2225
import java.util.HashSet;
2326
import java.util.List;
2427
import java.util.Map;
2528
import java.util.Set;
2629
import org.apache.hadoop.conf.Configuration;
30+
import org.apache.hadoop.fs.FileSystem;
2731
import org.apache.hadoop.hbase.ServerName;
2832
import org.apache.hadoop.hbase.replication.ReplicationException;
2933
import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
@@ -55,11 +59,34 @@ public class ReplicationChecker {
5559

5660
public ReplicationChecker(Configuration conf, ZKWatcher zkw,
5761
HBaseFsck.ErrorReporter errorReporter) {
58-
this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf);
62+
this.peerStorage = getReplicationPeerStorage(conf, zkw);
5963
this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
6064
this.errorReporter = errorReporter;
6165
}
6266

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+
6390
public boolean hasUnDeletedQueues() {
6491
return errorReporter.getErrorList()
6592
.contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE);

0 commit comments

Comments
 (0)