View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master.handler;
20  
21  import java.io.IOException;
22  import java.util.List;
23  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.fs.FileSystem;
28  import org.apache.hadoop.fs.Path;
29  import org.apache.hadoop.hbase.CoordinatedStateException;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.Server;
32  import org.apache.hadoop.hbase.TableName;
33  import org.apache.hadoop.hbase.MetaTableAccessor;
34  import org.apache.hadoop.hbase.master.AssignmentManager;
35  import org.apache.hadoop.hbase.master.HMaster;
36  import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
37  import org.apache.hadoop.hbase.master.MasterFileSystem;
38  import org.apache.hadoop.hbase.master.MasterServices;
39  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
40  import org.apache.hadoop.hbase.util.FSTableDescriptors;
41  import org.apache.hadoop.hbase.util.FSUtils;
42  import org.apache.hadoop.hbase.util.ModifyRegionUtils;
43  
44  /**
45   * Truncate the table by removing META and the HDFS files and recreating it.
46   * If the 'preserveSplits' option is set to true, the region splits are preserved on recreate.
47   *
48   * If the operation fails in the middle it may require hbck to fix the system state.
49   */
50  @InterfaceAudience.Private
51  public class TruncateTableHandler extends DeleteTableHandler {
52    private static final Log LOG = LogFactory.getLog(TruncateTableHandler.class);
53  
54    private final boolean preserveSplits;
55  
56    public TruncateTableHandler(final TableName tableName, final Server server,
57        final MasterServices masterServices, boolean preserveSplits) {
58      super(tableName, server, masterServices);
59      this.preserveSplits = preserveSplits;
60    }
61  
62    @Override
63    protected void handleTableOperation(List<HRegionInfo> regions)
64        throws IOException, CoordinatedStateException {
65      MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
66      if (cpHost != null) {
67        cpHost.preTruncateTableHandler(this.tableName);
68      }
69  
70      // 1. Wait because of region in transition
71      waitRegionInTransition(regions);
72  
73      // 2. Remove table from hbase:meta and HDFS
74      removeTableData(regions);
75  
76      // -----------------------------------------------------------------------
77      // PONR: At this point the table is deleted.
78      //       If the recreate fails, the user can only re-create the table.
79      // -----------------------------------------------------------------------
80  
81      // 3. Recreate the regions
82      recreateTable(regions);
83  
84      if (cpHost != null) {
85        cpHost.postTruncateTableHandler(this.tableName);
86      }
87    }
88  
89    private void recreateTable(final List<HRegionInfo> regions) throws IOException {
90      MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
91      Path tempdir = mfs.getTempDir();
92      FileSystem fs = mfs.getFileSystem();
93  
94      AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
95  
96      // 1. Set table znode
97      CreateTableHandler.checkAndSetEnablingTable(assignmentManager, tableName, false);
98      try {
99        // 1. Create Table Descriptor
100       Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
101       new FSTableDescriptors(server.getConfiguration())
102         .createTableDescriptorForTableDirectory(tempTableDir, this.hTableDescriptor, false);
103       Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
104 
105       HRegionInfo[] newRegions;
106       if (this.preserveSplits) {
107         newRegions = recreateRegionInfo(regions);
108         LOG.info("Truncate will preserve " + newRegions.length + " regions");
109       } else {
110         newRegions = new HRegionInfo[1];
111         newRegions[0] = new HRegionInfo(this.tableName, null, null);
112         LOG.info("Truncate will not preserve the regions");
113       }
114 
115       // 2. Create Regions
116       List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
117         masterServices.getConfiguration(), tempdir,
118         this.hTableDescriptor, newRegions, null);
119 
120       // 3. Move Table temp directory to the hbase root location
121       if (!fs.rename(tempTableDir, tableDir)) {
122         throw new IOException("Unable to move table from temp=" + tempTableDir +
123           " to hbase root=" + tableDir);
124       }
125 
126       // 4. Add regions to META
127       MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(),
128         regionInfos, hTableDescriptor.getRegionReplication());
129 
130       // 5. Trigger immediate assignment of the regions in round-robin fashion
131       ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
132 
133       // 6. Set table enabled flag up in zk.
134       try {
135         assignmentManager.getTableStateManager().setTableState(tableName,
136           ZooKeeperProtos.Table.State.ENABLED);
137       } catch (CoordinatedStateException e) {
138         throw new IOException("Unable to ensure that " + tableName + " will be" +
139           " enabled because of a ZooKeeper issue", e);
140       }
141     } catch (IOException e) {
142       CreateTableHandler.removeEnablingTable(assignmentManager, tableName);
143       throw e;
144     }
145   }
146 
147   private static HRegionInfo[] recreateRegionInfo(final List<HRegionInfo> regions) {
148     HRegionInfo[] newRegions = new HRegionInfo[regions.size()];
149     int index = 0;
150     for (HRegionInfo hri: regions) {
151       newRegions[index++] = new HRegionInfo(hri.getTable(), hri.getStartKey(), hri.getEndKey());
152     }
153     return newRegions;
154   }
155 }