View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.master.procedure;
20  
21  import java.io.InputStream;
22  import java.io.IOException;
23  import java.io.OutputStream;
24  import java.security.PrivilegedExceptionAction;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.List;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.hbase.TableName;
33  import org.apache.hadoop.hbase.TableNotDisabledException;
34  import org.apache.hadoop.hbase.TableNotFoundException;
35  import org.apache.hadoop.hbase.HRegionInfo;
36  import org.apache.hadoop.hbase.HTableDescriptor;
37  import org.apache.hadoop.hbase.exceptions.HBaseException;
38  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
39  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
40  import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
41  import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.TruncateTableState;
42  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
43  import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
44  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
45  import org.apache.hadoop.security.UserGroupInformation;
46  
47  @InterfaceAudience.Private
48  public class TruncateTableProcedure
49      extends StateMachineProcedure<MasterProcedureEnv, TruncateTableState>
50      implements TableProcedureInterface {
51    private static final Log LOG = LogFactory.getLog(TruncateTableProcedure.class);
52  
53    private boolean preserveSplits;
54    private List<HRegionInfo> regions;
55    private UserGroupInformation user;
56    private HTableDescriptor hTableDescriptor;
57    private TableName tableName;
58  
59    public TruncateTableProcedure() {
60      // Required by the Procedure framework to create the procedure on replay
61    }
62  
63    public TruncateTableProcedure(final MasterProcedureEnv env, final TableName tableName,
64        boolean preserveSplits) {
65      this.tableName = tableName;
66      this.preserveSplits = preserveSplits;
67      this.user = env.getRequestUser().getUGI();
68      this.setOwner(this.user.getShortUserName());
69    }
70  
71    @Override
72    protected Flow executeFromState(final MasterProcedureEnv env, TruncateTableState state)
73        throws InterruptedException {
74      if (LOG.isTraceEnabled()) {
75        LOG.trace(this + " execute state=" + state);
76      }
77      try {
78        switch (state) {
79          case TRUNCATE_TABLE_PRE_OPERATION:
80            // Verify if we can truncate the table
81            if (!prepareTruncate(env)) {
82              assert isFailed() : "the truncate should have an exception here";
83              return Flow.NO_MORE_STATE;
84            }
85  
86            // TODO: Move out... in the acquireLock()
87            LOG.debug("waiting for '" + getTableName() + "' regions in transition");
88            regions = ProcedureSyncWait.getRegionsFromMeta(env, getTableName());
89            assert regions != null && !regions.isEmpty() : "unexpected 0 regions";
90            ProcedureSyncWait.waitRegionInTransition(env, regions);
91  
92            // Call coprocessors
93            preTruncate(env);
94  
95            setNextState(TruncateTableState.TRUNCATE_TABLE_REMOVE_FROM_META);
96            break;
97          case TRUNCATE_TABLE_REMOVE_FROM_META:
98            hTableDescriptor = env.getMasterServices().getTableDescriptors().get(tableName);
99            DeleteTableProcedure.deleteFromMeta(env, getTableName(), regions);
100           DeleteTableProcedure.deleteAssignmentState(env, getTableName());
101           setNextState(TruncateTableState.TRUNCATE_TABLE_CLEAR_FS_LAYOUT);
102           break;
103         case TRUNCATE_TABLE_CLEAR_FS_LAYOUT:
104           DeleteTableProcedure.deleteFromFs(env, getTableName(), regions, true);
105           if (!preserveSplits) {
106             // if we are not preserving splits, generate a new single region
107             regions = Arrays.asList(ModifyRegionUtils.createHRegionInfos(hTableDescriptor, null));
108           } else {
109             regions = recreateRegionInfo(regions);
110           }
111           setNextState(TruncateTableState.TRUNCATE_TABLE_CREATE_FS_LAYOUT);
112           break;
113         case TRUNCATE_TABLE_CREATE_FS_LAYOUT:
114           regions = CreateTableProcedure.createFsLayout(env, hTableDescriptor, regions);
115           CreateTableProcedure.updateTableDescCache(env, getTableName());
116           setNextState(TruncateTableState.TRUNCATE_TABLE_ADD_TO_META);
117           break;
118         case TRUNCATE_TABLE_ADD_TO_META:
119           regions = CreateTableProcedure.addTableToMeta(env, hTableDescriptor, regions);
120           setNextState(TruncateTableState.TRUNCATE_TABLE_ASSIGN_REGIONS);
121           break;
122         case TRUNCATE_TABLE_ASSIGN_REGIONS:
123           CreateTableProcedure.assignRegions(env, getTableName(), regions);
124           setNextState(TruncateTableState.TRUNCATE_TABLE_POST_OPERATION);
125           hTableDescriptor = null;
126           regions = null;
127           break;
128         case TRUNCATE_TABLE_POST_OPERATION:
129           postTruncate(env);
130           LOG.debug("truncate '" + getTableName() + "' completed");
131           return Flow.NO_MORE_STATE;
132         default:
133           throw new UnsupportedOperationException("unhandled state=" + state);
134       }
135     } catch (HBaseException|IOException e) {
136       LOG.warn("Retriable error trying to truncate table=" + getTableName() + " state=" + state, e);
137     }
138     return Flow.HAS_MORE_STATE;
139   }
140 
141   @Override
142   protected void rollbackState(final MasterProcedureEnv env, final TruncateTableState state) {
143     if (state == TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION) {
144       // nothing to rollback, pre-truncate is just table-state checks.
145       // We can fail if the table does not exist or is not disabled.
146       return;
147     }
148 
149     // The truncate doesn't have a rollback. The execution will succeed, at some point.
150     throw new UnsupportedOperationException("unhandled state=" + state);
151   }
152 
153   @Override
154   protected TruncateTableState getState(final int stateId) {
155     return TruncateTableState.valueOf(stateId);
156   }
157 
158   @Override
159   protected int getStateId(final TruncateTableState state) {
160     return state.getNumber();
161   }
162 
163   @Override
164   protected TruncateTableState getInitialState() {
165     return TruncateTableState.TRUNCATE_TABLE_PRE_OPERATION;
166   }
167 
168   @Override
169   public TableName getTableName() {
170     return tableName;
171   }
172 
173   @Override
174   public TableOperationType getTableOperationType() {
175     return TableOperationType.EDIT;
176   }
177 
178   @Override
179   public boolean abort(final MasterProcedureEnv env) {
180     // TODO: We may be able to abort if the procedure is not started yet.
181     return false;
182   }
183 
184   @Override
185   protected boolean acquireLock(final MasterProcedureEnv env) {
186     if (env.waitInitialized(this)) return false;
187     return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
188   }
189 
190   @Override
191   protected void releaseLock(final MasterProcedureEnv env) {
192     env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
193   }
194 
195   @Override
196   public void toStringClassDetails(StringBuilder sb) {
197     sb.append(getClass().getSimpleName());
198     sb.append(" (table=");
199     sb.append(getTableName());
200     sb.append(" preserveSplits=");
201     sb.append(preserveSplits);
202     sb.append(")");
203   }
204 
205   @Override
206   public void serializeStateData(final OutputStream stream) throws IOException {
207     super.serializeStateData(stream);
208 
209     MasterProcedureProtos.TruncateTableStateData.Builder state =
210       MasterProcedureProtos.TruncateTableStateData.newBuilder()
211         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
212         .setPreserveSplits(preserveSplits);
213     if (hTableDescriptor != null) {
214       state.setTableSchema(hTableDescriptor.convert());
215     } else {
216       state.setTableName(ProtobufUtil.toProtoTableName(tableName));
217     }
218     if (regions != null) {
219       for (HRegionInfo hri: regions) {
220         state.addRegionInfo(HRegionInfo.convert(hri));
221       }
222     }
223     state.build().writeDelimitedTo(stream);
224   }
225 
226   @Override
227   public void deserializeStateData(final InputStream stream) throws IOException {
228     super.deserializeStateData(stream);
229 
230     MasterProcedureProtos.TruncateTableStateData state =
231       MasterProcedureProtos.TruncateTableStateData.parseDelimitedFrom(stream);
232     user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
233     if (state.hasTableSchema()) {
234       hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
235       tableName = hTableDescriptor.getTableName();
236     } else {
237       tableName = ProtobufUtil.toTableName(state.getTableName());
238     }
239     preserveSplits = state.getPreserveSplits();
240     if (state.getRegionInfoCount() == 0) {
241       regions = null;
242     } else {
243       regions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
244       for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
245         regions.add(HRegionInfo.convert(hri));
246       }
247     }
248   }
249 
250   private static List<HRegionInfo> recreateRegionInfo(final List<HRegionInfo> regions) {
251     ArrayList<HRegionInfo> newRegions = new ArrayList<HRegionInfo>(regions.size());
252     for (HRegionInfo hri: regions) {
253       newRegions.add(new HRegionInfo(hri.getTable(), hri.getStartKey(), hri.getEndKey()));
254     }
255     return newRegions;
256   }
257 
258   private boolean prepareTruncate(final MasterProcedureEnv env) throws IOException {
259     try {
260       env.getMasterServices().checkTableModifiable(getTableName());
261     } catch (TableNotFoundException|TableNotDisabledException e) {
262       setFailure("master-truncate-table", e);
263       return false;
264     }
265     return true;
266   }
267 
268   private boolean preTruncate(final MasterProcedureEnv env)
269       throws IOException, InterruptedException {
270     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
271     if (cpHost != null) {
272       final TableName tableName = getTableName();
273       user.doAs(new PrivilegedExceptionAction<Void>() {
274         @Override
275         public Void run() throws Exception {
276           cpHost.preTruncateTableHandler(tableName);
277           return null;
278         }
279       });
280     }
281     return true;
282   }
283 
284   private void postTruncate(final MasterProcedureEnv env)
285       throws IOException, InterruptedException {
286     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
287     if (cpHost != null) {
288       final TableName tableName = getTableName();
289       user.doAs(new PrivilegedExceptionAction<Void>() {
290         @Override
291         public Void run() throws Exception {
292           cpHost.postTruncateTableHandler(tableName);
293           return null;
294         }
295       });
296     }
297   }
298 }