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.regionserver.storefiletracker;
019
020import java.io.IOException;
021import java.util.function.BiConsumer;
022import java.util.function.Consumer;
023import org.apache.hadoop.conf.Configuration;
024import org.apache.hadoop.hbase.DoNotRetryIOException;
025import org.apache.hadoop.hbase.HBaseIOException;
026import org.apache.hadoop.hbase.TableName;
027import org.apache.hadoop.hbase.TableNotEnabledException;
028import org.apache.hadoop.hbase.TableNotFoundException;
029import org.apache.hadoop.hbase.client.TableDescriptor;
030import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineTableProcedure;
031import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
032import org.apache.hadoop.hbase.master.procedure.ModifyTableProcedure;
033import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
034import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
035import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
036import org.apache.yetus.audience.InterfaceAudience;
037import org.slf4j.Logger;
038import org.slf4j.LoggerFactory;
039
040import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
041import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyStoreFileTrackerState;
042import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.ModifyStoreFileTrackerStateData;
043
044/**
045 * This procedure is used to change the store file tracker implementation.
046 * <p/>
047 * Typically we need to schedule two {@link ModifyTableProcedure} (or three if the table is already
048 * in {@code MIGRATION} but the {@code dstSFT} is not what we expected) to do this, so we introduce
049 * this procedure to simplify the work of our users.
050 */
051@InterfaceAudience.Private
052public abstract class ModifyStoreFileTrackerProcedure
053  extends AbstractStateMachineTableProcedure<ModifyStoreFileTrackerState> {
054
055  private static final Logger LOG = LoggerFactory.getLogger(ModifyStoreFileTrackerProcedure.class);
056
057  private TableName tableName;
058
059  private String dstSFT;
060
061  protected ModifyStoreFileTrackerProcedure() {
062  }
063
064  protected ModifyStoreFileTrackerProcedure(MasterProcedureEnv env, TableName tableName,
065    String dstSFT) throws HBaseIOException {
066    super(env);
067    checkDstSFT(dstSFT);
068    this.tableName = tableName;
069    this.dstSFT = dstSFT;
070    preflightChecks(env, true);
071  }
072
073  private void checkDstSFT(String dstSFT) throws DoNotRetryIOException {
074    if (
075      MigrationStoreFileTracker.class
076        .isAssignableFrom(StoreFileTrackerFactory.getTrackerClass(dstSFT))
077    ) {
078      throw new DoNotRetryIOException("Do not need to transfer to " + dstSFT);
079    }
080  }
081
082  @Override
083  public TableName getTableName() {
084    return tableName;
085  }
086
087  @Override
088  public TableOperationType getTableOperationType() {
089    return TableOperationType.EDIT;
090  }
091
092  private enum StoreFileTrackerState {
093    NEED_FINISH_PREVIOUS_MIGRATION_FIRST,
094    NEED_START_MIGRATION,
095    NEED_FINISH_MIGRATION,
096    ALREADY_FINISHED
097  }
098
099  private StoreFileTrackerState checkState(Configuration conf, String dstSFT) {
100    // there are 4 possible conditions:
101    // 1. The table or family has already made use of the dstSFT. In this way we just finish the
102    // procedure.
103    // 2. The table or family is not using the dstSFT but also not using migration SFT,
104    // then we just go to the MODIFY_STORE_FILE_TRACKER_MIGRATION state.
105    // 3. The table or family has already been using migration SFT and the dst SFT is what we
106    // expect, just go to MODIFY_STORE_FILE_TRACKER_FINISH.
107    // 4. The table or family is using migration SFT and the dst SFT is not what we
108    // expect, then need to schedule a MTP to change it to the dst SFT of the current migration
109    // SFT first, and then go to MODIFY_STORE_FILE_TRACKER_MIGRATION.
110    Class<? extends StoreFileTracker> clazz = StoreFileTrackerFactory.getTrackerClass(conf);
111    Class<? extends StoreFileTracker> dstSFTClass = StoreFileTrackerFactory.getTrackerClass(dstSFT);
112    if (clazz.equals(dstSFTClass)) {
113      return StoreFileTrackerState.ALREADY_FINISHED;
114    }
115    if (!MigrationStoreFileTracker.class.isAssignableFrom(clazz)) {
116      return StoreFileTrackerState.NEED_START_MIGRATION;
117    }
118    Class<? extends StoreFileTracker> currentDstSFT = StoreFileTrackerFactory
119      .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL);
120    if (currentDstSFT.equals(dstSFTClass)) {
121      return StoreFileTrackerState.NEED_FINISH_MIGRATION;
122    } else {
123      return StoreFileTrackerState.NEED_FINISH_PREVIOUS_MIGRATION_FIRST;
124    }
125  }
126
127  private final String getRestoreSFT(Configuration conf) {
128    Class<? extends StoreFileTracker> currentDstSFT = StoreFileTrackerFactory
129      .getStoreFileTrackerClassForMigration(conf, MigrationStoreFileTracker.DST_IMPL);
130    return StoreFileTrackerFactory.getStoreFileTrackerName(currentDstSFT);
131  }
132
133  protected abstract void preCheck(TableDescriptor current) throws IOException;
134
135  protected abstract Configuration createConf(Configuration conf, TableDescriptor current);
136
137  protected abstract TableDescriptor createRestoreTableDescriptor(TableDescriptor current,
138    String restoreSFT);
139
140  private Flow preCheckAndTryRestoreSFT(MasterProcedureEnv env) throws IOException {
141    // Checks whether the table exists
142    if (!env.getMasterServices().getTableDescriptors().exists(getTableName())) {
143      throw new TableNotFoundException(getTableName());
144    }
145    if (!isTableEnabled(env)) {
146      throw new TableNotEnabledException(tableName);
147    }
148    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
149    preCheck(current);
150    Configuration conf = createConf(env.getMasterConfiguration(), current);
151    StoreFileTrackerState state = checkState(conf, dstSFT);
152    switch (state) {
153      case NEED_FINISH_PREVIOUS_MIGRATION_FIRST:
154        TableDescriptor td = createRestoreTableDescriptor(current, getRestoreSFT(conf));
155        addChildProcedure(new ModifyTableProcedure(env, td));
156        setNextState(
157          ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION);
158        return Flow.HAS_MORE_STATE;
159      case NEED_START_MIGRATION:
160        setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_START_MIGRATION);
161        return Flow.HAS_MORE_STATE;
162      case NEED_FINISH_MIGRATION:
163        setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION);
164        return Flow.HAS_MORE_STATE;
165      case ALREADY_FINISHED:
166        return Flow.NO_MORE_STATE;
167      default:
168        throw new UnsupportedOperationException("unhandled state=" + state);
169    }
170  }
171
172  protected abstract TableDescriptor createMigrationTableDescriptor(Configuration conf,
173    TableDescriptor current);
174
175  protected final void migrate(Configuration conf, BiConsumer<String, String> setValue) {
176    setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL,
177      StoreFileTrackerFactory.Trackers.MIGRATION.name());
178    setValue.accept(MigrationStoreFileTracker.SRC_IMPL,
179      StoreFileTrackerFactory.getStoreFileTrackerName(conf));
180    setValue.accept(MigrationStoreFileTracker.DST_IMPL, dstSFT);
181  }
182
183  protected abstract TableDescriptor createFinishTableDescriptor(TableDescriptor current);
184
185  protected final void finish(BiConsumer<String, String> setValue, Consumer<String> removeValue) {
186    setValue.accept(StoreFileTrackerFactory.TRACKER_IMPL, dstSFT);
187    removeValue.accept(MigrationStoreFileTracker.SRC_IMPL);
188    removeValue.accept(MigrationStoreFileTracker.DST_IMPL);
189  }
190
191  private void migrate(MasterProcedureEnv env) throws IOException {
192    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
193    TableDescriptor td = createMigrationTableDescriptor(env.getMasterConfiguration(), current);
194    addChildProcedure(new ModifyTableProcedure(env, td));
195    setNextState(ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION);
196  }
197
198  private void finish(MasterProcedureEnv env) throws IOException {
199    TableDescriptor current = env.getMasterServices().getTableDescriptors().get(tableName);
200    TableDescriptor td = createFinishTableDescriptor(current);
201    addChildProcedure(new ModifyTableProcedure(env, td));
202  }
203
204  @Override
205  protected Flow executeFromState(MasterProcedureEnv env, ModifyStoreFileTrackerState state)
206    throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
207    try {
208      switch (state) {
209        case MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION:
210          return preCheckAndTryRestoreSFT(env);
211        case MODIFY_STORE_FILE_TRACKER_START_MIGRATION:
212          migrate(env);
213          return Flow.HAS_MORE_STATE;
214        case MODIFY_STORE_FILE_TRACKER_FINISH_MIGRATION:
215          finish(env);
216          return Flow.NO_MORE_STATE;
217        default:
218          throw new UnsupportedOperationException("unhandled state=" + state);
219      }
220    } catch (IOException e) {
221      if (isRollbackSupported(state)) {
222        setFailure("master-modify-SFT", e);
223      } else {
224        LOG.warn("Retriable error trying to modify SFT for table={} (in state={})", getTableName(),
225          state, e);
226      }
227    }
228    return Flow.HAS_MORE_STATE;
229  }
230
231  @Override
232  protected void rollbackState(MasterProcedureEnv env, ModifyStoreFileTrackerState state)
233    throws IOException, InterruptedException {
234    if (isRollbackSupported(state)) {
235      return;
236    }
237    throw new UnsupportedOperationException("unhandled state=" + state);
238  }
239
240  @Override
241  protected ModifyStoreFileTrackerState getState(int stateId) {
242    return ModifyStoreFileTrackerState.forNumber(stateId);
243  }
244
245  @Override
246  protected int getStateId(ModifyStoreFileTrackerState state) {
247    return state.getNumber();
248  }
249
250  @Override
251  protected ModifyStoreFileTrackerState getInitialState() {
252    return ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION;
253  }
254
255  @Override
256  protected boolean isRollbackSupported(ModifyStoreFileTrackerState state) {
257    return state == ModifyStoreFileTrackerState.MODIFY_STORE_FILE_TRACKER_FINISH_PREVIOUS_MIGRATION;
258  }
259
260  @Override
261  protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
262    super.serializeStateData(serializer);
263    serializer.serialize(ModifyStoreFileTrackerStateData.newBuilder()
264      .setTableName(ProtobufUtil.toProtoTableName(tableName)).setDstSft(dstSFT).build());
265  }
266
267  @Override
268  protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
269    super.deserializeStateData(serializer);
270    ModifyStoreFileTrackerStateData data =
271      serializer.deserialize(ModifyStoreFileTrackerStateData.class);
272    this.tableName = ProtobufUtil.toTableName(data.getTableName());
273    this.dstSFT = data.getDstSft();
274  }
275}