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}