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.master.assignment; 019 020import java.io.IOException; 021import org.apache.hadoop.fs.FileSystem; 022import org.apache.hadoop.fs.Path; 023import org.apache.hadoop.hbase.MetaTableAccessor; 024import org.apache.hadoop.hbase.backup.HFileArchiver; 025import org.apache.hadoop.hbase.client.RegionInfo; 026import org.apache.hadoop.hbase.favored.FavoredNodesManager; 027import org.apache.hadoop.hbase.master.MasterFileSystem; 028import org.apache.hadoop.hbase.master.MasterServices; 029import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure; 030import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; 031import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer; 032import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException; 033import org.apache.hadoop.hbase.procedure2.ProcedureYieldException; 034import org.apache.hadoop.hbase.util.FSUtils; 035import org.apache.yetus.audience.InterfaceAudience; 036import org.slf4j.Logger; 037import org.slf4j.LoggerFactory; 038 039import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 040 041import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 042import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos; 043import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.GCRegionState; 044 045/** 046 * GC a Region that is no longer in use. It has been split or merged away. 047 * Caller determines if it is GC time. This Procedure does not check. 048 * <p>This is a Region StateMachine Procedure. We take a read lock on the Table and then 049 * exclusive on the Region. 050 */ 051@InterfaceAudience.Private 052public class GCRegionProcedure extends AbstractStateMachineRegionProcedure<GCRegionState> { 053 private static final Logger LOG = LoggerFactory.getLogger(GCRegionProcedure.class); 054 055 public GCRegionProcedure(final MasterProcedureEnv env, final RegionInfo hri) { 056 super(env, hri); 057 } 058 059 public GCRegionProcedure() { 060 // Required by the Procedure framework to create the procedure on replay 061 super(); 062 } 063 064 @Override 065 public TableOperationType getTableOperationType() { 066 return TableOperationType.REGION_GC; 067 } 068 069 @Override 070 protected Flow executeFromState(MasterProcedureEnv env, GCRegionState state) 071 throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException { 072 if (LOG.isTraceEnabled()) { 073 LOG.trace(this + " execute state=" + state); 074 } 075 MasterServices masterServices = env.getMasterServices(); 076 try { 077 switch (state) { 078 case GC_REGION_PREPARE: 079 // Nothing to do to prepare. 080 setNextState(GCRegionState.GC_REGION_ARCHIVE); 081 break; 082 case GC_REGION_ARCHIVE: 083 MasterFileSystem mfs = masterServices.getMasterFileSystem(); 084 FileSystem fs = mfs.getFileSystem(); 085 if (HFileArchiver.exists(masterServices.getConfiguration(), fs, getRegion())) { 086 if (LOG.isDebugEnabled()) { 087 LOG.debug("Archiving region=" + getRegion().getShortNameToLog()); 088 } 089 HFileArchiver.archiveRegion(masterServices.getConfiguration(), fs, getRegion()); 090 } 091 FileSystem walFs = mfs.getWALFileSystem(); 092 // Cleanup the directories on WAL filesystem also 093 Path regionWALDir = FSUtils.getWALRegionDir(env.getMasterConfiguration(), 094 getRegion().getTable(), getRegion().getEncodedName()); 095 if (walFs.exists(regionWALDir)) { 096 if (!walFs.delete(regionWALDir, true)) { 097 LOG.debug("Failed to delete {}", regionWALDir); 098 } 099 } 100 Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(env.getMasterConfiguration(), 101 getRegion().getTable(), getRegion().getEncodedName()); 102 if (walFs.exists(wrongRegionWALDir)) { 103 if (!walFs.delete(wrongRegionWALDir, true)) { 104 LOG.debug("Failed to delete {}", regionWALDir); 105 } 106 } 107 setNextState(GCRegionState.GC_REGION_PURGE_METADATA); 108 break; 109 case GC_REGION_PURGE_METADATA: 110 // TODO: Purge metadata before removing from HDFS? This ordering is copied 111 // from CatalogJanitor. 112 AssignmentManager am = masterServices.getAssignmentManager(); 113 if (am != null) { 114 if (am.getRegionStates() != null) { 115 am.getRegionStates().deleteRegion(getRegion()); 116 } 117 } 118 MetaTableAccessor.deleteRegionInfo(masterServices.getConnection(), getRegion()); 119 masterServices.getServerManager().removeRegion(getRegion()); 120 FavoredNodesManager fnm = masterServices.getFavoredNodesManager(); 121 if (fnm != null) { 122 fnm.deleteFavoredNodesForRegions(Lists.newArrayList(getRegion())); 123 } 124 return Flow.NO_MORE_STATE; 125 default: 126 throw new UnsupportedOperationException(this + " unhandled state=" + state); 127 } 128 } catch (IOException ioe) { 129 // TODO: This is going to spew log? Add retry backoff 130 LOG.warn("Error trying to GC " + getRegion().getShortNameToLog() + "; retrying...", ioe); 131 } 132 return Flow.HAS_MORE_STATE; 133 } 134 135 @Override 136 protected void rollbackState(MasterProcedureEnv env, GCRegionState state) throws IOException, InterruptedException { 137 // no-op 138 } 139 140 @Override 141 protected GCRegionState getState(int stateId) { 142 return GCRegionState.forNumber(stateId); 143 } 144 145 @Override 146 protected int getStateId(GCRegionState state) { 147 return state.getNumber(); 148 } 149 150 @Override 151 protected GCRegionState getInitialState() { 152 return GCRegionState.GC_REGION_PREPARE; 153 } 154 155 @Override 156 protected void serializeStateData(ProcedureStateSerializer serializer) 157 throws IOException { 158 super.serializeStateData(serializer); 159 // Double serialization of regionname. Superclass is also serializing. Fix. 160 final MasterProcedureProtos.GCRegionStateData.Builder msg = 161 MasterProcedureProtos.GCRegionStateData.newBuilder() 162 .setRegionInfo(ProtobufUtil.toRegionInfo(getRegion())); 163 serializer.serialize(msg.build()); 164 } 165 166 @Override 167 protected void deserializeStateData(ProcedureStateSerializer serializer) 168 throws IOException { 169 super.deserializeStateData(serializer); 170 final MasterProcedureProtos.GCRegionStateData msg = 171 serializer.deserialize(MasterProcedureProtos.GCRegionStateData.class); 172 setRegion(ProtobufUtil.toRegionInfo(msg.getRegionInfo())); 173 } 174}