001/** 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.util.hbck; 019 020import java.util.ArrayList; 021import java.util.HashMap; 022import java.util.HashSet; 023import java.util.List; 024import java.util.Map; 025import java.util.Set; 026import org.apache.hadoop.conf.Configuration; 027import org.apache.hadoop.hbase.ServerName; 028import org.apache.hadoop.hbase.replication.ReplicationException; 029import org.apache.hadoop.hbase.replication.ReplicationPeerStorage; 030import org.apache.hadoop.hbase.replication.ReplicationQueueInfo; 031import org.apache.hadoop.hbase.replication.ReplicationQueueStorage; 032import org.apache.hadoop.hbase.replication.ReplicationStorageFactory; 033import org.apache.hadoop.hbase.util.HBaseFsck; 034import org.apache.hadoop.hbase.util.HBaseFsck.ErrorReporter; 035import org.apache.hadoop.hbase.zookeeper.ZKWatcher; 036import org.apache.yetus.audience.InterfaceAudience; 037import org.slf4j.Logger; 038import org.slf4j.LoggerFactory; 039 040/** 041 * Check and fix undeleted replication queues for removed peerId. 042 */ 043@InterfaceAudience.Private 044public class ReplicationChecker { 045 046 private static final Logger LOG = LoggerFactory.getLogger(ReplicationChecker.class); 047 048 private final ErrorReporter errorReporter; 049 // replicator with its queueIds for removed peers 050 private Map<ServerName, List<String>> undeletedQueueIds = new HashMap<>(); 051 // replicator with its undeleted queueIds for removed peers in hfile-refs queue 052 private Set<String> undeletedHFileRefsPeerIds = new HashSet<>(); 053 054 private final ReplicationPeerStorage peerStorage; 055 private final ReplicationQueueStorage queueStorage; 056 057 public ReplicationChecker(Configuration conf, ZKWatcher zkw, ErrorReporter errorReporter) { 058 this.peerStorage = ReplicationStorageFactory.getReplicationPeerStorage(zkw, conf); 059 this.queueStorage = ReplicationStorageFactory.getReplicationQueueStorage(zkw, conf); 060 this.errorReporter = errorReporter; 061 } 062 063 public boolean hasUnDeletedQueues() { 064 return errorReporter.getErrorList() 065 .contains(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE); 066 } 067 068 private Map<ServerName, List<String>> getUnDeletedQueues() throws ReplicationException { 069 Map<ServerName, List<String>> undeletedQueues = new HashMap<>(); 070 Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds()); 071 for (ServerName replicator : queueStorage.getListOfReplicators()) { 072 for (String queueId : queueStorage.getAllQueues(replicator)) { 073 ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); 074 if (!peerIds.contains(queueInfo.getPeerId())) { 075 undeletedQueues.computeIfAbsent(replicator, key -> new ArrayList<>()).add(queueId); 076 LOG.debug( 077 "Undeleted replication queue for removed peer found: " + 078 "[removedPeerId={}, replicator={}, queueId={}]", 079 queueInfo.getPeerId(), replicator, queueId); 080 } 081 } 082 } 083 return undeletedQueues; 084 } 085 086 private Set<String> getUndeletedHFileRefsPeers() throws ReplicationException { 087 Set<String> undeletedHFileRefsPeerIds = 088 new HashSet<>(queueStorage.getAllPeersFromHFileRefsQueue()); 089 Set<String> peerIds = new HashSet<>(peerStorage.listPeerIds()); 090 undeletedHFileRefsPeerIds.removeAll(peerIds); 091 if (LOG.isDebugEnabled()) { 092 for (String peerId : undeletedHFileRefsPeerIds) { 093 LOG.debug("Undeleted replication hfile-refs queue for removed peer {} found", peerId); 094 } 095 } 096 return undeletedHFileRefsPeerIds; 097 } 098 099 public void checkUnDeletedQueues() throws ReplicationException { 100 undeletedQueueIds = getUnDeletedQueues(); 101 undeletedQueueIds.forEach((replicator, queueIds) -> { 102 queueIds.forEach(queueId -> { 103 ReplicationQueueInfo queueInfo = new ReplicationQueueInfo(queueId); 104 String msg = "Undeleted replication queue for removed peer found: " + 105 String.format("[removedPeerId=%s, replicator=%s, queueId=%s]", queueInfo.getPeerId(), 106 replicator, queueId); 107 errorReporter.reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, 108 msg); 109 }); 110 }); 111 undeletedHFileRefsPeerIds = getUndeletedHFileRefsPeers(); 112 undeletedHFileRefsPeerIds.stream() 113 .map( 114 peerId -> "Undeleted replication hfile-refs queue for removed peer " + peerId + " found") 115 .forEach(msg -> errorReporter 116 .reportError(HBaseFsck.ErrorReporter.ERROR_CODE.UNDELETED_REPLICATION_QUEUE, msg)); 117 } 118 119 public void fixUnDeletedQueues() throws ReplicationException { 120 for (Map.Entry<ServerName, List<String>> replicatorAndQueueIds : undeletedQueueIds.entrySet()) { 121 ServerName replicator = replicatorAndQueueIds.getKey(); 122 for (String queueId : replicatorAndQueueIds.getValue()) { 123 queueStorage.removeQueue(replicator, queueId); 124 } 125 queueStorage.removeReplicatorIfQueueIsEmpty(replicator); 126 } 127 for (String peerId : undeletedHFileRefsPeerIds) { 128 queueStorage.removePeerFromHFileRefs(peerId); 129 } 130 } 131}