Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,86 @@
/*
* 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.hbase;

import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* Helper class for supporting different versions of HBase for creating
* {@link ReplicationQueueStorage} and {@link ReplicationPeerStorage}.
*/
public class ReplicationStorageFactoryHelper {

private static final Logger LOG = LoggerFactory.getLogger(ReplicationStorageFactoryHelper.class);

public static ReplicationPeerStorage getReplicationPeerStorage(Configuration conf, ZKWatcher zkw,
FileSystem fs) {
// Case HBase >= 2.6.0: Invoke the method that requires three parameters
try {
Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage",
FileSystem.class, ZKWatcher.class, Configuration.class);
return (ReplicationPeerStorage) method.invoke(null, fs, zkw, conf);
} catch (NoSuchMethodException e) {
LOG.debug("No getReplicationPeerStorage method with FileSystem as a parameter, "
+ "should be HBase 2.6-", e);
} catch (IllegalAccessException | InvocationTargetException e) {
// getReplicationPeerStorage method does not throw any exceptions, so should not arrive here
throw new RuntimeException(e);
}
// Case HBase < 2.6.0: Fall back to the method that requires only two parameters
try {
Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage",
ZKWatcher.class, Configuration.class);
return (ReplicationPeerStorage) method.invoke(null, zkw, conf);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
throw new RuntimeException(e);
}
}

public static ReplicationQueueStorage getReplicationQueueStorage(Configuration conf,
ZKWatcher zkw, Connection conn) {
try {
Method method = ReplicationStorageFactory.class.getMethod("getReplicationQueueStorage",
Connection.class, Configuration.class);
return (ReplicationQueueStorage) method.invoke(null, conn, conf);
} catch (NoSuchMethodException e) {
LOG.debug("No getReplicationQueueStorage method with Connection as a parameter, "
+ "should be HBase 2.x", e);
} catch (IllegalAccessException | InvocationTargetException e) {
// getReplicationQueueStorage method does not throw any exceptions, so should not arrive here
throw new RuntimeException(e);
}
try {
Method method = ReplicationStorageFactory.class.getMethod("getReplicationQueueStorage",
ZKWatcher.class, Configuration.class);
return (ReplicationQueueStorage) method.invoke(null, zkw, conf);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
// getReplicationQueueStorage method does not throw any exceptions, so should not arrive here
throw new RuntimeException(e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,6 @@
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Bytes.ByteArrayComparator;
Expand Down Expand Up @@ -155,6 +154,7 @@
import org.apache.hadoop.util.Tool;
import org.apache.hbase.HBCKFsUtils;
import org.apache.hbase.HBCKMetaTableAccessor;
import org.apache.hbase.ReplicationStorageFactoryHelper;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.apache.zookeeper.KeeperException;
Expand Down Expand Up @@ -3841,7 +3841,7 @@ private synchronized HbckInfo getOrCreateInfo(String name) {
}

public void checkAndFixReplication() throws ReplicationException {
ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, errors);
ReplicationChecker checker = new ReplicationChecker(getConf(), zkw, rootFs, connection, errors);
checker.checkUnDeletedQueues();

if (checker.hasUnDeletedQueues() && this.fixReplication) {
Expand Down Expand Up @@ -5475,7 +5475,7 @@ public void cleanReplicationBarrier() throws IOException {
return;
}
ReplicationQueueStorage queueStorage =
ReplicationStorageFactory.getReplicationQueueStorage(zkw, getConf());
ReplicationStorageFactoryHelper.getReplicationQueueStorage(getConf(), zkw, connection);
List<ReplicationPeerDescription> peerDescriptions = admin.listReplicationPeers();
if (peerDescriptions != null && peerDescriptions.size() > 0) {
List<String> peers = peerDescriptions.stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,9 +17,6 @@
*/
package org.apache.hbase.hbck1;

import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
Expand All @@ -29,12 +26,13 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.replication.ReplicationException;
import org.apache.hadoop.hbase.replication.ReplicationPeerStorage;
import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
import org.apache.hadoop.hbase.replication.ReplicationQueueStorage;
import org.apache.hadoop.hbase.replication.ReplicationStorageFactory;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.hbase.ReplicationStorageFactoryHelper;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -57,36 +55,13 @@ public class ReplicationChecker {
private final ReplicationPeerStorage peerStorage;
private final ReplicationQueueStorage queueStorage;

public ReplicationChecker(Configuration conf, ZKWatcher zkw,
public ReplicationChecker(Configuration conf, ZKWatcher zkw, FileSystem fs, Connection conn,
HBaseFsck.ErrorReporter errorReporter) {
this.peerStorage = getReplicationPeerStorage(conf, zkw);
this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf);
this.peerStorage = ReplicationStorageFactoryHelper.getReplicationPeerStorage(conf, zkw, fs);
this.queueStorage = ReplicationStorageFactoryHelper.getReplicationQueueStorage(conf, zkw, conn);
this.errorReporter = errorReporter;
}

private ReplicationPeerStorage getReplicationPeerStorage(Configuration conf, ZKWatcher zkw)
throws AssertionError {
ReplicationPeerStorage peerStorage;
try {
// Case HBase >= 2.6.0: Invoke the method that requires three parameters
Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage",
FileSystem.class, ZKWatcher.class, Configuration.class);
FileSystem fileSystem = FileSystem.get(conf);
peerStorage = (ReplicationPeerStorage) method.invoke(null, fileSystem, zkw, conf);
} catch (IOException | NoSuchMethodException | IllegalAccessException
| InvocationTargetException e1) {
// Case HBase < 2.6.0: Fall back to the method that requires only two parameters
try {
Method method = ReplicationStorageFactory.class.getMethod("getReplicationPeerStorage",
ZKWatcher.class, Configuration.class);
peerStorage = (ReplicationPeerStorage) method.invoke(null, zkw, conf);
} catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e2) {
throw new AssertionError("should not happen", e2);
}
}
return peerStorage;
}

public boolean hasUnDeletedQueues() {
return errorReporter.getErrorList()
.contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE);
Expand Down