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.IOException;
22  import java.io.InputStream;
23  import java.io.OutputStream;
24  import java.security.PrivilegedExceptionAction;
25  import java.util.ArrayList;
26  import java.util.List;
27  import java.util.concurrent.atomic.AtomicBoolean;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.HRegionInfo;
34  import org.apache.hadoop.hbase.HTableDescriptor;
35  import org.apache.hadoop.hbase.MetaTableAccessor;
36  import org.apache.hadoop.hbase.TableExistsException;
37  import org.apache.hadoop.hbase.TableName;
38  import org.apache.hadoop.hbase.TableStateManager;
39  import org.apache.hadoop.hbase.classification.InterfaceAudience;
40  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
41  import org.apache.hadoop.hbase.exceptions.HBaseException;
42  import org.apache.hadoop.hbase.master.AssignmentManager;
43  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
44  import org.apache.hadoop.hbase.master.MasterFileSystem;
45  import org.apache.hadoop.hbase.procedure2.StateMachineProcedure;
46  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
47  import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos;
48  import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CreateTableState;
49  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
50  import org.apache.hadoop.hbase.util.FSTableDescriptors;
51  import org.apache.hadoop.hbase.util.FSUtils;
52  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
53  import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
54  import org.apache.hadoop.security.UserGroupInformation;
55  
56  import com.google.common.collect.Lists;
57  
58  @InterfaceAudience.Private
59  public class CreateTableProcedure
60      extends StateMachineProcedure<MasterProcedureEnv, CreateTableState>
61      implements TableProcedureInterface {
62    private static final Log LOG = LogFactory.getLog(CreateTableProcedure.class);
63  
64    private final AtomicBoolean aborted = new AtomicBoolean(false);
65  
66    // used for compatibility with old clients
67    private final ProcedurePrepareLatch syncLatch;
68  
69    private HTableDescriptor hTableDescriptor;
70    private List<HRegionInfo> newRegions;
71    private UserGroupInformation user;
72  
73    public CreateTableProcedure() {
74      // Required by the Procedure framework to create the procedure on replay
75      syncLatch = null;
76    }
77  
78    public CreateTableProcedure(final MasterProcedureEnv env,
79        final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions) {
80      this(env, hTableDescriptor, newRegions, null);
81    }
82  
83    public CreateTableProcedure(final MasterProcedureEnv env,
84        final HTableDescriptor hTableDescriptor, final HRegionInfo[] newRegions,
85        final ProcedurePrepareLatch syncLatch) {
86      this.hTableDescriptor = hTableDescriptor;
87      this.newRegions = newRegions != null ? Lists.newArrayList(newRegions) : null;
88      this.user = env.getRequestUser().getUGI();
89      this.setOwner(this.user.getShortUserName());
90  
91      // used for compatibility with clients without procedures
92      // they need a sync TableExistsException
93      this.syncLatch = syncLatch;
94    }
95  
96    @Override
97    protected Flow executeFromState(final MasterProcedureEnv env, final CreateTableState state)
98        throws InterruptedException {
99      if (LOG.isTraceEnabled()) {
100       LOG.trace(this + " execute state=" + state);
101     }
102     try {
103       switch (state) {
104         case CREATE_TABLE_PRE_OPERATION:
105           // Verify if we can create the table
106           boolean exists = !prepareCreate(env);
107           ProcedurePrepareLatch.releaseLatch(syncLatch, this);
108 
109           if (exists) {
110             assert isFailed() : "the delete should have an exception here";
111             return Flow.NO_MORE_STATE;
112           }
113 
114           preCreate(env);
115           setNextState(CreateTableState.CREATE_TABLE_WRITE_FS_LAYOUT);
116           break;
117         case CREATE_TABLE_WRITE_FS_LAYOUT:
118           newRegions = createFsLayout(env, hTableDescriptor, newRegions);
119           setNextState(CreateTableState.CREATE_TABLE_ADD_TO_META);
120           break;
121         case CREATE_TABLE_ADD_TO_META:
122           newRegions = addTableToMeta(env, hTableDescriptor, newRegions);
123           setNextState(CreateTableState.CREATE_TABLE_ASSIGN_REGIONS);
124           break;
125         case CREATE_TABLE_ASSIGN_REGIONS:
126           assignRegions(env, getTableName(), newRegions);
127           setNextState(CreateTableState.CREATE_TABLE_UPDATE_DESC_CACHE);
128           break;
129         case CREATE_TABLE_UPDATE_DESC_CACHE:
130           updateTableDescCache(env, getTableName());
131           setNextState(CreateTableState.CREATE_TABLE_POST_OPERATION);
132           break;
133         case CREATE_TABLE_POST_OPERATION:
134           postCreate(env);
135           return Flow.NO_MORE_STATE;
136         default:
137           throw new UnsupportedOperationException("unhandled state=" + state);
138       }
139     } catch (HBaseException|IOException e) {
140       LOG.error("Error trying to create table=" + getTableName() + " state=" + state, e);
141       setFailure("master-create-table", e);
142     }
143     return Flow.HAS_MORE_STATE;
144   }
145 
146   @Override
147   protected void rollbackState(final MasterProcedureEnv env, final CreateTableState state)
148       throws IOException {
149     if (LOG.isTraceEnabled()) {
150       LOG.trace(this + " rollback state=" + state);
151     }
152     try {
153       switch (state) {
154         case CREATE_TABLE_POST_OPERATION:
155           break;
156         case CREATE_TABLE_UPDATE_DESC_CACHE:
157           DeleteTableProcedure.deleteTableDescriptorCache(env, getTableName());
158           break;
159         case CREATE_TABLE_ASSIGN_REGIONS:
160           DeleteTableProcedure.deleteAssignmentState(env, getTableName());
161           break;
162         case CREATE_TABLE_ADD_TO_META:
163           DeleteTableProcedure.deleteFromMeta(env, getTableName(), newRegions);
164           break;
165         case CREATE_TABLE_WRITE_FS_LAYOUT:
166           DeleteTableProcedure.deleteFromFs(env, getTableName(), newRegions, false);
167           break;
168         case CREATE_TABLE_PRE_OPERATION:
169           DeleteTableProcedure.deleteTableStates(env, getTableName());
170           // TODO-MAYBE: call the deleteTable coprocessor event?
171           ProcedurePrepareLatch.releaseLatch(syncLatch, this);
172           break;
173         default:
174           throw new UnsupportedOperationException("unhandled state=" + state);
175       }
176     } catch (HBaseException e) {
177       LOG.warn("Failed rollback attempt step=" + state + " table=" + getTableName(), e);
178       throw new IOException(e);
179     } catch (IOException e) {
180       // This will be retried. Unless there is a bug in the code,
181       // this should be just a "temporary error" (e.g. network down)
182       LOG.warn("Failed rollback attempt step=" + state + " table=" + getTableName(), e);
183       throw e;
184     }
185   }
186 
187   @Override
188   protected CreateTableState getState(final int stateId) {
189     return CreateTableState.valueOf(stateId);
190   }
191 
192   @Override
193   protected int getStateId(final CreateTableState state) {
194     return state.getNumber();
195   }
196 
197   @Override
198   protected CreateTableState getInitialState() {
199     return CreateTableState.CREATE_TABLE_PRE_OPERATION;
200   }
201 
202   @Override
203   protected void setNextState(final CreateTableState state) {
204     if (aborted.get()) {
205       setAbortFailure("create-table", "abort requested");
206     } else {
207       super.setNextState(state);
208     }
209   }
210 
211   @Override
212   public TableName getTableName() {
213     return hTableDescriptor.getTableName();
214   }
215 
216   @Override
217   public TableOperationType getTableOperationType() {
218     return TableOperationType.CREATE;
219   }
220 
221   @Override
222   public boolean abort(final MasterProcedureEnv env) {
223     aborted.set(true);
224     return true;
225   }
226 
227   @Override
228   public void toStringClassDetails(StringBuilder sb) {
229     sb.append(getClass().getSimpleName());
230     sb.append(" (table=");
231     sb.append(getTableName());
232     sb.append(")");
233   }
234 
235   @Override
236   public void serializeStateData(final OutputStream stream) throws IOException {
237     super.serializeStateData(stream);
238 
239     MasterProcedureProtos.CreateTableStateData.Builder state =
240       MasterProcedureProtos.CreateTableStateData.newBuilder()
241         .setUserInfo(MasterProcedureUtil.toProtoUserInfo(this.user))
242         .setTableSchema(hTableDescriptor.convert());
243     if (newRegions != null) {
244       for (HRegionInfo hri: newRegions) {
245         state.addRegionInfo(HRegionInfo.convert(hri));
246       }
247     }
248     state.build().writeDelimitedTo(stream);
249   }
250 
251   @Override
252   public void deserializeStateData(final InputStream stream) throws IOException {
253     super.deserializeStateData(stream);
254 
255     MasterProcedureProtos.CreateTableStateData state =
256       MasterProcedureProtos.CreateTableStateData.parseDelimitedFrom(stream);
257     user = MasterProcedureUtil.toUserInfo(state.getUserInfo());
258     hTableDescriptor = HTableDescriptor.convert(state.getTableSchema());
259     if (state.getRegionInfoCount() == 0) {
260       newRegions = null;
261     } else {
262       newRegions = new ArrayList<HRegionInfo>(state.getRegionInfoCount());
263       for (HBaseProtos.RegionInfo hri: state.getRegionInfoList()) {
264         newRegions.add(HRegionInfo.convert(hri));
265       }
266     }
267   }
268 
269   @Override
270   protected boolean acquireLock(final MasterProcedureEnv env) {
271     if (!getTableName().isSystemTable() && env.waitInitialized(this)) {
272       return false;
273     }
274     return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());
275   }
276 
277   @Override
278   protected void releaseLock(final MasterProcedureEnv env) {
279     env.getProcedureQueue().releaseTableExclusiveLock(this, getTableName());
280   }
281 
282   private boolean prepareCreate(final MasterProcedureEnv env) throws IOException {
283     final TableName tableName = getTableName();
284     if (MetaTableAccessor.tableExists(env.getMasterServices().getConnection(), tableName)) {
285       setFailure("master-create-table", new TableExistsException(getTableName()));
286       return false;
287     }
288     // During master initialization, the ZK state could be inconsistent from failed DDL
289     // in the past. If we fail here, it would prevent master to start.  We should force
290     // setting the system table state regardless the table state.
291     boolean skipTableStateCheck =
292         !(env.getMasterServices().isInitialized()) && tableName.isSystemTable();
293     if (!skipTableStateCheck) {
294       TableStateManager tsm = env.getMasterServices().getAssignmentManager().getTableStateManager();
295       if (tsm.isTableState(tableName, true, ZooKeeperProtos.Table.State.ENABLING,
296           ZooKeeperProtos.Table.State.ENABLED)) {
297         LOG.warn("The table " + tableName + " does not exist in meta but has a znode. " +
298                "run hbck to fix inconsistencies.");
299         setFailure("master-create-table", new TableExistsException(getTableName()));
300         return false;
301       }
302     }
303     return true;
304   }
305 
306   private void preCreate(final MasterProcedureEnv env)
307       throws IOException, InterruptedException {
308     if (!getTableName().isSystemTable()) {
309       ProcedureSyncWait.getMasterQuotaManager(env)
310         .checkNamespaceTableAndRegionQuota(getTableName(), newRegions.size());
311     }
312 
313     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
314     if (cpHost != null) {
315       final HRegionInfo[] regions = newRegions == null ? null :
316         newRegions.toArray(new HRegionInfo[newRegions.size()]);
317       user.doAs(new PrivilegedExceptionAction<Void>() {
318         @Override
319         public Void run() throws Exception {
320           cpHost.preCreateTableHandler(hTableDescriptor, regions);
321           return null;
322         }
323       });
324     }
325   }
326 
327   private void postCreate(final MasterProcedureEnv env)
328       throws IOException, InterruptedException {
329     final MasterCoprocessorHost cpHost = env.getMasterCoprocessorHost();
330     if (cpHost != null) {
331       final HRegionInfo[] regions = (newRegions == null) ? null :
332         newRegions.toArray(new HRegionInfo[newRegions.size()]);
333       user.doAs(new PrivilegedExceptionAction<Void>() {
334         @Override
335         public Void run() throws Exception {
336           cpHost.postCreateTableHandler(hTableDescriptor, regions);
337           return null;
338         }
339       });
340     }
341   }
342 
343   protected interface CreateHdfsRegions {
344     List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
345       final Path tableRootDir, final TableName tableName,
346       final List<HRegionInfo> newRegions) throws IOException;
347   }
348 
349   protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
350       final HTableDescriptor hTableDescriptor, final List<HRegionInfo> newRegions)
351       throws IOException {
352     return createFsLayout(env, hTableDescriptor, newRegions, new CreateHdfsRegions() {
353       @Override
354       public List<HRegionInfo> createHdfsRegions(final MasterProcedureEnv env,
355           final Path tableRootDir, final TableName tableName,
356           final List<HRegionInfo> newRegions) throws IOException {
357         HRegionInfo[] regions = newRegions != null ?
358           newRegions.toArray(new HRegionInfo[newRegions.size()]) : null;
359         return ModifyRegionUtils.createRegions(env.getMasterConfiguration(),
360             tableRootDir, hTableDescriptor, regions, null);
361       }
362     });
363   }
364 
365   protected static List<HRegionInfo> createFsLayout(final MasterProcedureEnv env,
366       final HTableDescriptor hTableDescriptor, List<HRegionInfo> newRegions,
367       final CreateHdfsRegions hdfsRegionHandler) throws IOException {
368     final MasterFileSystem mfs = env.getMasterServices().getMasterFileSystem();
369     final Path tempdir = mfs.getTempDir();
370 
371     // 1. Create Table Descriptor
372     // using a copy of descriptor, table will be created enabling first
373     final Path tempTableDir = FSUtils.getTableDir(tempdir, hTableDescriptor.getTableName());
374     new FSTableDescriptors(env.getMasterConfiguration()).createTableDescriptorForTableDirectory(
375       tempTableDir, hTableDescriptor, false);
376 
377     // 2. Create Regions
378     newRegions = hdfsRegionHandler.createHdfsRegions(env, tempdir,
379       hTableDescriptor.getTableName(), newRegions);
380 
381     // 3. Move Table temp directory to the hbase root location
382     final Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), hTableDescriptor.getTableName());
383     FileSystem fs = mfs.getFileSystem();
384     if (!fs.delete(tableDir, true) && fs.exists(tableDir)) {
385       throw new IOException("Couldn't delete " + tableDir);
386     }
387     if (!fs.rename(tempTableDir, tableDir)) {
388       throw new IOException("Unable to move table from temp=" + tempTableDir +
389         " to hbase root=" + tableDir);
390     }
391     return newRegions;
392   }
393 
394   protected static List<HRegionInfo> addTableToMeta(final MasterProcedureEnv env,
395       final HTableDescriptor hTableDescriptor,
396       final List<HRegionInfo> regions) throws IOException {
397     if (regions != null && regions.size() > 0) {
398       ProcedureSyncWait.waitMetaRegions(env);
399 
400       // Add regions to META
401       addRegionsToMeta(env, hTableDescriptor, regions);
402       // Add replicas if needed
403       List<HRegionInfo> newRegions = addReplicas(env, hTableDescriptor, regions);
404 
405       // Setup replication for region replicas if needed
406       if (hTableDescriptor.getRegionReplication() > 1) {
407         ServerRegionReplicaUtil.setupRegionReplicaReplication(env.getMasterConfiguration());
408       }
409       return newRegions;
410     }
411     return regions;
412   }
413 
414   /**
415    * Create any replicas for the regions (the default replicas that was
416    * already created is passed to the method)
417    * @param hTableDescriptor descriptor to use
418    * @param regions default replicas
419    * @return the combined list of default and non-default replicas
420    */
421   private static List<HRegionInfo> addReplicas(final MasterProcedureEnv env,
422       final HTableDescriptor hTableDescriptor,
423       final List<HRegionInfo> regions) {
424     int numRegionReplicas = hTableDescriptor.getRegionReplication() - 1;
425     if (numRegionReplicas <= 0) {
426       return regions;
427     }
428     List<HRegionInfo> hRegionInfos =
429         new ArrayList<HRegionInfo>((numRegionReplicas+1)*regions.size());
430     for (int i = 0; i < regions.size(); i++) {
431       for (int j = 1; j <= numRegionReplicas; j++) {
432         hRegionInfos.add(RegionReplicaUtil.getRegionInfoForReplica(regions.get(i), j));
433       }
434     }
435     hRegionInfos.addAll(regions);
436     return hRegionInfos;
437   }
438 
439   protected static void assignRegions(final MasterProcedureEnv env,
440       final TableName tableName, final List<HRegionInfo> regions)
441       throws HBaseException, IOException {
442     ProcedureSyncWait.waitRegionServers(env);
443 
444     final AssignmentManager assignmentManager = env.getMasterServices().getAssignmentManager();
445 
446     // Mark the table as Enabling
447     assignmentManager.getTableStateManager().setTableState(tableName,
448         ZooKeeperProtos.Table.State.ENABLING);
449 
450     // Trigger immediate assignment of the regions in round-robin fashion
451     ModifyRegionUtils.assignRegions(assignmentManager, regions);
452 
453     // Enable table
454     assignmentManager.getTableStateManager()
455       .setTableState(tableName, ZooKeeperProtos.Table.State.ENABLED);
456   }
457 
458   /**
459    * Add the specified set of regions to the hbase:meta table.
460    */
461   protected static void addRegionsToMeta(final MasterProcedureEnv env,
462       final HTableDescriptor hTableDescriptor,
463       final List<HRegionInfo> regionInfos) throws IOException {
464     MetaTableAccessor.addRegionsToMeta(env.getMasterServices().getConnection(),
465       regionInfos, hTableDescriptor.getRegionReplication());
466   }
467 
468   protected static void updateTableDescCache(final MasterProcedureEnv env,
469       final TableName tableName) throws IOException {
470     env.getMasterServices().getTableDescriptors().get(tableName);
471   }
472 
473   @Override
474   protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
475     // system tables are created on bootstrap internally by the system
476     // the client does not know about this procedures.
477     return !getTableName().isSystemTable();
478   }
479 }