HBASE-13536 Cleanup the handlers that are no longer being used.

This commit is contained in:
Srikanth Srungarapu 2015-04-24 12:40:59 -07:00
parent 1a0c61c974
commit 8ab6698120
6 changed files with 0 additions and 772 deletions

View File

@ -1,218 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.backup.HFileArchiver;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.RegionStates;
import org.apache.hadoop.hbase.regionserver.HRegion;
@InterfaceAudience.Private
public class DeleteTableHandler extends TableEventHandler {
private static final Log LOG = LogFactory.getLog(DeleteTableHandler.class);
protected HTableDescriptor hTableDescriptor = null;
public DeleteTableHandler(TableName tableName, Server server,
final MasterServices masterServices) {
super(EventType.C_M_DELETE_TABLE, tableName, server, masterServices);
}
@Override
protected void prepareWithTableLock() throws IOException {
// The next call fails if no such table.
hTableDescriptor = getTableDescriptor();
}
protected void waitRegionInTransition(final List<HRegionInfo> regions)
throws IOException, CoordinatedStateException {
AssignmentManager am = this.masterServices.getAssignmentManager();
RegionStates states = am.getRegionStates();
long waitTime = server.getConfiguration().
getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
for (HRegionInfo region : regions) {
long done = System.currentTimeMillis() + waitTime;
while (System.currentTimeMillis() < done) {
if (states.isRegionInState(region, State.FAILED_OPEN)) {
am.regionOffline(region);
}
if (!states.isRegionInTransition(region)) break;
try {
Thread.sleep(waitingTimeForEvents);
} catch (InterruptedException e) {
LOG.warn("Interrupted while sleeping");
throw (InterruptedIOException)new InterruptedIOException().initCause(e);
}
LOG.debug("Waiting on region to clear regions in transition; "
+ am.getRegionStates().getRegionTransitionState(region));
}
if (states.isRegionInTransition(region)) {
throw new IOException("Waited hbase.master.wait.on.region (" +
waitTime + "ms) for region to leave region " +
region.getRegionNameAsString() + " in transitions");
}
}
}
@Override
protected void handleTableOperation(List<HRegionInfo> regions)
throws IOException, CoordinatedStateException {
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteTableHandler(this.tableName);
}
// 1. Wait because of region in transition
waitRegionInTransition(regions);
try {
// 2. Remove table from hbase:meta and HDFS
removeTableData(regions);
} finally {
// 3. Update table descriptor cache
LOG.debug("Removing '" + tableName + "' descriptor.");
this.masterServices.getTableDescriptors().remove(tableName);
AssignmentManager am = this.masterServices.getAssignmentManager();
// 4. Clean up regions of the table in RegionStates.
LOG.debug("Removing '" + tableName + "' from region states.");
am.getRegionStates().tableDeleted(tableName);
// 5. If entry for this table in zk, and up in AssignmentManager, remove it.
LOG.debug("Marking '" + tableName + "' as deleted.");
am.getTableStateManager().setDeletedTable(tableName);
// 6.Clean any remaining rows for this table.
cleanAnyRemainingRows();
}
if (cpHost != null) {
cpHost.postDeleteTableHandler(this.tableName);
}
((HMaster) this.server).getMasterQuotaManager().removeTableFromNamespaceQuota(tableName);
}
/**
* There may be items for this table still up in hbase:meta in the case where the
* info:regioninfo column was empty because of some write error. Remove ALL rows from hbase:meta
* that have to do with this table. See HBASE-12980.
* @throws IOException
*/
private void cleanAnyRemainingRows() throws IOException {
Scan tableScan = MetaTableAccessor.getScanForTableName(tableName);
try (Table metaTable =
this.masterServices.getConnection().getTable(TableName.META_TABLE_NAME)) {
List<Delete> deletes = new ArrayList<Delete>();
try (ResultScanner resScanner = metaTable.getScanner(tableScan)) {
for (Result result : resScanner) {
deletes.add(new Delete(result.getRow()));
}
}
if (!deletes.isEmpty()) {
LOG.warn("Deleting some vestigal " + deletes.size() + " rows of " + this.tableName +
" from " + TableName.META_TABLE_NAME);
if (LOG.isDebugEnabled()) {
for (Delete d: deletes) LOG.debug("Purging " + d);
}
metaTable.delete(deletes);
}
}
}
/**
* Removes the table from hbase:meta and archives the HDFS files.
*/
protected void removeTableData(final List<HRegionInfo> regions)
throws IOException, CoordinatedStateException {
// 1. Remove regions from META
LOG.debug("Deleting regions from META");
MetaTableAccessor.deleteRegions(this.server.getConnection(), regions);
// -----------------------------------------------------------------------
// NOTE: At this point we still have data on disk, but nothing in hbase:meta
// if the rename below fails, hbck will report an inconsistency.
// -----------------------------------------------------------------------
// 2. Move the table in /hbase/.tmp
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
Path tempTableDir = mfs.moveTableToTemp(tableName);
// 3. Archive regions from FS (temp directory)
FileSystem fs = mfs.getFileSystem();
for (HRegionInfo hri : regions) {
LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
HFileArchiver.archiveRegion(fs, mfs.getRootDir(),
tempTableDir, HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
}
// 4. Delete table directory from FS (temp directory)
if (!fs.delete(tempTableDir, true)) {
LOG.error("Couldn't delete " + tempTableDir);
}
LOG.debug("Table '" + tableName + "' archived!");
}
@Override
protected void releaseTableLock() {
super.releaseTableLock();
try {
masterServices.getTableLockManager().tableDeleted(tableName);
} catch (IOException ex) {
LOG.warn("Received exception from TableLockManager.tableDeleted:", ex); //not critical
}
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" + tableName;
}
}

View File

@ -1,145 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.ResultScanner;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@InterfaceAudience.Private
public class ModifyTableHandler extends TableEventHandler {
private static final Log LOG = LogFactory.getLog(ModifyTableHandler.class);
private final HTableDescriptor htd;
public ModifyTableHandler(final TableName tableName,
final HTableDescriptor htd, final Server server,
final MasterServices masterServices) {
super(EventType.C_M_MODIFY_TABLE, tableName, server, masterServices);
// This is the new schema we are going to write out as this modification.
this.htd = htd;
}
@Override
protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock();
// Check operation is possible on the table in its current state
// Also checks whether the table exists
if (masterServices.getAssignmentManager().getTableStateManager()
.isTableState(this.htd.getTableName(), ZooKeeperProtos.Table.State.ENABLED)
&& this.htd.getRegionReplication() != getTableDescriptor().getRegionReplication()) {
throw new IOException("REGION_REPLICATION change is not supported for enabled tables");
}
}
@Override
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preModifyTableHandler(this.tableName, this.htd);
}
// Update descriptor
HTableDescriptor oldHtd = getTableDescriptor();
this.masterServices.getTableDescriptors().add(this.htd);
deleteFamilyFromFS(hris, oldHtd.getFamiliesKeys());
removeReplicaColumnsIfNeeded(this.htd.getRegionReplication(), oldHtd.getRegionReplication(),
htd.getTableName());
// Setup replication for region replicas if needed
if (htd.getRegionReplication() > 1 && oldHtd.getRegionReplication() <= 1) {
ServerRegionReplicaUtil.setupRegionReplicaReplication(server.getConfiguration());
}
if (cpHost != null) {
cpHost.postModifyTableHandler(this.tableName, this.htd);
}
}
private void removeReplicaColumnsIfNeeded(int newReplicaCount, int oldReplicaCount,
TableName table) throws IOException {
if (newReplicaCount >= oldReplicaCount) return;
Set<byte[]> tableRows = new HashSet<byte[]>();
Scan scan = MetaTableAccessor.getScanForTableName(table);
scan.addColumn(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER);
Connection connection = this.masterServices.getConnection();
try (Table metaTable = connection.getTable(TableName.META_TABLE_NAME)) {
ResultScanner resScanner = metaTable.getScanner(scan);
for (Result result : resScanner) {
tableRows.add(result.getRow());
}
MetaTableAccessor.removeRegionReplicasFromMeta(tableRows, newReplicaCount,
oldReplicaCount - newReplicaCount, masterServices.getConnection());
}
}
/**
* Removes from hdfs the families that are not longer present in the new table descriptor.
*/
private void deleteFamilyFromFS(final List<HRegionInfo> hris, final Set<byte[]> oldFamilies) {
try {
Set<byte[]> newFamilies = this.htd.getFamiliesKeys();
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
for (byte[] familyName: oldFamilies) {
if (!newFamilies.contains(familyName)) {
LOG.debug("Removing family=" + Bytes.toString(familyName) +
" from table=" + this.tableName);
for (HRegionInfo hri: hris) {
// Delete the family directory in FS for all the regions one by one
mfs.deleteFamilyFromFS(hri, familyName);
}
}
}
} catch (IOException e) {
LOG.warn("Unable to remove on-disk directories for the removed families", e);
}
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() + "-" +
tableName;
}
}

View File

@ -1,89 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.InvalidFamilyOperationException;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
/**
* Handles adding a new family to an existing table.
*/
@InterfaceAudience.Private
public class TableAddFamilyHandler extends TableEventHandler {
private final HColumnDescriptor familyDesc;
public TableAddFamilyHandler(TableName tableName, HColumnDescriptor familyDesc,
Server server, final MasterServices masterServices) {
super(EventType.C_M_ADD_FAMILY, tableName, server, masterServices);
this.familyDesc = familyDesc;
}
@Override
protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor();
if (htd.hasFamily(familyDesc.getName())) {
throw new InvalidFamilyOperationException("Family '" +
familyDesc.getNameAsString() + "' already exists so cannot be added");
}
}
@Override
protected void handleTableOperation(List<HRegionInfo> hris)
throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getMasterCoprocessorHost();
if(cpHost != null){
cpHost.preAddColumnHandler(this.tableName, this.familyDesc);
}
// Update table descriptor
this.masterServices.getMasterFileSystem().addColumn(tableName, familyDesc);
if(cpHost != null){
cpHost.postAddColumnHandler(this.tableName, this.familyDesc);
}
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
String family = "UnknownFamily";
if(familyDesc != null) {
family = familyDesc.getNameAsString();
}
return getClass().getSimpleName() + "-" + name + "-" +
getSeqid() + "-" + tableName + "-" + family;
}
}

View File

@ -1,90 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.util.Bytes;
/**
* Handles Deleting a column family from an existing table.
*/
@InterfaceAudience.Private
public class TableDeleteFamilyHandler extends TableEventHandler {
private byte [] familyName;
public TableDeleteFamilyHandler(TableName tableName, byte [] familyName,
Server server, final MasterServices masterServices) throws IOException {
super(EventType.C_M_DELETE_FAMILY, tableName, server, masterServices);
this.familyName = familyName;
}
@Override
protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor();
this.familyName = hasColumnFamily(htd, familyName);
}
@Override
protected void handleTableOperation(List<HRegionInfo> hris) throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preDeleteColumnHandler(this.tableName, this.familyName);
}
// Update table descriptor
this.masterServices.getMasterFileSystem().deleteColumn(tableName, familyName);
// Remove the column family from the file system
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
for (HRegionInfo hri : hris) {
// Delete the family directory in FS for all the regions one by one
mfs.deleteFamilyFromFS(hri, familyName);
}
if (cpHost != null) {
cpHost.postDeleteColumnHandler(this.tableName, this.familyName);
}
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
String family = "UnknownFamily";
if(familyName != null) {
family = Bytes.toString(familyName);
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() +
"-" + tableName + "-" + family;
}
}

View File

@ -1,84 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.executor.EventType;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterServices;
/**
* Handles adding a new family to an existing table.
*/
@InterfaceAudience.Private
public class TableModifyFamilyHandler extends TableEventHandler {
private final HColumnDescriptor familyDesc;
public TableModifyFamilyHandler(TableName tableName,
HColumnDescriptor familyDesc, Server server,
final MasterServices masterServices) {
super(EventType.C_M_MODIFY_FAMILY, tableName, server, masterServices);
this.familyDesc = familyDesc;
}
@Override
protected void prepareWithTableLock() throws IOException {
super.prepareWithTableLock();
HTableDescriptor htd = getTableDescriptor();
hasColumnFamily(htd, familyDesc.getName());
}
@Override
protected void handleTableOperation(List<HRegionInfo> regions) throws IOException {
MasterCoprocessorHost cpHost = ((HMaster) this.server)
.getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preModifyColumnHandler(this.tableName, this.familyDesc);
}
// Update table descriptor
this.masterServices.getMasterFileSystem().modifyColumn(tableName, familyDesc);
if (cpHost != null) {
cpHost.postModifyColumnHandler(this.tableName, this.familyDesc);
}
}
@Override
public String toString() {
String name = "UnknownServerName";
if(server != null && server.getServerName() != null) {
name = server.getServerName().toString();
}
String family = "UnknownFamily";
if(familyDesc != null) {
family = familyDesc.getNameAsString();
}
return getClass().getSimpleName() + "-" + name + "-" + getSeqid() +
"-" + tableName + "-" + family;
}
}

View File

@ -1,146 +0,0 @@
/**
*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.master.handler;
import java.io.IOException;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CoordinatedStateException;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.master.AssignmentManager;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
import org.apache.hadoop.hbase.util.FSTableDescriptors;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.ModifyRegionUtils;
/**
* Truncate the table by removing META and the HDFS files and recreating it.
* If the 'preserveSplits' option is set to true, the region splits are preserved on recreate.
*
* If the operation fails in the middle it may require hbck to fix the system state.
*/
@InterfaceAudience.Private
public class TruncateTableHandler extends DeleteTableHandler {
private static final Log LOG = LogFactory.getLog(TruncateTableHandler.class);
private final boolean preserveSplits;
public TruncateTableHandler(final TableName tableName, final Server server,
final MasterServices masterServices, boolean preserveSplits) {
super(tableName, server, masterServices);
this.preserveSplits = preserveSplits;
}
@Override
protected void handleTableOperation(List<HRegionInfo> regions)
throws IOException, CoordinatedStateException {
MasterCoprocessorHost cpHost = ((HMaster) this.server).getMasterCoprocessorHost();
if (cpHost != null) {
cpHost.preTruncateTableHandler(this.tableName);
}
// 1. Wait because of region in transition
waitRegionInTransition(regions);
// 2. Remove table from hbase:meta and HDFS
removeTableData(regions);
// -----------------------------------------------------------------------
// PONR: At this point the table is deleted.
// If the recreate fails, the user can only re-create the table.
// -----------------------------------------------------------------------
// 3. Recreate the regions
recreateTable(regions);
if (cpHost != null) {
cpHost.postTruncateTableHandler(this.tableName);
}
}
private void recreateTable(final List<HRegionInfo> regions) throws IOException {
MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
Path tempdir = mfs.getTempDir();
FileSystem fs = mfs.getFileSystem();
AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
// 1. Set table znode
CreateTableHandler.checkAndSetEnablingTable(assignmentManager, tableName);
try {
// 1. Create Table Descriptor
Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
new FSTableDescriptors(server.getConfiguration())
.createTableDescriptorForTableDirectory(tempTableDir, this.hTableDescriptor, false);
Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
HRegionInfo[] newRegions;
if (this.preserveSplits) {
newRegions = regions.toArray(new HRegionInfo[regions.size()]);
LOG.info("Truncate will preserve " + newRegions.length + " regions");
} else {
newRegions = new HRegionInfo[1];
newRegions[0] = new HRegionInfo(this.tableName, null, null);
LOG.info("Truncate will not preserve the regions");
}
// 2. Create Regions
List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
masterServices.getConfiguration(), tempdir,
this.hTableDescriptor, newRegions, null);
// 3. Move Table temp directory to the hbase root location
if (!fs.rename(tempTableDir, tableDir)) {
throw new IOException("Unable to move table from temp=" + tempTableDir +
" to hbase root=" + tableDir);
}
// 4. Add regions to META
MetaTableAccessor.addRegionsToMeta(masterServices.getConnection(),
regionInfos, hTableDescriptor.getRegionReplication());
// 5. Trigger immediate assignment of the regions in round-robin fashion
ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
// 6. Set table enabled flag up in zk.
try {
assignmentManager.getTableStateManager().setTableState(tableName,
ZooKeeperProtos.Table.State.ENABLED);
} catch (CoordinatedStateException e) {
throw new IOException("Unable to ensure that " + tableName + " will be" +
" enabled because of a ZooKeeper issue", e);
}
} catch (IOException e) {
CreateTableHandler.removeEnablingTable(assignmentManager, tableName);
throw e;
}
}
}