### Eclipse Workspace Patch 1.0 #P apache-trunk Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/OpenedRegionHandler.java (working copy) @@ -117,6 +117,10 @@ + regionInfo.getRegionNameAsString() + " but " + "this table is disabled, triggering close of region"); assignmentManager.unassign(regionInfo); + } else if (this.assignmentManager.isRegionInMerging(regionInfo)) { + LOG.debug("Opened region " + regionInfo.getRegionNameAsString() + + " but " + "this region is in merging, triggering close of region"); + assignmentManager.unassign(regionInfo); } } } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeTransaction.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeTransaction.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeTransaction.java (revision 0) @@ -0,0 +1,513 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaEditor; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.master.RegionMergeManager.MergeTransactionData; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.StoreFile; +import org.apache.hadoop.hbase.regionserver.wal.HLogUtil; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; + +/** + * Executes region merge as a "transaction". Call {@link #prepare} to check + * whether the transaction is available, {@link #offlineRegion} to offline + * merging regions, {@link #executeMerging} to merge files on fs, + * {@link #completeMerging} and {@link #cancelMerging} to remove useless data. + * + *

+ * This class is not thread safe. Caller needs to ensure merge is run by one + * thread only. + */ +public class MergeTransaction { + private static final Log LOG = LogFactory.getLog(MergeTransaction.class); + + private final HTableDescriptor htd; + private final HRegionInfo regionA; + private final HRegionInfo regionB; + // Region C merged from regionA and regionB + private HRegionInfo mergedRegion; + private final MergeTransactionData transactionData; + private final RegionMergeManager mergeManager; + + // Which state does this merge transaction start from + private final JournalState initialState; + // Current state of this merge transaction + protected JournalState currentState; + + private boolean successful = false; + + private final Path rootDir; + private final FileSystem fs; + + private final MonitoredTask status; + + /** + * State of the transaction journal. Each enum is a state in the merge + * transaction. Used to redo after master restart + */ + enum JournalState { + /** + * Initial state when requesting a merge + */ + CREATE_MERGE, + /** + * Enter the offline region state after {@link #prepare()} + */ + OFFLINE_REGION, + /** + * Enter the state executing merge after {@link #offlineRegion()} + */ + EXECUTE_MERGING, + /** + * Enter the state completing merging after region merged + */ + COMPLETE_MERGING, + /** + * Enter the state canceling merging if merge transaction is unavailable + */ + CANCEL_MERGING, + /** + * State representing transaction being over + */ + END + } + + public MergeTransaction(final MasterServices masterServices, + final RegionMergeManager mergeManager, + final MergeTransactionData transactionData, final HRegionInfo mergeA, + final HRegionInfo mergeB, final JournalState initialState, + final MonitoredTask status) throws FileNotFoundException, IOException { + this.rootDir = masterServices.getMasterFileSystem().getRootDir(); + this.fs = masterServices.getMasterFileSystem().getFileSystem(); + this.mergeManager = mergeManager; + this.transactionData = transactionData; + this.htd = masterServices.getTableDescriptors().get( + transactionData.getTableName()); + this.regionA = mergeA; + this.regionB = mergeB; + this.initialState = initialState; + this.currentState = initialState; + this.status = status; + } + + /** + * Does checks for merge inputs. + * + * Set state to OFFLINE_REGION if the merge is available else set the state to + * CANCEL_MERGING if it is not(e.g. merge region is a parent or has + * reference). + * @throws IOException + * @throws KeeperException + * @throws NoNodeException + */ + void prepare() throws IOException, NoNodeException, KeeperException { + status.setStatus("Preparing..."); + boolean available = true; + if (regionA.isSplit() || regionA.isOffline()) { + available = false; + LOG.debug("Region " + regionA.getRegionNameAsString() + + " is split or offline"); + } else if (regionB.isSplit() || regionB.isOffline()) { + available = false; + LOG.debug("Region " + regionB.getRegionNameAsString() + + " is split or offline"); + } else if (containsReferenceFile(regionA)) { + available = false; + LOG.debug("Region " + regionA.getRegionNameAsString() + + " has reference , cancel the merge"); + } else if (containsReferenceFile(regionB)) { + available = false; + LOG.debug("Region " + regionB.getRegionNameAsString() + + " has reference , cancel the merge"); + } else if (!ensureNoParentRegion()) { + available = false; + LOG.debug("Exist parent region and can't be cleared, cancel the merge"); + } + if (available) { + this.currentState = JournalState.OFFLINE_REGION; + } else { + this.currentState = JournalState.CANCEL_MERGING; + } + setJournalStateOnZK(); + } + + /** + * Run the transaction according to the current state + * @param masterServices + * @throws IOException + * @throws KeeperException + * @throws InterruptedException + */ + public void execute(MasterServices masterServices) throws IOException, + KeeperException, InterruptedException { + while (!masterServices.isStopped()) { + if (this.currentState.equals(JournalState.END)) { + break; + } + switch (this.currentState) { + case CREATE_MERGE: + prepare(); + break; + case OFFLINE_REGION: + offlineRegion(masterServices); + break; + case EXECUTE_MERGING: + executeMerging(masterServices); + break; + case COMPLETE_MERGING: + completeMerging(masterServices); + break; + case CANCEL_MERGING: + cancelMerging(masterServices); + break; + default: + throw new RuntimeException("Unhandled merge state: " + currentState); + } + } + } + + /** + * Offline the two merging regions + * @param masterServices + * @throws KeeperException + * @throws InterruptedException + * @throws IOException + */ + void offlineRegion(MasterServices masterServices) throws KeeperException, + InterruptedException, IOException { + status.setStatus("Offline region..."); + masterServices.getAssignmentManager().unassign(this.regionA); + masterServices.getAssignmentManager().unassign(this.regionB); + mergeManager.waitUntilRegionsOffline(this.regionA, this.regionB); + mergeManager.waitUntilNoDeadServerInProcess(masterServices); + boolean conditionA = false; + if (mergeManager.isRegionSplit(this.regionA) + || mergeManager.isRegionSplit(this.regionB)) { + LOG.info("Canceling merge because region " + + (mergeManager.isRegionSplit(this.regionA) ? regionA + .getEncodedName() : regionB.getEncodedName()) + " has been split"); + this.currentState = JournalState.CANCEL_MERGING; + } else if ((conditionA = containsEditsLogFile(this.regionA)) + || containsEditsLogFile(this.regionB)) { + LOG.info("Canceling merge because region " + + (conditionA ? regionA.getEncodedName() : regionB.getEncodedName()) + + " has edits log file"); + this.currentState = JournalState.CANCEL_MERGING; + }else { + this.currentState = JournalState.EXECUTE_MERGING; + } + setJournalStateOnZK(); + } + + /** + * Merging the files in HDFS, add merged region to META and assign it + * @param masterServices + * @throws NoNodeException + * @throws KeeperException + * @throws IOException + */ + void executeMerging(MasterServices masterServices) throws NoNodeException, + KeeperException, IOException { + status.setStatus("Merging regions in fs..."); + this.mergedRegion = HRegion.generateFixedRegionByMerge(htd, regionA, + regionB); + Pair mergedRegionInMeta = null; + boolean isMergedRegionSplit = false; + if (this.initialState == JournalState.EXECUTE_MERGING) { + // It is a redo from this step + mergedRegionInMeta = MetaReader.getRegion( + masterServices.getCatalogTracker(), mergedRegion.getRegionName()); + if (mergedRegionInMeta == null) { + isMergedRegionSplit = checkRegionSplit( + masterServices.getCatalogTracker(), mergedRegion); + LOG.debug("Redo merging " + transactionData + "from " + initialState + + "and check whether merged region" + + mergedRegion.getRegionNameAsString() + " is split =" + + isMergedRegionSplit); + } + } + if (!isMergedRegionSplit && mergedRegionInMeta == null) { + this.mergedRegion = HRegion.merge(masterServices.getConfiguration(), fs, + rootDir, htd, regionA, regionB); + MetaEditor.addRegionToMeta(masterServices.getCatalogTracker(), + this.mergedRegion); + } + status.setStatus("Assigning merged region " + + mergedRegion.getRegionNameAsString()); + if (!isMergedRegionSplit + && (mergedRegionInMeta == null || mergedRegionInMeta.getSecond() == null)) { + if (mergeManager.isRegionOffline(mergedRegion)) { + masterServices.getAssignmentManager().assign(this.mergedRegion, true); + } + } + this.currentState = JournalState.COMPLETE_MERGING; + setJournalStateOnZK(); + } + + /** + * Delete merging regions from META after completing the merge successfully + * @param masterServices + * @throws KeeperException + * @throws IOException + * @throws InterruptedException + */ + void completeMerging(MasterServices masterServices) throws KeeperException, + IOException, InterruptedException { + status.setStatus("Completing merging... "); + // Remove regions from .META. + if (this.regionA != null) { + MetaEditor.deleteRegion(masterServices.getCatalogTracker(), this.regionA); + } + if (this.regionB != null) { + MetaEditor.deleteRegion(masterServices.getCatalogTracker(), this.regionB); + } + this.mergeManager.waitUntilNoDeadServerInProcess(masterServices); + this.mergeManager.finishMergeTransaction(transactionData, true); + this.currentState = JournalState.END; + this.successful = true; + } + + /** + * Merge is unavailable, cancel it; Assign the merging region if it is offline + * because of merging + * @param masterServices + * @throws NoNodeException + * @throws KeeperException + * @throws InterruptedException + */ + void cancelMerging(MasterServices masterServices) throws NoNodeException, + KeeperException, InterruptedException { + status.setStatus("Canceling merging... "); + this.mergeManager.assignMergingRegionIfOffline(this.regionA); + this.mergeManager.assignMergingRegionIfOffline(this.regionB); + this.mergeManager.finishMergeTransaction(transactionData, true); + this.currentState = JournalState.END; + } + + protected void setJournalStateOnZK() throws NoNodeException, + KeeperException { + LOG.debug("Setting state " + currentState + " on zk for merging request " + + transactionData); + mergeManager.setJournalStateOnZK(transactionData.getTransactionName(), + this.currentState); + } + + /** + * Check whether region has Reference file + * @param region + * @return true if region has reference file + * @throws IOException + */ + private boolean containsReferenceFile(final HRegionInfo region) + throws IOException { + Path tabledir = new Path(rootDir, region.getTableNameAsString()); + boolean hasReference = false; + for (HColumnDescriptor family : htd.getFamilies()) { + Path p = HStore.getStoreHomedir(tabledir, region.getEncodedName(), + family.getName()); + if (!fs.exists(p)) continue; + // Look for reference files. Call listStatus with anonymous instance of + // PathFilter. + FileStatus[] ps = FSUtils.listStatus(fs, p, new PathFilter() { + public boolean accept(Path path) { + return StoreFile.isReference(path); + } + }); + + if (ps != null && ps.length > 0) { + hasReference = true; + break; + } + } + return hasReference; + } + + /** + * Check whether region has edits log file + * @param region + * @return true if region contains edits log file + * @throws IOException + */ + private boolean containsEditsLogFile(final HRegionInfo region) + throws IOException { + Path tableDir = new Path(rootDir, region.getTableNameAsString()); + Path regiondir = HRegion.getRegionDir(tableDir, region.getEncodedName()); + Set files = HLogUtil.getSplitEditFilesSorted(fs, regiondir); + return files != null && !files.isEmpty(); + } + + /** + * Between offline region and start merging, make sure there is no parent + * region of the two regions in .META. If exists, fixing up daughters would + * cause daughter regions(we have merged one) online again when we restart + * master, so we clear the parent region in this function to prevent the above + * case + * @return true if there is no parent region of merging regions + * @throws IOException + */ + private boolean ensureNoParentRegion() throws IOException { + Map splitParents = mergeManager + .getSplitParents(regionA.getTableName()); + for (Map.Entry splitParent : splitParents.entrySet()) { + // check whether its range contains merging region + if (doRegionsOverlap(regionA, splitParent.getKey()) + || doRegionsOverlap(regionB, splitParent.getKey())) { + LOG.warn("Region " + regionA.getEncodedName() + " or region " + + regionB.getEncodedName() + " has parent region " + + splitParent.getKey().getRegionNameAsString() + + ", try to clean it"); + if (!this.mergeManager.cleanParent(splitParent.getKey(), + splitParent.getValue())) { + LOG.warn("Failed cleaning parent region " + + splitParent.getKey().getRegionNameAsString() + + ", canceling merge"); + return false; + } + } + } + + return true; + } + + /** + * Check whether the region is split from the META + * @param catalogTracker + * @param checkRegion + * @param result + * @return true if region is split + * @throws IOException + */ + static boolean checkRegionSplit(final CatalogTracker catalogTracker, + final HRegionInfo checkRegion) throws IOException { + final AtomicBoolean isSplit = new AtomicBoolean(false); + byte[] startRow = checkRegion.getRegionName(); + final byte[] startKey = checkRegion.getStartKey(); + final byte[] endKey = checkRegion.getEndKey(); + final byte[] tableName = checkRegion.getTableName(); + MetaReader.fullScan(catalogTracker, new MetaReader.Visitor() { + @Override + public boolean visit(Result r) throws IOException { + if (r == null || r.isEmpty()) return true; + HRegionInfo info = HRegionInfo.getHRegionInfo(r); + if (info == null) return true; // Keep scanning + // other table,stop scanning + if (Bytes.compareTo(info.getTableName(), tableName) > 0) return false; + if (Bytes.compareTo(info.getTableName(), tableName) == 0) { + if (Bytes.compareTo(info.getStartKey(), startKey) == 0) { + if (Bytes.compareTo(info.getEndKey(), endKey) == 0) { + // find itself,keep scanning + return true; + } + isSplit.set(true); + return false; + } else if (Bytes.compareTo(info.getStartKey(), startKey) > 0) { + return false; + } + } + return true; + } + }, startRow); + return isSplit.get(); + } + + public boolean isSuccessful() { + return this.successful; + } + + public HRegionInfo getMergedRegion() { + return this.mergedRegion; + } + + public JournalState getCurrentState() { + return this.currentState; + } + + /** + * Check whether there is overlap between region A and region B + * @param regionA + * @param regionB + * @return true if regions' key ranges overlap + */ + private static boolean doRegionsOverlap(HRegionInfo regionA, + HRegionInfo regionB) { + // if A's startKey < B's startkey, swap them + if (Bytes.compareTo(regionA.getStartKey(), regionB.getStartKey()) < 0) { + return doRegionsOverlap(regionB, regionA); + } else { + if (Bytes.compareTo(regionA.getStartKey(), regionB.getEndKey()) < 0 + || Bytes.equals(regionB.getEndKey(), HConstants.EMPTY_END_ROW)) { + return true; + } else { + return false; + } + } + } + + /** + * Get the bytes for a specified JournalState + * @param state + * @return the bytes of given journal state + */ + static byte[] getBytesFromJournalState(JournalState state) { + return Bytes.toBytes(state.toString()); + + } + + /** + * Get an JournalState from bytes. Throws a {@link RuntimeException} if the + * data represents unknown bytes + * @param data + * @return the journal state as the given bytes + */ + static JournalState parseJournalStateFromBytes(byte[] data) { + String stateString = Bytes.toString(data); + for (JournalState state : JournalState.values()) { + if (state.toString().equals(stateString)) + return state; + } + throw new RuntimeException("Unknown JournalState " + stateString); + } +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java (working copy) @@ -102,6 +102,8 @@ protected final Server server; private ServerManager serverManager; + + private RegionMergeManager mergeManager; private CatalogTracker catalogTracker; @@ -1026,6 +1028,11 @@ LOG.info("Opened region " + regionNameStr + "but this table is disabled, triggering close of region"); unassign(regionInfo); + } else if (isRegionInMerging(regionInfo)) { + LOG.debug("Opened region " + + regionInfo.getRegionNameAsString() + " but " + + "this region is in merging, triggering close of region"); + unassign(regionInfo); } } } @@ -2166,6 +2173,8 @@ Map allRegions = MetaReader.fullScan( catalogTracker, disabledOrDisablingOrEnabling, true); if (allRegions == null || allRegions.isEmpty()) return; + // Remove the merging regions + removeMergingRegions(allRegions); // Determine what type of assignment to do on startup boolean retainAssignment = server.getConfiguration(). @@ -2718,6 +2727,9 @@ public void handleSplitReport(final ServerName sn, final HRegionInfo parent, final HRegionInfo a, final HRegionInfo b) { regionOffline(parent); + if (this.mergeManager != null) { + this.mergeManager.notifyRegionSplit(parent); + } regionOnline(a, sn); regionOnline(b, sn); @@ -2799,4 +2811,90 @@ } return true; } + + void setMergeManager(RegionMergeManager mergeManager) { + assert this.mergeManager == null; + this.mergeManager = mergeManager; + } + + public RegionMergeManager getMergeManager() { + return this.mergeManager; + } + + public boolean isRegionInMerging(HRegionInfo regionInfo) { + return this.mergeManager != null + && this.mergeManager.isRegionInMerging(regionInfo); + } + + /** + * Offline the specified region which is merging now + * @param regionInfo + */ + public void offlineMergingRegion(HRegionInfo regionInfo) { + // Region is merging so should not be reassigned, just delete the CLOSED + // node + LOG.debug("Region is in merging so deleting ZK node and removing from " + + "regions in transition, skipping assignment of region " + + regionInfo.getRegionNameAsString()); + try { + if (!ZKAssign.deleteClosedNode(watcher, regionInfo.getEncodedName())) { + // Could also be in OFFLINE mode + ZKAssign.deleteOfflineNode(watcher, regionInfo.getEncodedName()); + } + } catch (KeeperException.NoNodeException nne) { + LOG.debug("Tried to delete closed node for " + regionInfo + " but it " + + "does not exist so just offlining"); + } catch (KeeperException e) { + this.server.abort("Error deleting CLOSED node in ZK for region " + + regionInfo.getEncodedName(), e); + } + regionOffline(regionInfo); + if (this.mergeManager != null) { + this.mergeManager.notifyRegionOffline(regionInfo); + } + } + + /** + * Remove the merging regions from the specified map of regions + * @param regions + */ + private void removeMergingRegions(Map regions) { + if (this.mergeManager != null && this.mergeManager.isAnyRegionInMerging()) { + Iterator iterator = regions.keySet().iterator(); + while (iterator.hasNext()) { + if (this.mergeManager.isRegionInMerging(iterator.next())) { + iterator.remove(); + } + } + } + } + + /** + * Get the merging region for the specified list of regions + * @param regions + * @return list of region which are merging + */ + public List getMergingRegions(List regions) { + List regionsInMerging = new ArrayList(); + if (this.mergeManager != null && this.mergeManager.isAnyRegionInMerging()) { + for (HRegionInfo region : regions) { + if (this.mergeManager.isRegionInMerging(region)) { + regionsInMerging.add(region); + } + } + } + return regionsInMerging; + } + + /** + * Get the merging region num for the specified table + * @param tableName + * @return number of merging regions of given table + */ + public int getNumOfMergingRegionsForTable(String tableName) { + if (this.mergeManager != null) { + return mergeManager.getNumOfMergingRegionsForTable(tableName); + } + return 0; + } } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ServerShutdownHandler.java (working copy) @@ -319,6 +319,11 @@ + " is disabled. Hence not assigning region" + hri.getEncodedName()); return false; } + if (assignmentManager.isRegionInMerging(hri)) { + LOG.info("Region " + hri.getRegionNameAsString() + + " is merging. Hence not assigning it"); + return false; + } return true; } Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTransaction.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTransaction.java (revision 0) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMergeTransaction.java (revision 0) @@ -0,0 +1,605 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.security.SecureRandom; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.LargeTests; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.master.MergeTransaction.JournalState; +import org.apache.hadoop.hbase.master.RegionMergeManager.MergeTransactionData; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.JVMClusterUtil; +import org.apache.hadoop.hbase.util.Pair; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import com.google.common.base.Joiner; + +/** + * Test the {@link MergeTransaction} class against two merging regions. + */ +@Category(LargeTests.class) +public class TestMergeTransaction { + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final Log LOG = LogFactory.getLog(TestMergeTransaction.class); + private static final byte[] FAMILYNAME = Bytes.toBytes("fam"); + private static final byte[] QUALIFIER = Bytes.toBytes("q"); + private static final int slaves = 2; + + private static byte[] ROW = Bytes.toBytes("testRow"); + private static final int INITIAL_REGION_NUM = 10; + private static final int ROWSIZE = 200; + private static byte[][] ROWS = makeN(ROW, ROWSIZE); + + private enum ExceptionLocation { + NONE, + BEFORE_OFFLINE, + BEFORE_EXECUTE, + BEFORE_COMPLETE, + BEFORE_CANCEL, + SET_ZK + } + + // Throw exception in certain step, simulating redo + private static ExceptionLocation exceptionThrownLocation = ExceptionLocation.NONE; + + private static TestingMergeTransaction currentMergeTransaction; + + private static boolean forceEnterCancelState = false; + + private static AtomicInteger successfullyCompletedCount = new AtomicInteger(0); + + @BeforeClass + public static void beforeAllTests() throws Exception { + TEST_UTIL.getConfiguration().setClass( + "hbase.master.merge.transaction.impl", TestingMergeTransaction.class, + MergeTransaction.class); + TEST_UTIL.getConfiguration().setBoolean("hbase.hregion.memstore.mslab.enabled", false); + TEST_UTIL.getConfiguration().setInt("hbase.merge.number.tries", 1); + // Start a cluster of two regionservers. + TEST_UTIL.startMiniCluster(slaves); + } + + @AfterClass + public static void afterAllTests() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @After + public void after() { + currentMergeTransaction = null; + forceEnterCancelState = false; + } + + @Test + public void testWholesomeMerge() throws Exception { + LOG.info("Starting testWholesomeMerge"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testWholesomeMerge"); + HTable table = createTableAndLoadData(master, tablename); + + // Merge 2nd and 3th region + mergeAndVerify(master, tablename, 1, 2, INITIAL_REGION_NUM - 1); + + // Merge 1st and 2nd region + mergeAndVerify(master, tablename, 0, 1, INITIAL_REGION_NUM - 2); + + verifyRowCount(table, ROWSIZE); + + table.close(); + } + + @Test + public void testRedoMergeWhenOfflineRegionEncountersException() + throws Exception { + LOG.info("Starting testRedoMergeWhenOfflineRegion"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testRedoMergeWhenOfflineRegion"); + HTable table = createTableAndLoadData(master, tablename); + + // Throw exception when offline region + exceptionThrownLocation = ExceptionLocation.BEFORE_OFFLINE; + requestMergeRegion(master, tablename, 0, 1); + waitAndVerifyResubmitQueueSize(master, 1); + + exceptionThrownLocation = ExceptionLocation.SET_ZK; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyResubmitQueueSize(master, 1); + + exceptionThrownLocation = ExceptionLocation.NONE; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyRegionNum(master, tablename, INITIAL_REGION_NUM - 1); + verifyRowCount(table, ROWSIZE); + + table.close(); + } + + @Test + public void testRedoMergeWhenExecuteMergeEncountersException() + throws Exception { + LOG.info("Starting testRedoMergeWhenExecuteMerge"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testRedoMergeWhenExecuteMerge"); + HTable table = createTableAndLoadData(master, tablename); + + // Throw exception when execute merge + exceptionThrownLocation = ExceptionLocation.BEFORE_EXECUTE; + requestMergeRegion(master, tablename, 0, 1); + waitAndVerifyResubmitQueueSize(master, 1); + + exceptionThrownLocation = ExceptionLocation.SET_ZK; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyResubmitQueueSize(master, 1); + + exceptionThrownLocation = ExceptionLocation.NONE; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyRegionNum(master, tablename, INITIAL_REGION_NUM - 1); + verifyRowCount(table, ROWSIZE); + table.close(); + } + + @Test + public void testRedoMergeWhenCompleteMergeEncountersException() + throws Exception { + LOG.info("Starting testRedoMergeWhenCompleteMerge"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testRedoMergeWhenCompleteMerge"); + HTable table = createTableAndLoadData(master, tablename); + + // Throw exception when complete merge + exceptionThrownLocation = ExceptionLocation.BEFORE_COMPLETE; + requestMergeRegion(master, tablename, 0, 1); + waitAndVerifyResubmitQueueSize(master, 1); + exceptionThrownLocation = ExceptionLocation.NONE; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyRegionNum(master, tablename, INITIAL_REGION_NUM - 1); + verifyRowCount(table, ROWSIZE); + + table.close(); + } + + @Test + public void testRedoMergeWhenCancelMergeEncountersException() + throws Exception { + LOG.info("Starting testRedoMergeWhenCancelMerge"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testRedoMergeWhenCancelMerge"); + HTable table = createTableAndLoadData(master, tablename); + + // Throw exception when cancel merge + exceptionThrownLocation = ExceptionLocation.BEFORE_CANCEL; + requestMergeRegion(master, tablename, 0, 1); + waitAndVerifyResubmitQueueSize(master, 1); + exceptionThrownLocation = ExceptionLocation.NONE; + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitAndVerifyRegionNum(master, tablename, INITIAL_REGION_NUM); + verifyRowCount(table, ROWSIZE); + + table.close(); + } + + @Test + public void testConcurrentRegionMergingAndRegionSplitting() + throws Exception { + LOG.info("Starting testConcurrentRegionMergingAndRegionSplitting"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + final HMaster master = cluster.getMaster(); + final byte[] tablename = Bytes + .toBytes("testConcurrentRegionMergingAndRegionSplitting"); + final int numRegions = 100; + HTable table = createTableAndLoadData(master, tablename, numRegions); + final List> tableRegions = MetaReader + .getTableRegionsAndLocations(master.getCatalogTracker(), + Bytes.toString(tablename)); + assert tableRegions.size()==numRegions; + + final HBaseAdmin admin = TEST_UTIL.getHBaseAdmin(); + SecureRandom random = new SecureRandom(); + int successfulCountBeforeTest = successfullyCompletedCount.get(); + + final List splitPoints = new ArrayList(); + for (int i = 0; i < numRegions; i++) { + HRegionInfo regionInfo = tableRegions.get(i).getFirst(); + byte[] splitPoint=ROWS[random.nextInt(ROWSIZE / numRegions) + + i * ROWSIZE / numRegions]; + assertTrue(regionInfo.getRegionNameAsString() + " doesn't contain row " + + Bytes.toString(splitPoint), regionInfo.containsRow(splitPoint)); + splitPoints.add(splitPoint); + } + + Thread mergeRequestThread=new Thread(){ + public void run() { + for (int i = 0; i < numRegions; i += 2) { + try { + RegionMergeManager.createMergeRequest(master.getZooKeeper(), + Bytes.toString(tablename), tableRegions.get(i).getFirst() + .getEncodedName(), tableRegions.get(i + 1).getFirst() + .getEncodedName(), false); + } catch (Exception e) { + fail("Failed creating merge request for region " + + tableRegions.get(i).getFirst().getEncodedName() + "and " + + tableRegions.get(i + 1).getFirst().getEncodedName()); + } + } + } + }; + + Thread splitRequestThread = new Thread() { + public void run() { + for (int i = 0; i < numRegions; i++) { + try{ + admin.split(tableRegions.get(i).getFirst().getRegionName(), + splitPoints.get(i)); + } catch (Exception e) { + LOG.info("Region may be in merging, failed calling split, " + + e.getMessage()); + } + + } + } + }; + mergeRequestThread.start(); + splitRequestThread.start(); + mergeRequestThread.join(); + splitRequestThread.join(); + waitUntilNoRegionInMerging(master.assignmentManager.getMergeManager()); + + int successfulCountThisTest = successfullyCompletedCount.get() - successfulCountBeforeTest; + LOG.info("Successfully merge " + successfulCountThisTest * 2 + " regions in testing"); + verifyRowCount(table, ROWSIZE); + table.close(); + } + + /** + * Restart master and regionserver when do merging + * @throws Exception + */ + @Test + public void testRedoMergeWhenServerRestart() throws Exception { + LOG.info("Starting testRedoMergeWhenServerRestart"); + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + byte[] tablename = Bytes.toBytes("testRedoMergeWhenServerRestart"); + HTable table = createTableAndLoadData(master, tablename); + + LOG.info("Test restarting servers after entering state " + + JournalState.CREATE_MERGE); + restartServersAfterExpectedState(table, tablename, + JournalState.CREATE_MERGE); + + LOG.info("Test restarting servers after entering state " + + JournalState.OFFLINE_REGION); + restartServersAfterExpectedState(table, tablename, + JournalState.OFFLINE_REGION); + + LOG.info("Test restarting servers after entering state " + + JournalState.EXECUTE_MERGING); + restartServersAfterExpectedState(table, tablename, + JournalState.EXECUTE_MERGING); + + LOG.info("Test restarting servers after entering state " + + JournalState.COMPLETE_MERGING); + restartServersAfterExpectedState(table, tablename, + JournalState.COMPLETE_MERGING); + + LOG.info("Test restarting servers after entering state " + + JournalState.CANCEL_MERGING); + forceEnterCancelState = true; + restartServersAfterExpectedState(table, tablename, + JournalState.CANCEL_MERGING); + forceEnterCancelState = false; + + table.close(); + } + + private void restartServersAfterExpectedState(final HTable table, + final byte[] tablename, JournalState expectedState) throws Exception { + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + int regionNumBeforeMerging = getTableRegionsNum(master, tablename); + requestMergeRegion(master, tablename, 0, 1); + LOG.info("Restarting servers after entering state " + + JournalState.CREATE_MERGE); + waitUntilEnterExpectedState(expectedState); + restartMasterAndServer(); + // Get master again after restart; + master = cluster.getMaster(); + master.assignmentManager.getMergeManager().triggerResubmissionForTest(); + waitUntilEnterExpectedState(JournalState.END); + + if (currentMergeTransaction.isSuccessful()) { + waitAndVerifyRegionNum(master, tablename, regionNumBeforeMerging - 1); + } + verifyRowCount(table, ROWSIZE); + } + + private void restartMasterAndServer() throws Exception { + MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster(); + HMaster master = cluster.getMaster(); + List liveRSs = cluster + .getLiveRegionServerThreads(); + assert liveRSs.size() == slaves; + LOG.info("Aborting regionserver..."); + SecureRandom random = new SecureRandom(); + HRegionServer rs = liveRSs.get(random.nextInt(slaves)).getRegionServer(); + rs.abort("Aborting for tests", new Exception("Trace info")); + cluster.waitForRegionServerToStop(rs.getServerName(), 15 * 1000); + + List liveMasters = cluster + .getLiveMasterThreads(); + assert liveMasters.size() == 1; + LOG.info("Aborting master..."); + master.abort("Aborting for tests", new Exception("Trace info")); + cluster.waitForMasterToStop(master.getServerName(), 15 * 1000); + + master = cluster.startMaster().getMaster(); + TEST_UTIL.ensureSomeRegionServersAvailable(slaves); + long timeout = System.currentTimeMillis() + (15 * 1000); + while (!master.isInitialized() && System.currentTimeMillis() < timeout) { + Thread.sleep(50); + } + } + + private void waitUntilNoRegionInMerging(final RegionMergeManager regionMergeManager) + throws Exception { + long timeout = System.currentTimeMillis() + (15 * 1000); + while (System.currentTimeMillis() < timeout) { + if (regionMergeManager.getMergingRegions().isEmpty()) { + break; + } + Thread.sleep(50); + } + assertTrue("Timeout waiting merging, regions in merging:" + + regionMergeManager.getMergingRegions(), + regionMergeManager.getMergingRegions().isEmpty()); + } + + private void waitUntilEnterExpectedState(JournalState expectedState) + throws Exception { + long timeout = System.currentTimeMillis() + (15 * 1000); + while (currentMergeTransaction == null + || currentMergeTransaction.currentState.ordinal() < expectedState + .ordinal()) { + Thread.sleep(5); + if (System.currentTimeMillis() > timeout) { + fail("Timeout waiting merge to enter state " + expectedState); + break; + } + } + } + + /** + * An MergeTransaction instance used in this test. + */ + public static class TestingMergeTransaction extends MergeTransaction { + + public TestingMergeTransaction(MasterServices masterServices, + RegionMergeManager mergeManager, MergeTransactionData transactionData, + HRegionInfo mergeA, HRegionInfo mergeB, JournalState initialState, + MonitoredTask status) throws FileNotFoundException, IOException { + super(masterServices, mergeManager, transactionData, mergeA, mergeB, + initialState, status); + currentMergeTransaction = this; + } + + @Override + void offlineRegion(MasterServices masterServices) throws KeeperException, + InterruptedException, IOException { + throwExceptionIfTest(ExceptionLocation.BEFORE_OFFLINE); + super.offlineRegion(masterServices); + } + + @Override + void executeMerging(MasterServices masterServices) throws NoNodeException, + KeeperException, IOException { + throwExceptionIfTest(ExceptionLocation.BEFORE_EXECUTE); + super.executeMerging(masterServices); + } + + @Override + void completeMerging(MasterServices masterServices) throws KeeperException, + IOException, InterruptedException { + throwExceptionIfTest(ExceptionLocation.BEFORE_COMPLETE); + super.completeMerging(masterServices); + successfullyCompletedCount.incrementAndGet(); + } + + @Override + void cancelMerging(MasterServices masterServices) throws NoNodeException, + KeeperException, InterruptedException { + throwExceptionIfTest(ExceptionLocation.BEFORE_CANCEL); + super.cancelMerging(masterServices); + } + + @Override + void prepare() throws IOException, NoNodeException, KeeperException { + super.prepare(); + if (forceEnterCancelState + || exceptionThrownLocation == ExceptionLocation.BEFORE_CANCEL) { + super.currentState = JournalState.CANCEL_MERGING; + super.setJournalStateOnZK(); + } + } + + @Override + protected void setJournalStateOnZK() throws NoNodeException, + KeeperException { + throwExceptionIfTest(ExceptionLocation.SET_ZK); + super.setJournalStateOnZK(); + } + + void throwExceptionIfTest(ExceptionLocation expectedLocation) { + if (exceptionThrownLocation == expectedLocation) { + throw new RuntimeException("Test"); + } + } + + } + + private HTable createTableAndLoadData(HMaster master, byte[] tablename) + throws Exception { + return createTableAndLoadData(master, tablename, INITIAL_REGION_NUM); + } + + private HTable createTableAndLoadData(HMaster master, byte[] tablename, + int numRegions) throws Exception { + assertTrue("ROWSIZE must > numregions:" + numRegions, ROWSIZE > numRegions); + byte[][] splitRows = new byte[numRegions - 1][]; + for (int i = 0; i < splitRows.length; i++) { + splitRows[i] = ROWS[(i + 1) * ROWSIZE / numRegions]; + } + + HTable table = TEST_UTIL.createTable(tablename, FAMILYNAME, splitRows); + loadData(table); + verifyRowCount(table, ROWSIZE); + + // sleep here is an ugly hack to allow region transitions to finish + long timeout = System.currentTimeMillis() + (15 * 1000); + List> tableRegions; + while (System.currentTimeMillis() < timeout) { + tableRegions = MetaReader.getTableRegionsAndLocations( + master.getCatalogTracker(), Bytes.toString(tablename)); + if (tableRegions.size() == numRegions) + break; + Thread.sleep(250); + } + + tableRegions = MetaReader.getTableRegionsAndLocations( + master.getCatalogTracker(), Bytes.toString(tablename)); + LOG.info("Regions after load: " + Joiner.on(',').join(tableRegions)); + assertEquals(numRegions, tableRegions.size()); + return table; + } + + private void loadData(HTable table) throws IOException { + for (int i = 0; i < ROWSIZE; i++) { + Put put = new Put(ROWS[i]); + put.add(FAMILYNAME, QUALIFIER, Bytes.toBytes(i)); + table.put(put); + } + } + + private void verifyRowCount(HTable table, int expectedRegionNum) + throws IOException { + ResultScanner scanner = table.getScanner(new Scan()); + int rowCount = 0; + while (scanner.next() != null) { + rowCount++; + } + assertEquals(expectedRegionNum, rowCount); + scanner.close(); + } + + + private void mergeAndVerify(HMaster master, byte[] tablename, int regionAnum, + int regionBnum, int expectedRegionNum) throws Exception { + requestMergeRegion(master, tablename, regionAnum, regionBnum); + waitAndVerifyRegionNum(master, tablename, expectedRegionNum); + + } + + private void requestMergeRegion(HMaster master, byte[] tablename, + int regionAnum, int regionBnum) throws Exception { + List> tableRegions = MetaReader + .getTableRegionsAndLocations(master.getCatalogTracker(), + Bytes.toString(tablename)); + TEST_UTIL.getHBaseAdmin().mergeRegion(Bytes.toString(tablename), + tableRegions.get(regionAnum).getFirst().getEncodedName(), + tableRegions.get(regionBnum).getFirst().getEncodedName(), false); + } + + private void waitAndVerifyRegionNum(HMaster master, byte[] tablename, + int expectedRegionNum) throws Exception { + List> tableRegions; + long timeout = System.currentTimeMillis() + (15 * 1000); + while (System.currentTimeMillis() < timeout) { + tableRegions = MetaReader.getTableRegionsAndLocations( + master.getCatalogTracker(), Bytes.toString(tablename)); + if (tableRegions.size() == expectedRegionNum)break; + Thread.sleep(250); + } + + tableRegions = MetaReader.getTableRegionsAndLocations( + master.getCatalogTracker(), Bytes.toString(tablename)); + LOG.info("Regions after merge:" + Joiner.on(',').join(tableRegions)); + assertEquals(expectedRegionNum, tableRegions.size()); + } + + private int getTableRegionsNum(HMaster master, byte[] tablename) + throws Exception { + return MetaReader.getTableRegionsAndLocations(master.getCatalogTracker(), + Bytes.toString(tablename)).size(); + } + + private void waitAndVerifyResubmitQueueSize(HMaster master, int expectedSize) + throws Exception { + int queueSize; + long timeout = System.currentTimeMillis() + (15 * 1000); + while (System.currentTimeMillis() < timeout) { + queueSize = master.getAssignmentManager().getMergeManager().resubmitQueusSize(); + if (queueSize == expectedSize) break; + Thread.sleep(250); + } + + queueSize = master.getAssignmentManager().getMergeManager().resubmitQueusSize(); + assertEquals(expectedSize, queueSize); + } + + private static byte[][] makeN(byte[] base, int n) { + byte[][] ret = new byte[n][]; + for (int i = 0; i < n; i++) { + ret[i] = Bytes.add(base, Bytes.toBytes(String.format("%04d", i))); + } + return ret; + } + +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (working copy) @@ -131,6 +131,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest; @@ -269,6 +271,9 @@ // Cluster status zk tracker and local setter private ClusterStatusTracker clusterStatusTracker; + // manager of region merge transactions + private RegionMergeManager regionMergeManager; + // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting // operations/debugging. @@ -504,6 +509,7 @@ if (this.catalogTracker != null) this.catalogTracker.stop(); if (this.serverManager != null) this.serverManager.stop(); if (this.assignmentManager != null) this.assignmentManager.stop(); + if (this.regionMergeManager != null) this.regionMergeManager.stop(); if (this.fileSystemManager != null) this.fileSystemManager.stop(); this.zooKeeper.close(); } @@ -552,6 +558,9 @@ this.catalogTracker, this.balancer, this.executorService, this.metricsMaster); zooKeeper.registerListenerFirst(assignmentManager); + this.regionMergeManager = new RegionMergeManager(this, this.zooKeeper); + this.regionMergeManager.start(); + this.regionServerTracker = new RegionServerTracker(zooKeeper, this, this.serverManager); this.regionServerTracker.start(); @@ -1473,6 +1482,23 @@ } @Override + public MergeRegionResponse mergeRegion(RpcController controller, + MergeRegionRequest req) throws ServiceException { + MergeRegionResponse mrr = MergeRegionResponse.newBuilder().build(); + try{ + LOG.info("Received request to merge region: " + + req.getRegionAEncodedName() + " and " + req.getRegionBEncodedName() + + " in table " + req.getTableName() + " on " + this.serverName); + RegionMergeManager.createMergeRequest(this.zooKeeper, req.getTableName(), + req.getRegionAEncodedName(), req.getRegionBEncodedName(), + req.getForce()); + } catch (KeeperException ke) { + throw new ServiceException(ke); + } + return mrr; + } + + @Override public void createTable(HTableDescriptor hTableDescriptor, byte [][] splitKeys) throws IOException { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java (working copy) @@ -137,6 +137,8 @@ .getTableRegionsAndLocations(this.ct, tableName, true); int countOfRegionsInTable = tableRegionsAndLocations.size(); List regions = regionsToAssignWithServerName(tableRegionsAndLocations); + List regionsInMerging = assignmentManager.getMergingRegions(regions); + regions.removeAll(regionsInMerging); int regionsCount = regions.size(); if (regionsCount == 0) { done = true; @@ -266,7 +268,8 @@ } private boolean isDone(final List regions) { - return regions != null && regions.size() >= this.countOfRegionsInTable; + return regions != null && (regions.size() + assignmentManager + .getNumOfMergingRegionsForTable(tableNameStr)) >= this.countOfRegionsInTable; } } } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMerge.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMerge.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/OnlineMerge.java (revision 0) @@ -0,0 +1,239 @@ +/** + * Copyright The Apache Software Foundation + * + * 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.util; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.ServerName; +import org.apache.hadoop.hbase.catalog.CatalogTracker; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnection; +import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.master.RegionMergeManager; +import org.apache.hadoop.hbase.master.RegionMergeManager.MergeTransactionData; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.util.GenericOptionsParser; +import org.apache.hadoop.util.Tool; +import org.apache.hadoop.util.ToolRunner; + +/** + * Utility that can merge any two regions in the same table: adjacent, + * overlapping or disjoint. Different from {@link Merge}, it needn't shutdown + * cluster or disable table; + * + */ +public class OnlineMerge extends Configured implements Tool { + static final Log LOG = LogFactory.getLog(OnlineMerge.class); + + private String tableName; // Name of table + private String region1EncodedName; // EncodedName of region 1 + private String region2EncodedName; // EncodedName of region 1 + private HRegionInfo region1Info; // Info of region 1 + private HRegionInfo region2Info; // Info of region 1 + private HRegionInfo mergedRegionInfo; // Info of merged region + private HTableDescriptor htd; + private boolean isForce = false; + private boolean sync = true; + private CatalogTracker catalogTracker; + + /** default constructor */ + public OnlineMerge() { + super(); + } + + /** + * @param conf configuration + */ + public OnlineMerge(Configuration conf) { + setConf(conf); + } + + private int parseArgs(String[] args) throws IOException { + GenericOptionsParser parser = new GenericOptionsParser(getConf(), args); + + String[] remainingArgs = parser.getRemainingArgs(); + if (remainingArgs.length != 3) { + usage(); + return -1; + } + tableName = remainingArgs[0]; + region1EncodedName = remainingArgs[1]; + region2EncodedName = remainingArgs[2]; + int status = 0; + if (region1EncodedName.equals(region2EncodedName)) { + LOG.error("Can't merge a region with itself"); + status = -1; + } + return status; + } + + private void usage() { + System.err.println("Usage: bin/hbase org.apache.hadoop.hbase.util.OnlineMerge" + + " [-force] [-async] [-show] " + + " \n"); + } + + @SuppressWarnings("deprecation") + public int run(String[] args) throws Exception { + if (parseArgs(args) != 0) { + return -1; + } + HBaseAdmin hbaseAdmin = new HBaseAdmin(getConf()); + HConnection connection = HConnectionManager.getConnection(getConf()); + ZooKeeperWatcher zkw = connection.getZooKeeperWatcher(); + try { + if (sync) { + this.htd = connection.getHTableDescriptor(Bytes.toBytes(tableName)); + if (htd == null) { + System.err.println("Couldn't get table descriptor"); + return -1; + } + this.catalogTracker = new CatalogTracker(zkw, getConf(), connection); + this.catalogTracker.start(); + this.initRegionInfoByEncodedName(catalogTracker); + if (this.mergedRegionInfo == null) { + System.err.println("Couldn't get region info by encoded name"); + return -1; + } + } + String transactionName = MergeTransactionData.generateTransactionName( + tableName, region1EncodedName, region2EncodedName, isForce); + System.out.println("Send merge request " + transactionName); + hbaseAdmin.mergeRegion(tableName, region1EncodedName, region2EncodedName, + isForce); + if (sync) { + String transactionNode = ZKUtil.joinZNode(zkw.mergeZNode, transactionName); + System.out.print("Waiting for completion..."); + while (true) { + boolean existed = ZKUtil.watchAndCheckExists(zkw, transactionNode); + if (!existed) { + break; + } else { + System.out.print("."); + Thread.sleep(1000); + } + } + System.out.println(""); + Pair mergedRegionLocation = MetaReader + .getRegion(catalogTracker, mergedRegionInfo.getRegionName()); + if (mergedRegionLocation != null) { + System.out.println("Complete merging, new region =" + + mergedRegionInfo.getRegionNameAsString() + + ", assigned on the server " + mergedRegionLocation.getSecond()); + } else { + System.err.println("Merge is cancelled, please see the logs on the master"); + } + } + } finally { + if (this.catalogTracker != null) { + this.catalogTracker.stop(); + } + connection.close(); + } + return 0; + } + + private void setForce(boolean force) { + this.isForce = force; + } + + private void setSync(boolean sync) { + this.sync = sync; + } + + private void initRegionInfoByEncodedName(CatalogTracker ct) + throws IOException { + if (tableName != null && (region1Info == null || region2Info == null)) { + List tableRegions = MetaReader.getTableRegions(ct, + Bytes.toBytes(tableName)); + for (HRegionInfo regionInfo : tableRegions) { + if (regionInfo.getEncodedName().equals(region1EncodedName)) { + region1Info = regionInfo; + } else if (regionInfo.getEncodedName().equals(region2EncodedName)) { + region2Info = regionInfo; + } + if (region1Info != null && region2Info != null) { + this.mergedRegionInfo = HRegion.generateFixedRegionByMerge(this.htd, + this.region1Info, this.region2Info); + break; + } + } + } + } + + static void showMergeRequests() throws Exception { + HConnection connection = HConnectionManager.getConnection(HBaseConfiguration.create()); + try { + ZooKeeperWatcher zkw = connection.getZooKeeperWatcher(); + List mergingRequests = ZKUtil.listChildrenNoWatch(zkw, + zkw.mergeZNode); + System.out.println("Handling merge requests:"); + for (String mergeRequest : mergingRequests) { + System.out.println(mergeRequest); + } + } finally { + connection.close(); + } + } + + public static void main(String[] args) { + int status; + OnlineMerge onlineMerge = new OnlineMerge(); + try { + while (true) { + if (args.length >= 1 && "-force".equals(args[0])) { + String[] newArgs = new String[args.length - 1]; + for (int i = 1; i < args.length; i++) { + newArgs[i - 1] = args[i]; + } + args = newArgs; + onlineMerge.setForce(true); + } else if (args.length >= 1 && "-async".equals(args[0])) { + String[] newArgs = new String[args.length - 1]; + for (int i = 1; i < args.length; i++) { + newArgs[i - 1] = args[i]; + } + args = newArgs; + onlineMerge.setSync(false); + } else if (args.length >= 1 && "-show".equals(args[0])) { + showMergeRequests(); + System.exit(0); + } else { + break; + } + } + status = ToolRunner.run(HBaseConfiguration.create(), onlineMerge, args); + } catch (Exception e) { + LOG.error("exiting due to error", e); + status = -1; + } + System.exit(status); + } +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/CatalogJanitor.java (working copy) @@ -36,13 +36,17 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Chore; import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HTableDescriptor; import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.backup.HFileArchiver; import org.apache.hadoop.hbase.catalog.MetaEditor; import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.MetaScanner; import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitor; +import org.apache.hadoop.hbase.client.MetaScanner.MetaScannerVisitorBase; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.util.Bytes; @@ -110,6 +114,20 @@ * an ordered map of split parents. */ Pair> getSplitParents() throws IOException { + return getSplitParents(null); + } + + /** + * Scans META and returns a number of scanned rows, and an ordered map of + * split parents. if the given table name is null, return split parents of all + * tables, else only the specified table + * @param tableName null represents all tables + * @return pair of scanned rows, and map of split parent regioninfos + * @throws IOException + */ + Pair> getSplitParents(final byte[] tableName) + throws IOException { + final boolean isTableSpecified = (tableName != null && tableName.length != 0); // TODO: Only works with single .META. region currently. Fix. final AtomicInteger count = new AtomicInteger(0); // Keep Map of found split parents. There are candidates for cleanup. @@ -124,13 +142,22 @@ count.incrementAndGet(); HRegionInfo info = HRegionInfo.getHRegionInfo(r); if (info == null) return true; // Keep scanning + if (isTableSpecified && Bytes.compareTo(info.getTableName(), tableName) > 0) { + // Another table, stop scanning + return false; + } if (info.isSplitParent()) splitParents.put(info, r); // Returning true means "keep scanning" return true; } }; - // Run full scan of .META. catalog table passing in our custom visitor - MetaReader.fullScan(this.server.getCatalogTracker(), visitor); + + byte[] startRow = (!isTableSpecified) ? HConstants.EMPTY_START_ROW + : HRegionInfo.createRegionName(tableName, HConstants.EMPTY_START_ROW, + HConstants.ZEROES, false); + // Run full scan of .META. catalog table passing in our custom visitor with + // the start row + MetaReader.fullScan(this.server.getCatalogTracker(), visitor, startRow); return new Pair>(count.get(), splitParents); } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java (working copy) @@ -4313,6 +4313,121 @@ return dstRegion; } + /** + * Generate a fixed region info when merging + * @param tabledesc + * @param a merging region A + * @param b merging region B + * @return the merged region info + * @throws IOException + */ + public static HRegionInfo generateFixedRegionByMerge( + final HTableDescriptor tabledesc, final HRegionInfo a, final HRegionInfo b) + throws IOException { + if (!a.getTableNameAsString().equals(b.getTableNameAsString())) { + throw new IllegalArgumentException("Regions do not belong to the same table"); + } + // Presume both are of same region type -- i.e. both user or catalog + // table regions. This way can use comparator. + final byte[] startKey = (a.getComparator().matchingRows(a.getStartKey(), 0, + a.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0, + HConstants.EMPTY_BYTE_ARRAY.length) || b.getComparator().matchingRows( + b.getStartKey(), 0, b.getStartKey().length,HConstants.EMPTY_BYTE_ARRAY, 0, + HConstants.EMPTY_BYTE_ARRAY.length)) ? HConstants.EMPTY_BYTE_ARRAY : + (a.getComparator().compareRows(a.getStartKey(), 0,a.getStartKey().length, + b.getStartKey(), 0, b.getStartKey().length) <= 0 ? a.getStartKey() : b.getStartKey()); + final byte[] endKey = (a.getComparator().matchingRows(a.getEndKey(), 0, + a.getEndKey().length, HConstants.EMPTY_BYTE_ARRAY, 0, + HConstants.EMPTY_BYTE_ARRAY.length) || a.getComparator().matchingRows( + b.getEndKey(), 0, b.getEndKey().length, HConstants.EMPTY_BYTE_ARRAY, 0, + HConstants.EMPTY_BYTE_ARRAY.length)) ? HConstants.EMPTY_BYTE_ARRAY : (a + .getComparator().compareRows(a.getEndKey(), 0, a.getEndKey().length, + b.getEndKey(), 0, b.getEndKey().length) <= 0 ? b.getEndKey() : a + .getEndKey()); + + HRegionInfo newRegionInfo = new HRegionInfo(tabledesc.getName(), startKey, + endKey, false, Math.max(a.getRegionId(), b.getRegionId()) + 1); + return newRegionInfo; + } + + /** + * Merge two regions whether they are adjacent or not. + * @param conf + * @param fs + * @param rootDir + * @param tabledesc + * @param a merging region A + * @param b merging region B + * @return the merged region info + * @throws IOException + */ + public static HRegionInfo merge(final Configuration conf, + final FileSystem fs, final Path rootDir, + final HTableDescriptor tabledesc, final HRegionInfo a, final HRegionInfo b) + throws IOException { + HRegionInfo newRegionInfo = generateFixedRegionByMerge(tabledesc, a, b); + Path tableDir = HTableDescriptor.getTableDir(rootDir, a.getTableName()); + String newRegionEncodedName = newRegionInfo.getEncodedName(); + Path newRegionDir = HRegion.getRegionDir(tableDir, newRegionEncodedName); + if (!fs.exists(newRegionDir)) { + if (!fs.mkdirs(newRegionDir)) { + throw new IOException("Failed making dirs " + newRegionDir); + } + } + + LOG.info("Starting merge of regions on fs : " + a.getRegionNameAsString() + + " and " + b.getRegionNameAsString() + " into new region " + + newRegionInfo.getRegionNameAsString() + " with start key <" + + Bytes.toStringBinary(newRegionInfo.getStartKey()) + "> and end key <" + + Bytes.toStringBinary(newRegionInfo.getEndKey()) + ">"); + + // Move HStoreFiles under new region directory + Map> byFamily = new TreeMap>( + Bytes.BYTES_COMPARATOR); + HRegionInfo[] regionInfos = new HRegionInfo[] { a, b }; + for (HRegionInfo hri : regionInfos) { + for (HColumnDescriptor family : tabledesc.getFamilies()) { + Path familyDir = HStore.getStoreHomedir(tableDir, hri.getEncodedName(), + family.getName()); + FileStatus files[] = fs.listStatus(familyDir); + if (files == null || files.length == 0) continue; + List familyFileList = byFamily.get(family.getName()); + if (familyFileList == null) { + familyFileList = new ArrayList(); + byFamily.put(family.getName(), familyFileList); + } + for (int i = 0; i < files.length; i++) { + if (StoreFile.isReference(files[i].getPath())) { + throw new IOException("Reference file still exists:" + + files[i].getPath()); + } + familyFileList.add(files[i].getPath()); + } + } + } + + for (Map.Entry> es : byFamily.entrySet()) { + byte[] colFamily = es.getKey(); + makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily); + // Because we compacted the source regions we should have no more than two + // HStoreFiles per family and there will be no reference store + List srcFiles = es.getValue(); + for (Path hsf : srcFiles) { + fs.rename(hsf, StoreFile.getUniqueFile(fs, HStore.getStoreHomedir( + tableDir, newRegionInfo.getEncodedName(), colFamily))); + } + } + + // Archive out the 'A' region + HFileArchiver.archiveRegion(conf, fs, a); + // Archive out the 'B' region + HFileArchiver.archiveRegion(conf, fs, b); + + LOG.info("Merge completed on the fs. New region is " + + newRegionInfo.getRegionNameAsString()); + return newRegionInfo; + } + /* * Fills a map with a vector of store files keyed by column family. * @param byFamily Map to fill. Index: hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (revision 1433297) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java (working copy) @@ -985,6 +985,23 @@ /** * Create a table. * @param tableName + * @param family + * @param splitRows + * @return An HTable instance for the created table. + * @throws IOException + */ + public HTable createTable(byte[] tableName, byte[] family, byte[][] splitRows) + throws IOException { + HTableDescriptor desc = new HTableDescriptor(tableName); + HColumnDescriptor hcd = new HColumnDescriptor(family); + desc.addFamily(hcd); + getHBaseAdmin().createTable(desc, splitRows); + return new HTable(getConfiguration(), tableName); + } + + /** + * Create a table. + * @param tableName * @param families * @param c Configuration to use * @param numVersions Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/ClosedRegionHandler.java (working copy) @@ -92,6 +92,11 @@ @Override public void process() { LOG.debug("Handling CLOSED event for " + regionInfo.getEncodedName()); + // Check if this region is in merging or not + if (this.assignmentManager.isRegionInMerging(this.regionInfo)) { + assignmentManager.offlineMergingRegion(regionInfo); + return; + } // Check if this table is being disabled or not if (this.assignmentManager.getZKTable(). isDisablingOrDisabledTable(this.regionInfo.getTableNameAsString())) { Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionMergeManager.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionMergeManager.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/RegionMergeManager.java (revision 0) @@ -0,0 +1,728 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionException; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.catalog.MetaReader; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.master.MergeTransaction.JournalState; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.zookeeper.ZKTable; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.KeeperException.NoNodeException; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +/** + * Receive and handle merge request via zk, manage merge transaction and merging + * region list, resubmit the failed merges by a period + */ +public class RegionMergeManager extends ZooKeeperListener { + private static final Log LOG = LogFactory.getLog(RegionMergeManager.class); + + private MasterServices master; + private final AssignmentManager assignmentManager; + private final CatalogJanitor catalogJanitor; + private final Configuration conf; + + // A set used to store the regions which are merging; A region in merging + // means it's disabled, isn't online anywhere and won't be assigned any more; + private final ConcurrentSkipListSet mergingRegions; + // Set of region which are already split; Only add to this set if it is in + // mergingRegions + private final ConcurrentSkipListSet splitRegions; + + // Map of merge transaction in processing where the key is transaction name + private final ConcurrentHashMap processingMergeTransaction; + + // Queue of merge transactions which are waiting for resubmit + private final BlockingQueue resubmissionQueue; + + /** Resubmit thread schedule pool */ + private ScheduledExecutorService resubmitSchedulePool; + // Running interval of resubmit thread, 60s as default + private final int resubmitThreadPeriod; + static final int DEFAULT_RESUBMIT_PERIOD = 60; + + // Used for wait signal + private final Object present_object = new Object(); + + private final int timePerSleep = 10; + // 120s + private static final int DEFAULT_TIME_OUT = 120000; + private final int timeout; + /** + * If it is the first time we called {@link RegionMergeManager#fetchMergeRequest()}, + * we should make sure old merge transactions handled first + */ + private boolean isFirstFetchRequest = true; + + // Thread pool used to run merge request + private ExecutorService mergesPool; + private final int mergeThreads; + + /** + * Constructs a new merge manager. + * @param master + * @param watcher + * @throws KeeperException + * @throws IOException + */ + public RegionMergeManager(final MasterServices master, ZooKeeperWatcher watcher) + throws KeeperException, IOException { + super(watcher); + this.master = master; + this.conf = master.getConfiguration(); + this.timeout = conf.getInt("hbase.merge.wait.timeout.millis", DEFAULT_TIME_OUT); + this.catalogJanitor = new CatalogJanitor(master, master); + this.assignmentManager = master.getAssignmentManager(); + this.mergingRegions = new ConcurrentSkipListSet(); + this.splitRegions = new ConcurrentSkipListSet(); + this.processingMergeTransaction = new ConcurrentHashMap(); + this.resubmissionQueue = new LinkedBlockingQueue(); + this.mergeThreads = conf.getInt("hbase.merge.thread.count", 1); + this.resubmitThreadPeriod = conf.getInt("hbase.merge.resubmit.period", + DEFAULT_RESUBMIT_PERIOD); + this.assignmentManager.setMergeManager(this); + + } + + /** + * Submit the merge request to the pool + * @param transactionData + * @param initialState + * @throws IOException + */ + private void requestMerge(MergeTransactionData transactionData, + JournalState initialState) { + try { + this.mergesPool.execute(new MergeRequest(this, transactionData, + initialState, master)); + if (LOG.isDebugEnabled()) { + LOG.debug("Merge requested for " + transactionData + ",initialState= " + + initialState); + } + } catch (RejectedExecutionException ree) { + LOG.warn("Could not execute merge for " + transactionData, ree); + } + } + + /** + * Handle the merge transaction data from ZK + * @param transactionData + * @throws KeeperException + * @throws IOException + */ + synchronized private void handleMergeTransactionData( + MergeTransactionData transactionData) throws KeeperException { + String regionA = transactionData.getRegionA(); + String regionB = transactionData.getRegionB(); + if (mergingRegions.contains(regionA) || mergingRegions.contains(regionB)) { + LOG.warn("Merge request " + transactionData + " try to merge region " + + (mergingRegions.contains(regionA) ? regionA : regionB) + + "which is already in merging"); + finishMergeTransaction(transactionData, false); + } else if (regionA.equals(regionB)) { + LOG.warn("Error merge request:" + transactionData + + ",shouldn't merge the same region"); + finishMergeTransaction(transactionData, false); + } else { + String transactionNode = ZKUtil.joinZNode(watcher.mergeZNode, + transactionData.getTransactionName()); + byte[] data = ZKUtil.getData(watcher, transactionNode); + JournalState initialState = MergeTransaction + .parseJournalStateFromBytes(data); + mergingRegions.add(regionA); + mergingRegions.add(regionB); + requestMerge(transactionData, initialState); + } + } + + HRegionInfo getRegionInfoByEncodedName(String tableName, + String regionEncodeName) throws IOException { + RegionState regionState = this.assignmentManager.getRegionStates() + .getRegionState(regionEncodeName); + if (regionState != null) { + return regionState.getRegion(); + } + List tableRegions = MetaReader.getTableRegions( + this.master.getCatalogTracker(), Bytes.toBytes(tableName)); + for (HRegionInfo regionInfo : tableRegions) { + if (regionInfo.getEncodedName().equals(regionEncodeName)) { + return regionInfo; + } + } + return null; + } + + /** + * Starts the thread pool and the tracking of merge request. + * + * @throws KeeperException + * @throws IOException + */ + public void start() throws KeeperException, IOException { + this.mergesPool = Executors.newFixedThreadPool( + mergeThreads, + new ThreadFactoryBuilder() + .setNameFormat("MergeManager-Merge-Thread #%d").setDaemon(true) + .build()); + + this.resubmitSchedulePool = Executors.newScheduledThreadPool( + 1, + new ThreadFactoryBuilder() + .setNameFormat("MergeManager-Resubmit-Thread #%d").setDaemon(true) + .build()); + resubmitSchedulePool.scheduleAtFixedRate(new ResubmitThread(this, + this.resubmissionQueue), resubmitThreadPeriod, resubmitThreadPeriod, + TimeUnit.SECONDS); + watcher.registerListener(this); + fetchMergeRequest(); + } + + /** + * Shutdown the threadpools + */ + public void stop() { + if (mergesPool != null) { + this.mergesPool.shutdown(); + this.mergesPool = null; + } + if (this.resubmitSchedulePool != null) { + this.resubmitSchedulePool.shutdown(); + this.resubmitSchedulePool = null; + } + } + + /** + * Thread used to resubmit failed merge transaction + */ + private static class ResubmitThread extends Thread { + RegionMergeManager mergeManager; + BlockingQueue resubmitQueue; + + public ResubmitThread(RegionMergeManager mergeManager, + BlockingQueue resubmitQueue) { + super("MergeManager.ResubmitThread"); + this.mergeManager = mergeManager; + this.resubmitQueue = resubmitQueue; + setDaemon(true); + } + + @Override + public void run() { + MergeTransactionData transactionData = null; + try { + while ((transactionData = resubmitQueue.poll()) != null) { + String transactionNode = ZKUtil.joinZNode( + mergeManager.watcher.mergeZNode, + transactionData.getTransactionName()); + byte[] data = ZKUtil.getData(mergeManager.watcher, transactionNode); + JournalState initialState = MergeTransaction + .parseJournalStateFromBytes(data); + mergeManager.requestMerge(transactionData, initialState); + } + } catch (KeeperException e) { + LOG.warn("Failed resubmit transaction " + transactionData, e); + if (transactionData != null) { + mergeManager.addToResubmitQueue(transactionData); + } + } + } + } + + /** + * Fetch the merge request via zk + * @throws KeeperException + * @throws IOException + */ + synchronized private void fetchMergeRequest() throws KeeperException { + List mergeTransactions = ZKUtil.listChildrenAndWatchForNewChildren( + watcher, watcher.mergeZNode); + if (isFirstFetchRequest) { + isFirstFetchRequest = false; + List oldMergeTransactions = new ArrayList(); + List newMergeTransactions = new ArrayList(); + for (String transactionName : mergeTransactions) { + String transactionNode = ZKUtil.joinZNode(watcher.mergeZNode, + transactionName); + byte[] data = ZKUtil.getData(watcher, transactionNode); + JournalState initialState = MergeTransaction + .parseJournalStateFromBytes(data); + if (initialState != JournalState.CREATE_MERGE) { + oldMergeTransactions.add(transactionName); + } else { + newMergeTransactions.add(transactionName); + } + } + mergeTransactions.clear(); + mergeTransactions.addAll(oldMergeTransactions); + mergeTransactions.addAll(newMergeTransactions); + } + for (String transactionName : mergeTransactions) { + if (!MergeTransactionData.isValidTransactionName(transactionName)) { + LOG.warn("Invalid merge transaction name=" + transactionName + + ", clean it"); + ZKUtil.deleteNodeRecursively(watcher, + ZKUtil.joinZNode(watcher.mergeZNode, transactionName)); + } else if (this.processingMergeTransaction.containsKey(transactionName)) { + continue; + } else { + MergeTransactionData transactionData = new MergeTransactionData( + transactionName); + this.processingMergeTransaction.put(transactionName, transactionData); + handleMergeTransactionData(transactionData); + } + } + } + + /** + * Finish the merge transaction: 1.delete the transaction zk node; 2.remove + * the region in set if it is called after we have submit the request + * @param transactionData + * @param successfulSubmission true if it is called by the MergeRequest thread + * @throws KeeperException + */ + void finishMergeTransaction(MergeTransactionData transactionData, + boolean successfulSubmission) throws KeeperException { + ZKUtil.deleteNodeRecursively(watcher, + ZKUtil.joinZNode(watcher.mergeZNode, + transactionData.getTransactionName())); + this.processingMergeTransaction + .remove(transactionData.getTransactionName()); + if (successfulSubmission) { + String regionA = transactionData.getRegionA(); + String regionB = transactionData.getRegionB(); + this.mergingRegions.remove(regionA); + this.mergingRegions.remove(regionB); + this.splitRegions.remove(regionA); + this.splitRegions.remove(regionB); + } + } + + /** + * Add to the resubmit queue + * @param transactionData + */ + void addToResubmitQueue(MergeTransactionData transactionData) { + this.resubmissionQueue.add(transactionData); + } + + /** + * Used in test + * @return resubmit queue size + */ + int resubmitQueusSize() { + return this.resubmissionQueue.size(); + } + + /** + * Used in test + * @throws InterruptedException + */ + void triggerResubmissionForTest() throws InterruptedException { + ResubmitThread resubmitThread = new ResubmitThread(this, + this.resubmissionQueue); + resubmitThread.start(); + resubmitThread.join(); + } + + /** + * Assign the merging region if it is offline, called when cancelling the + * merge + * @param regionInfo + */ + void assignMergingRegionIfOffline(HRegionInfo regionInfo) { + String tableName = regionInfo.getTableNameAsString(); + ZKTable zkTable = assignmentManager.getZKTable(); + if (zkTable.isTablePresent(tableName) + && !zkTable.isDisablingOrDisabledTable(tableName)) { + if (isRegionOffline(regionInfo) && !isRegionSplit(regionInfo)) { + assignmentManager.assign(regionInfo, true); + } + } + this.mergingRegions.remove(regionInfo.getEncodedName()); + } + + /** + * Assign the merged region if it is offline, called after adding merged + * region to META + * @param regionInfo + * @throws InterruptedException + */ + void assignMergedRegionIfOffline(HRegionInfo mergedRegionInfo) + throws InterruptedException { + String tableName = mergedRegionInfo.getTableNameAsString(); + ZKTable zkTable = assignmentManager.getZKTable(); + waitUntilTableNotInEnabling(zkTable, tableName); + if (zkTable.isTablePresent(tableName) + && !zkTable.isDisablingOrDisabledTable(tableName)) { + if (isRegionOffline(mergedRegionInfo)) { + assignmentManager.assign(mergedRegionInfo, true); + } + } + } + + /** + * Wait until two regions are offline + * @param regionA + * @param regionB + * @throws InterruptedException + */ + void waitUntilRegionsOffline(HRegionInfo regionA, HRegionInfo regionB) + throws InterruptedException { + long expireTime = EnvironmentEdgeManager.currentTimeMillis() + this.timeout; + while (!this.master.isStopped() && EnvironmentEdgeManager.currentTimeMillis() < expireTime) { + if (isRegionOffline(regionA) && isRegionOffline(regionB)) { + return; + } else { + synchronized (present_object) { + present_object.wait(timePerSleep); + } + } + } + throw new RuntimeException("Timeout waiting region offline"); + } + + /** + * Wait until master is initialized + * @throws InterruptedException + */ + void waitUntilMasterInitialized() throws InterruptedException { + while (!this.master.isStopped()) { + if (((HMaster) this.master).isInitialized()) { + return; + } + Thread.sleep(timePerSleep); + } + throw new RuntimeException("Timeout waiting master initialized"); + } + + /** + * Wait until no dead server in process + * @param masterServices + * @throws InterruptedException + */ + void waitUntilNoDeadServerInProcess(MasterServices masterServices) + throws InterruptedException { + while (!this.master.isStopped()) { + if (!masterServices.getServerManager().areDeadServersInProgress()) { + return; + } + Thread.sleep(timePerSleep); + } + throw new RuntimeException("Timeout waiting no deadserver in process"); + } + + /** + * Wait until specified table not in enabling state + * @param zkTable + * @param tableName + * @throws InterruptedException + */ + void waitUntilTableNotInEnabling(ZKTable zkTable, String tableName) + throws InterruptedException { + long expireTime = EnvironmentEdgeManager.currentTimeMillis() + this.timeout; + while (!this.master.isStopped() && EnvironmentEdgeManager.currentTimeMillis() < expireTime) { + if (!zkTable.isEnablingTable(tableName)) { + return; + } + Thread.sleep(timePerSleep); + } + throw new RuntimeException("Timeout waiting table " + tableName + " not in enabling"); + } + + /** + * Check if the merging region is split + * @param regionInfo + * @return + */ + boolean isRegionSplit(HRegionInfo regionInfo) { + if (regionInfo.isSplitParent()) { + return true; + } + if (this.splitRegions.contains(regionInfo.getEncodedName())) { + return true; + } + return false; + } + + /** + * Check if the merging region is offline + * @param regionInfo + * @return + */ + boolean isRegionOffline(HRegionInfo regionInfo) { + if (!this.assignmentManager.getRegionStates().isRegionInTransition(regionInfo) + && this.assignmentManager.getRegionStates().getRegionServerOfRegion(regionInfo) == null) { + return true; + } + return false; + } + + /** + * Record the transaction's JournalState on the ZK + * @param transactionName + * @param state + * @throws NoNodeException + * @throws KeeperException + */ + void setJournalStateOnZK(String transactionName, JournalState state) + throws NoNodeException, KeeperException { + String transactionNode = ZKUtil.joinZNode(watcher.mergeZNode, + transactionName); + ZKUtil.setData(watcher, transactionNode, + MergeTransaction.getBytesFromJournalState(state)); + } + + /** + * Clean parent region of merging regions in prepare step + * @param parent + * @param rowContent + * @return + * @throws IOException + */ + boolean cleanParent(final HRegionInfo parent, Result rowContent) + throws IOException { + return this.catalogJanitor.cleanParent(parent, rowContent); + } + + /** + * Get split parents region for a specified table + * @param tableName + * @return map of split parent regioninfos + */ + Map getSplitParents(final byte[] tableName) + throws IOException { + return this.catalogJanitor.getSplitParents(tableName).getSecond(); + } + + /** + * Check whether a region is in merging, if true it shouldn't be assigned + * @param regionInfo + * @return true if region in the merging + */ + boolean isRegionInMerging(final HRegionInfo regionInfo) { + return this.mergingRegions.contains(regionInfo.getEncodedName()); + } + + /** + * Check if any regions are merging + * @return true if any regions in merging + */ + boolean isAnyRegionInMerging() { + return !this.mergingRegions.isEmpty(); + } + + public Set getMergingRegions() { + Set clone = new HashSet(this.mergingRegions.size()); + clone.addAll(this.mergingRegions); + return clone; + } + + /** + * Get the merging region num for the specified table + * @param tableName + * @return number of merging regions of given table + */ + public int getNumOfMergingRegionsForTable(String tableName) { + int num = 0; + for (MergeTransactionData transactionData : processingMergeTransaction + .values()) { + if (tableName.equals(transactionData.getTableName())) { + num += 2; + } + } + return num; + } + + /** + * AssignmentManager notifies MergeManager one region is offline + * @param regionInfo + */ + void notifyRegionOffline(final HRegionInfo regionInfo) { + if (this.mergingRegions.contains(regionInfo.getEncodedName())) { + synchronized (present_object) { + present_object.notify(); + } + } + } + + /** + * AssignmentManager notify MergeManager one region is split + * @param regionInfo + */ + void notifyRegionSplit(final HRegionInfo regionInfo) { + if (this.mergingRegions.contains(regionInfo.getEncodedName())) { + this.splitRegions.add(regionInfo.getEncodedName()); + synchronized (present_object) { + present_object.notifyAll(); + } + } + } + + // ZooKeeper events + + /** + * New transaction node has been created or deleted. + *

+ * When this happens we must: + *

    + *
  1. Find the new merge request and handle it
  2. + *
+ */ + @Override + public void nodeChildrenChanged(String path) { + if (path.equals(watcher.mergeZNode)) { + try { + LOG.debug("Merge request arrived or completed "); + fetchMergeRequest(); + } catch (KeeperException e) { + LOG.error("Unexpected ZK exception handling merge requests", e); + } catch (Throwable t) { + LOG.error("Unhandled exception: " + t.getMessage(), t); + } + } + } + + public static class MergeTransactionData { + public static final String SEPARATOR = "#"; + private final String transactionName; + private final String tableName; + private final String regionA; + private final String regionB; + private final boolean isForce; + + MergeTransactionData(String transactionName) { + this.transactionName = transactionName; + String[] elements = transactionName.split(SEPARATOR); + assert elements.length == 4; + this.tableName = elements[0]; + this.regionA = elements[1]; + this.regionB = elements[2]; + this.isForce = Boolean.parseBoolean(elements[3]); + } + + String getTransactionName() { + return this.transactionName; + } + + String getTableName() { + return this.tableName; + } + + String getRegionA() { + return this.regionA; + } + + String getRegionB() { + return this.regionB; + } + + boolean isForce() { + return this.isForce; + } + + @Override + public String toString() { + return "table=" + tableName + ", regionA=" + regionA + ", regionB=" + + regionB + ", force=" + isForce; + } + + /** + * Valid TransactionName=tableName+'#'+AEncodedName+"#"+BEncodedName+"#"+ + * "true/false" + * @param transactionName + * @return true if it is valid + */ + public static boolean isValidTransactionName(String transactionName) { + String[] elements = transactionName.split(SEPARATOR); + if (elements.length != 4) { + return false; + } + if (elements[1].length() != elements[2].length()) { + return false; + } + if (elements[1].equals(elements[2])) { + return false; + } + if (!elements[3].equals("true") && !elements[3].equals("false")) { + return false; + } + return true; + } + + public static String generateTransactionName(String tableName, + String regionAEncodedName, String regionBEncodedName, boolean force) { + return tableName + SEPARATOR + regionAEncodedName + SEPARATOR + + regionBEncodedName + SEPARATOR + force; + } + } + + /** + * Create a merge request via zk.Set force true if merge two non-adjacent + * regions, else the request will be skipped + * @param zkw + * @param tableName + * @param regionAEncodedName + * @param regionBEncodedName + * @param force if it is false, only merge two adjacent regions + * @throws KeeperException + */ + static void createMergeRequest(ZooKeeperWatcher zkw, String tableName, + String regionAEncodedName, String regionBEncodedName, boolean force) + throws KeeperException { + String transactionName = MergeTransactionData.generateTransactionName( + tableName, regionAEncodedName, regionBEncodedName, force); + if (MergeTransactionData.isValidTransactionName(transactionName)) { + String transactionNode = ZKUtil + .joinZNode(zkw.mergeZNode, transactionName); + ZKUtil.createAndNoWatch(zkw, transactionNode, + MergeTransaction.getBytesFromJournalState(JournalState.CREATE_MERGE)); + } else { + throw new IllegalArgumentException("Unknown merge parameters, table=" + + tableName + ", regionA=" + regionAEncodedName + ", regionB=" + + regionBEncodedName); + } + + } +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (working copy) @@ -91,6 +91,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest; @@ -917,6 +918,25 @@ } /** + * Create a protocol buffer MergeRegionRequest + * @param tableName + * @param regionAEncodedName + * @param regionBEncodedName + * @param force + * @return A MergeRegionRequest + */ + public static MergeRegionRequest buildMergeRegionRequest( + final String tableName, final String regionAEncodedName, + final String regionBEncodedName, final boolean force) { + MergeRegionRequest.Builder builder = MergeRegionRequest.newBuilder(); + builder.setTableName(tableName); + builder.setRegionAEncodedName(regionAEncodedName); + builder.setRegionBEncodedName(regionBEncodedName); + builder.setForce(force); + return builder.build(); + } + + /** * Create a protocol buffer AssignRegionRequest * * @param regionName Index: hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (working copy) @@ -103,6 +103,8 @@ public String splitLogZNode; // znode containing the state of the load balancer public String balancerZNode; + // znode used for merge + public String mergeZNode; // Certain ZooKeeper nodes need to be world-readable public static final ArrayList CREATOR_ALL_AND_WORLD_READABLE = @@ -166,6 +168,7 @@ ZKUtil.createAndFailSilent(this, tableZNode); ZKUtil.createAndFailSilent(this, splitLogZNode); ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode); + ZKUtil.createAndFailSilent(this, mergeZNode); } catch (KeeperException e) { throw new ZooKeeperConnectionException( prefix("Unexpected KeeperException creating base node"), e); @@ -215,6 +218,8 @@ conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME)); balancerZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer")); + mergeZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.merge", "merge")); } /** Index: hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java (working copy) @@ -1051,6 +1051,35 @@ } /** + * Creates the specified node with the specified data . Does not set watch + * + *

+ * Throws an exception if the node already exists. + * + *

+ * The node created is persistent and open access. + * + *

+ * Returns the version number of the created node if successful. + * + * @param zkw zk reference + * @param znode path of node to create + * @param data data of node to create + * @throws KeeperException if unexpected zookeeper exception + * @throws KeeperException.NodeExistsException if node already exists + */ + public static void createAndNoWatch(ZooKeeperWatcher zkw, String znode, + byte[] data) throws KeeperException, KeeperException.NodeExistsException { + try { + waitForZKConnectionIfAuthenticating(zkw); + zkw.getRecoverableZooKeeper().create(znode, data, createACL(zkw, znode), + CreateMode.PERSISTENT); + } catch (InterruptedException e) { + zkw.interruptedException(e); + } + } + + /** * Async creates the specified node with the specified data. * *

Throws an exception if the node already exists. Index: hbase-protocol/src/main/protobuf/MasterAdmin.proto =================================================================== --- hbase-protocol/src/main/protobuf/MasterAdmin.proto (revision 1433297) +++ hbase-protocol/src/main/protobuf/MasterAdmin.proto (working copy) @@ -63,6 +63,16 @@ message MoveRegionResponse { } +message MergeRegionRequest { + required string tableName = 1; + required string regionAEncodedName = 2; + required string regionBEncodedName = 3; + optional bool force = 4 [default = false]; +} + +message MergeRegionResponse { +} + message AssignRegionRequest { required RegionSpecifier region = 1; } @@ -193,6 +203,10 @@ /** Move the region region to the destination server. */ rpc moveRegion(MoveRegionRequest) returns(MoveRegionResponse); + + /** Merge two regions */ + rpc mergeRegion(MergeRegionRequest) + returns(MergeRegionResponse); /** Assign a region to a server chosen at random. */ rpc assignRegion(AssignRegionRequest) Index: hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 1433297) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (working copy) @@ -82,6 +82,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest; @@ -1470,6 +1471,32 @@ } /** + * Merge the given two regions. If want to merge two non-adjacent regions, set + * force true , otherwise the request will be skipped + * @param tableName + * @param regionAEncodedName merging region's encoded name + * @param regionBEncodedName merging region's encoded name + * @param force if it is false, only merge two adjacent regions + * @throws MasterNotRunningException + */ + public void mergeRegion(final String tableName, + final String regionAEncodedName, final String regionBEncodedName, + final boolean force) throws MasterNotRunningException { + MasterAdminKeepAliveConnection master = connection + .getKeepAliveMasterAdmin(); + try { + MergeRegionRequest request = RequestConverter.buildMergeRegionRequest( + tableName, regionAEncodedName, regionBEncodedName, force); + master.mergeRegion(null, request); + } catch (ServiceException se) { + LOG.error("Unexpected exception: " + se + + " from calling HMaster.mergeRegion"); + } finally { + master.close(); + } + } + + /** * @param regionName * Region name to assign. * @throws MasterNotRunningException Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeRequest.java =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeRequest.java (revision 0) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MergeRequest.java (revision 0) @@ -0,0 +1,238 @@ +/** + * Copyright The Apache Software Foundation + * + * 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; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.master.RegionMergeManager.MergeTransactionData; +import org.apache.hadoop.hbase.master.MergeTransaction.JournalState; +import org.apache.hadoop.hbase.monitoring.MonitoredTask; +import org.apache.hadoop.hbase.monitoring.TaskMonitor; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.zookeeper.KeeperException; + +import com.google.common.base.Preconditions; + +/** + * Handles processing region merges. Put in a queue, owned by MergeManager. + */ +class MergeRequest implements Runnable { + static final Log LOG = LogFactory.getLog(MergeRequest.class); + + private final MergeTransactionData transactionData; + private JournalState initialState; + private final RegionMergeManager mergeManager; + private final MasterServices masterServices; + + private static Class mergeTransactionClass; + + MergeRequest(final RegionMergeManager mergeManager, + final MergeTransactionData transactionData, JournalState initialState, + MasterServices masterServices) { + Preconditions.checkNotNull(masterServices); + this.transactionData = transactionData; + this.initialState = initialState; + this.mergeManager = mergeManager; + this.masterServices = masterServices; + } + + @Override + public void run() { + if (this.masterServices.isStopped()) { + LOG.debug("Skipping merge because server is stopped"); + return; + } + MonitoredTask status = TaskMonitor.get().createStatus( + "Merging " + transactionData + " with initialState=" + initialState); + try { + mergeManager.waitUntilMasterInitialized(); + HRegionInfo regionInfoA = this.mergeManager.getRegionInfoByEncodedName( + transactionData.getTableName(), transactionData.getRegionA()); + HRegionInfo regionInfoB = this.mergeManager.getRegionInfoByEncodedName( + transactionData.getTableName(), transactionData.getRegionB()); + if ((regionInfoA == null || regionInfoB == null)) { + if (!masterServices.getAssignmentManager().getZKTable() + .isTablePresent(transactionData.getTableName())) { + String msg = "Skip merging " + transactionData + + ", because table is not present now"; + LOG.info(msg); + status.abort(msg); + skipRequest(regionInfoA, regionInfoB); + return; + } else if (this.initialState == JournalState.CREATE_MERGE) { + String msg = "Skip merging " + transactionData + + ", because couldn't get the regioninfo, initialState=" + + initialState; + LOG.info(msg); + status.abort(msg); + skipRequest(regionInfoA, regionInfoB); + return; + } else if (this.initialState != JournalState.COMPLETE_MERGING) { + String msg = "Resubmit merging, because couldn't get the regioninfo, and initialState=" + + initialState; + LOG.warn(msg); + status.abort(msg); + mergeManager.addToResubmitQueue(transactionData); + return; + } + } else { + if (!transactionData.isForce() && !areAdjacent(regionInfoA, regionInfoB)) { + String msg = "Skip merging " + transactionData + + ", because two region are not adjacent, regionA=" + + regionInfoA.getRegionNameAsString() + ",regionB=" + + regionInfoB.getRegionNameAsString(); + LOG.info(msg); + status.abort(msg); + skipRequest(regionInfoA, regionInfoB); + return; + } + } + status.setStatus("Starting merging"); + LOG.info("Start merging " + transactionData + ", initialState=" + + initialState); + MergeTransaction mt = null; + int triesNumber = masterServices.getConfiguration().getInt( + "hbase.merge.number.tries", 5); + for (int i = 0; i < triesNumber; i++) { + try { + mt = createMergeTransaction(masterServices.getConfiguration(), + masterServices, mergeManager, transactionData, regionInfoA, + regionInfoB, initialState, status); + mt.execute(this.masterServices); + break; + } catch (IOException e) { + if (i == triesNumber - 1) { + throw e; + } + LOG.warn("Failed executing merge, " + transactionData + ", try=" + + (i + 1), e); + if (mt != null) { + this.initialState = mt.getCurrentState(); + } + } + } + HRegionInfo mergedRegion=mt.getMergedRegion(); + String msg = "Complete merging "+ transactionData + + ", successful=" + mt.isSuccessful() + + (mergedRegion == null ? "" : ", merged region=" + + mergedRegion.getRegionNameAsString()); + LOG.info(msg); + status.markComplete(msg); + } catch (InterruptedException e) { + String msg = "Merging" + transactionData + "interrupted by user "; + LOG.warn(msg, e); + status.abort(msg); + Thread.currentThread().interrupt(); + } catch (IOException e) { + String msg = "IOException in merging " + transactionData; + LOG.warn(msg, e); + status.abort(msg); + if (masterServices.getMasterFileSystem().checkFileSystem()) { + this.mergeManager.addToResubmitQueue(transactionData); + } + } catch (KeeperException e) { + String msg = "KeeperException in merging " + transactionData; + LOG.error(msg, e); + status.abort(msg); + masterServices.abort(msg, e); + } catch (Throwable e) { + String msg = "Unknown Exception in merging " + transactionData; + LOG.error(msg, e); + status.abort(msg); + this.mergeManager.addToResubmitQueue(transactionData); + } finally { + status.cleanup(); + } + } + + private void skipRequest(HRegionInfo regionInfoA, HRegionInfo regionInfoB) + throws InterruptedException, KeeperException { + if(regionInfoA!=null) this.mergeManager.assignMergingRegionIfOffline(regionInfoA); + if(regionInfoB!=null) this.mergeManager.assignMergingRegionIfOffline(regionInfoB); + this.mergeManager.finishMergeTransaction(transactionData, true); + } + + /** + * Check whether two regions are adjacent + * @param regionA + * @param regionB + * @return true if two regions are adjacent + */ + public static boolean areAdjacent(HRegionInfo regionA, HRegionInfo regionB) { + HRegionInfo a = regionA; + HRegionInfo b = regionB; + if (Bytes.compareTo(a.getStartKey(), b.getStartKey()) > 0) { + a = regionB; + b = regionA; + } + if (Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0) { + return true; + } + return false; + } + + /** + * Utility for constructing an instance of MergeTransaction class. + * @param conf + * @param masterServices + * @param mergeManager + * @param transactionData + * @param mergeA + * @param mergeB + * @param initialState + * @return instance of MergeTransaction class + */ + static MergeTransaction createMergeTransaction(final Configuration conf, + final MasterServices masterServices, final RegionMergeManager mergeManager, + final MergeTransactionData transactionData, final HRegionInfo mergeA, + final HRegionInfo mergeB, final JournalState initialState, + final MonitoredTask status) { + if (mergeTransactionClass == null) { + mergeTransactionClass = conf.getClass( + "hbase.master.merge.transaction.impl", MergeTransaction.class, + MergeTransaction.class); + } + try { + Constructor c = mergeTransactionClass + .getConstructor(MasterServices.class, RegionMergeManager.class, + MergeTransactionData.class, HRegionInfo.class, HRegionInfo.class, + JournalState.class, MonitoredTask.class); + return c.newInstance(masterServices, mergeManager, transactionData, + mergeA, mergeB, initialState, status); + } catch (InvocationTargetException ite) { + Throwable target = ite.getTargetException() != null ? ite + .getTargetException() : ite; + if (target.getCause() != null) + target = target.getCause(); + throw new RuntimeException("Failed construction of MergeTransaction: " + + mergeTransactionClass.toString(), target); + } catch (Exception e) { + throw new RuntimeException("Failed construction of MergeTransaction: " + + mergeTransactionClass.toString() + + ((e.getCause() != null) ? e.getCause().getMessage() : ""), e); + } + } + +} Index: hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java =================================================================== --- hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java (revision 1433297) +++ hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java (working copy) @@ -3404,6 +3404,1010 @@ // @@protoc_insertion_point(class_scope:MoveRegionResponse) } + public interface MergeRegionRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string tableName = 1; + boolean hasTableName(); + String getTableName(); + + // required string regionAEncodedName = 2; + boolean hasRegionAEncodedName(); + String getRegionAEncodedName(); + + // required string regionBEncodedName = 3; + boolean hasRegionBEncodedName(); + String getRegionBEncodedName(); + + // optional bool force = 4 [default = false]; + boolean hasForce(); + boolean getForce(); + } + public static final class MergeRegionRequest extends + com.google.protobuf.GeneratedMessage + implements MergeRegionRequestOrBuilder { + // Use MergeRegionRequest.newBuilder() to construct. + private MergeRegionRequest(Builder builder) { + super(builder); + } + private MergeRegionRequest(boolean noInit) {} + + private static final MergeRegionRequest defaultInstance; + public static MergeRegionRequest getDefaultInstance() { + return defaultInstance; + } + + public MergeRegionRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionRequest_fieldAccessorTable; + } + + private int bitField0_; + // required string tableName = 1; + public static final int TABLENAME_FIELD_NUMBER = 1; + private java.lang.Object tableName_; + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTableName() { + java.lang.Object ref = tableName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + tableName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getTableNameBytes() { + java.lang.Object ref = tableName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + tableName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string regionAEncodedName = 2; + public static final int REGIONAENCODEDNAME_FIELD_NUMBER = 2; + private java.lang.Object regionAEncodedName_; + public boolean hasRegionAEncodedName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getRegionAEncodedName() { + java.lang.Object ref = regionAEncodedName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + regionAEncodedName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getRegionAEncodedNameBytes() { + java.lang.Object ref = regionAEncodedName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + regionAEncodedName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string regionBEncodedName = 3; + public static final int REGIONBENCODEDNAME_FIELD_NUMBER = 3; + private java.lang.Object regionBEncodedName_; + public boolean hasRegionBEncodedName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getRegionBEncodedName() { + java.lang.Object ref = regionBEncodedName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + regionBEncodedName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getRegionBEncodedNameBytes() { + java.lang.Object ref = regionBEncodedName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + regionBEncodedName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // optional bool force = 4 [default = false]; + public static final int FORCE_FIELD_NUMBER = 4; + private boolean force_; + public boolean hasForce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public boolean getForce() { + return force_; + } + + private void initFields() { + tableName_ = ""; + regionAEncodedName_ = ""; + regionBEncodedName_ = ""; + force_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasTableName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRegionAEncodedName()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRegionBEncodedName()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getTableNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getRegionAEncodedNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getRegionBEncodedNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBool(4, force_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getTableNameBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getRegionAEncodedNameBytes()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(3, getRegionBEncodedNameBytes()); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(4, force_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest) obj; + + boolean result = true; + result = result && (hasTableName() == other.hasTableName()); + if (hasTableName()) { + result = result && getTableName() + .equals(other.getTableName()); + } + result = result && (hasRegionAEncodedName() == other.hasRegionAEncodedName()); + if (hasRegionAEncodedName()) { + result = result && getRegionAEncodedName() + .equals(other.getRegionAEncodedName()); + } + result = result && (hasRegionBEncodedName() == other.hasRegionBEncodedName()); + if (hasRegionBEncodedName()) { + result = result && getRegionBEncodedName() + .equals(other.getRegionBEncodedName()); + } + result = result && (hasForce() == other.hasForce()); + if (hasForce()) { + result = result && (getForce() + == other.getForce()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasTableName()) { + hash = (37 * hash) + TABLENAME_FIELD_NUMBER; + hash = (53 * hash) + getTableName().hashCode(); + } + if (hasRegionAEncodedName()) { + hash = (37 * hash) + REGIONAENCODEDNAME_FIELD_NUMBER; + hash = (53 * hash) + getRegionAEncodedName().hashCode(); + } + if (hasRegionBEncodedName()) { + hash = (37 * hash) + REGIONBENCODEDNAME_FIELD_NUMBER; + hash = (53 * hash) + getRegionBEncodedName().hashCode(); + } + if (hasForce()) { + hash = (37 * hash) + FORCE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getForce()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + tableName_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + regionAEncodedName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + regionBEncodedName_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + force_ = false; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.tableName_ = tableName_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.regionAEncodedName_ = regionAEncodedName_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.regionBEncodedName_ = regionBEncodedName_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.force_ = force_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.getDefaultInstance()) return this; + if (other.hasTableName()) { + setTableName(other.getTableName()); + } + if (other.hasRegionAEncodedName()) { + setRegionAEncodedName(other.getRegionAEncodedName()); + } + if (other.hasRegionBEncodedName()) { + setRegionBEncodedName(other.getRegionBEncodedName()); + } + if (other.hasForce()) { + setForce(other.getForce()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasTableName()) { + + return false; + } + if (!hasRegionAEncodedName()) { + + return false; + } + if (!hasRegionBEncodedName()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + tableName_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + regionAEncodedName_ = input.readBytes(); + break; + } + case 26: { + bitField0_ |= 0x00000004; + regionBEncodedName_ = input.readBytes(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + force_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // required string tableName = 1; + private java.lang.Object tableName_ = ""; + public boolean hasTableName() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getTableName() { + java.lang.Object ref = tableName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + tableName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setTableName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + tableName_ = value; + onChanged(); + return this; + } + public Builder clearTableName() { + bitField0_ = (bitField0_ & ~0x00000001); + tableName_ = getDefaultInstance().getTableName(); + onChanged(); + return this; + } + void setTableName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + tableName_ = value; + onChanged(); + } + + // required string regionAEncodedName = 2; + private java.lang.Object regionAEncodedName_ = ""; + public boolean hasRegionAEncodedName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getRegionAEncodedName() { + java.lang.Object ref = regionAEncodedName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + regionAEncodedName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setRegionAEncodedName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + regionAEncodedName_ = value; + onChanged(); + return this; + } + public Builder clearRegionAEncodedName() { + bitField0_ = (bitField0_ & ~0x00000002); + regionAEncodedName_ = getDefaultInstance().getRegionAEncodedName(); + onChanged(); + return this; + } + void setRegionAEncodedName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + regionAEncodedName_ = value; + onChanged(); + } + + // required string regionBEncodedName = 3; + private java.lang.Object regionBEncodedName_ = ""; + public boolean hasRegionBEncodedName() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getRegionBEncodedName() { + java.lang.Object ref = regionBEncodedName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + regionBEncodedName_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setRegionBEncodedName(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000004; + regionBEncodedName_ = value; + onChanged(); + return this; + } + public Builder clearRegionBEncodedName() { + bitField0_ = (bitField0_ & ~0x00000004); + regionBEncodedName_ = getDefaultInstance().getRegionBEncodedName(); + onChanged(); + return this; + } + void setRegionBEncodedName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + regionBEncodedName_ = value; + onChanged(); + } + + // optional bool force = 4 [default = false]; + private boolean force_ ; + public boolean hasForce() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public boolean getForce() { + return force_; + } + public Builder setForce(boolean value) { + bitField0_ |= 0x00000008; + force_ = value; + onChanged(); + return this; + } + public Builder clearForce() { + bitField0_ = (bitField0_ & ~0x00000008); + force_ = false; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:MergeRegionRequest) + } + + static { + defaultInstance = new MergeRegionRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:MergeRegionRequest) + } + + public interface MergeRegionResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class MergeRegionResponse extends + com.google.protobuf.GeneratedMessage + implements MergeRegionResponseOrBuilder { + // Use MergeRegionResponse.newBuilder() to construct. + private MergeRegionResponse(Builder builder) { + super(builder); + } + private MergeRegionResponse(boolean noInit) {} + + private static final MergeRegionResponse defaultInstance; + public static MergeRegionResponse getDefaultInstance() { + return defaultInstance; + } + + public MergeRegionResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionResponse_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_MergeRegionResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:MergeRegionResponse) + } + + static { + defaultInstance = new MergeRegionResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:MergeRegionResponse) + } + public interface AssignRegionRequestOrBuilder extends com.google.protobuf.MessageOrBuilder { @@ -14389,6 +15393,11 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest request, com.google.protobuf.RpcCallback done); + public abstract void mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request, + com.google.protobuf.RpcCallback done); + public abstract void assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request, @@ -14507,6 +15516,14 @@ } @java.lang.Override + public void mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request, + com.google.protobuf.RpcCallback done) { + impl.mergeRegion(controller, request, done); + } + + @java.lang.Override public void assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request, @@ -14665,36 +15682,38 @@ case 3: return impl.moveRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest)request); case 4: + return impl.mergeRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest)request); + case 5: return impl.assignRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest)request); - case 5: + case 6: return impl.unassignRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest)request); - case 6: + case 7: return impl.offlineRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest)request); - case 7: + case 8: return impl.deleteTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest)request); - case 8: + case 9: return impl.enableTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest)request); - case 9: + case 10: return impl.disableTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest)request); - case 10: + case 11: return impl.modifyTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest)request); - case 11: + case 12: return impl.createTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest)request); - case 12: + case 13: return impl.shutdown(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest)request); - case 13: + case 14: return impl.stopMaster(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest)request); - case 14: + case 15: return impl.balance(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest)request); - case 15: + case 16: return impl.setBalancerRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest)request); - case 16: + case 17: return impl.runCatalogScan(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest)request); - case 17: + case 18: return impl.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest)request); - case 18: + case 19: return impl.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest)request); - case 19: + case 20: return impl.execMasterService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request); default: throw new java.lang.AssertionError("Can't get here."); @@ -14719,36 +15738,38 @@ case 3: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest.getDefaultInstance(); case 4: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.getDefaultInstance(); + case 5: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest.getDefaultInstance(); - case 5: + case 6: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest.getDefaultInstance(); - case 6: + case 7: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest.getDefaultInstance(); - case 7: + case 8: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest.getDefaultInstance(); - case 8: + case 9: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest.getDefaultInstance(); - case 9: + case 10: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest.getDefaultInstance(); - case 10: + case 11: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest.getDefaultInstance(); - case 11: + case 12: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest.getDefaultInstance(); - case 12: + case 13: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest.getDefaultInstance(); - case 13: + case 14: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest.getDefaultInstance(); - case 14: + case 15: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest.getDefaultInstance(); - case 15: + case 16: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest.getDefaultInstance(); - case 16: + case 17: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest.getDefaultInstance(); - case 17: + case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.getDefaultInstance(); - case 18: + case 19: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance(); - case 19: + case 20: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14773,36 +15794,38 @@ case 3: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse.getDefaultInstance(); case 4: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance(); + case 5: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.getDefaultInstance(); - case 5: + case 6: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.getDefaultInstance(); - case 6: + case 7: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.getDefaultInstance(); - case 7: + case 8: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.getDefaultInstance(); - case 8: + case 9: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.getDefaultInstance(); - case 9: + case 10: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.getDefaultInstance(); - case 10: + case 11: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.getDefaultInstance(); - case 11: + case 12: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.getDefaultInstance(); - case 12: + case 13: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.getDefaultInstance(); - case 13: + case 14: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.getDefaultInstance(); - case 14: + case 15: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.getDefaultInstance(); - case 15: + case 16: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.getDefaultInstance(); - case 16: + case 17: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.getDefaultInstance(); - case 17: + case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance(); - case 18: + case 19: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(); - case 19: + case 20: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -14832,6 +15855,11 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest request, com.google.protobuf.RpcCallback done); + public abstract void mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request, + com.google.protobuf.RpcCallback done); + public abstract void assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request, @@ -14955,81 +15983,86 @@ done)); return; case 4: + this.mergeRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 5: this.assignRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 5: + case 6: this.unassignRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 6: + case 7: this.offlineRegion(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 7: + case 8: this.deleteTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 8: + case 9: this.enableTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 9: + case 10: this.disableTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 10: + case 11: this.modifyTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 11: + case 12: this.createTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 12: + case 13: this.shutdown(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 13: + case 14: this.stopMaster(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 14: + case 15: this.balance(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 15: + case 16: this.setBalancerRunning(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 16: + case 17: this.runCatalogScan(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 17: + case 18: this.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 18: + case 19: this.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); return; - case 19: + case 20: this.execMasterService(controller, (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest)request, com.google.protobuf.RpcUtil.specializeCallback( done)); @@ -15057,36 +16090,38 @@ case 3: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest.getDefaultInstance(); case 4: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.getDefaultInstance(); + case 5: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest.getDefaultInstance(); - case 5: + case 6: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest.getDefaultInstance(); - case 6: + case 7: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest.getDefaultInstance(); - case 7: + case 8: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest.getDefaultInstance(); - case 8: + case 9: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest.getDefaultInstance(); - case 9: + case 10: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest.getDefaultInstance(); - case 10: + case 11: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest.getDefaultInstance(); - case 11: + case 12: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest.getDefaultInstance(); - case 12: + case 13: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest.getDefaultInstance(); - case 13: + case 14: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest.getDefaultInstance(); - case 14: + case 15: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest.getDefaultInstance(); - case 15: + case 16: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest.getDefaultInstance(); - case 16: + case 17: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest.getDefaultInstance(); - case 17: + case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.getDefaultInstance(); - case 18: + case 19: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance(); - case 19: + case 20: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -15111,36 +16146,38 @@ case 3: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse.getDefaultInstance(); case 4: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance(); + case 5: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.getDefaultInstance(); - case 5: + case 6: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.getDefaultInstance(); - case 6: + case 7: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.getDefaultInstance(); - case 7: + case 8: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.getDefaultInstance(); - case 8: + case 9: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.getDefaultInstance(); - case 9: + case 10: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.getDefaultInstance(); - case 10: + case 11: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.getDefaultInstance(); - case 11: + case 12: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.getDefaultInstance(); - case 12: + case 13: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.getDefaultInstance(); - case 13: + case 14: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.getDefaultInstance(); - case 14: + case 15: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.getDefaultInstance(); - case 15: + case 16: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.getDefaultInstance(); - case 16: + case 17: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.getDefaultInstance(); - case 17: + case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance(); - case 18: + case 19: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(); - case 19: + case 20: return org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); @@ -15223,12 +16260,27 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse.getDefaultInstance())); } + public void mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance())); + } + public void assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.getDefaultInstance(), @@ -15243,7 +16295,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(6), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.getDefaultInstance(), @@ -15258,7 +16310,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(7), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.getDefaultInstance(), @@ -15273,7 +16325,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.getDefaultInstance(), @@ -15288,7 +16340,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(8), + getDescriptor().getMethods().get(9), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.getDefaultInstance(), @@ -15303,7 +16355,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(9), + getDescriptor().getMethods().get(10), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.getDefaultInstance(), @@ -15318,7 +16370,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.getDefaultInstance(), @@ -15333,7 +16385,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.getDefaultInstance(), @@ -15348,7 +16400,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.getDefaultInstance(), @@ -15363,7 +16415,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(13), + getDescriptor().getMethods().get(14), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.getDefaultInstance(), @@ -15378,7 +16430,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(14), + getDescriptor().getMethods().get(15), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.getDefaultInstance(), @@ -15393,7 +16445,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(15), + getDescriptor().getMethods().get(16), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.getDefaultInstance(), @@ -15408,7 +16460,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(16), + getDescriptor().getMethods().get(17), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.getDefaultInstance(), @@ -15423,7 +16475,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(17), + getDescriptor().getMethods().get(18), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance(), @@ -15438,7 +16490,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(18), + getDescriptor().getMethods().get(19), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(), @@ -15453,7 +16505,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request, com.google.protobuf.RpcCallback done) { channel.callMethod( - getDescriptor().getMethods().get(19), + getDescriptor().getMethods().get(20), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance(), @@ -15490,6 +16542,11 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest request) throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request) + throws com.google.protobuf.ServiceException; + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request) @@ -15626,12 +16683,24 @@ } + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse mergeRegion( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(4), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.getDefaultInstance()); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse assignRegion( com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(4), + getDescriptor().getMethods().get(5), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.getDefaultInstance()); @@ -15643,7 +16712,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(5), + getDescriptor().getMethods().get(6), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.getDefaultInstance()); @@ -15655,7 +16724,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(6), + getDescriptor().getMethods().get(7), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.getDefaultInstance()); @@ -15667,7 +16736,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(7), + getDescriptor().getMethods().get(8), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.getDefaultInstance()); @@ -15679,7 +16748,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(8), + getDescriptor().getMethods().get(9), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.getDefaultInstance()); @@ -15691,7 +16760,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(9), + getDescriptor().getMethods().get(10), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.getDefaultInstance()); @@ -15703,7 +16772,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(10), + getDescriptor().getMethods().get(11), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.getDefaultInstance()); @@ -15715,7 +16784,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(11), + getDescriptor().getMethods().get(12), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.getDefaultInstance()); @@ -15727,7 +16796,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(12), + getDescriptor().getMethods().get(13), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.getDefaultInstance()); @@ -15739,7 +16808,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(13), + getDescriptor().getMethods().get(14), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.getDefaultInstance()); @@ -15751,7 +16820,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(14), + getDescriptor().getMethods().get(15), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.getDefaultInstance()); @@ -15763,7 +16832,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(15), + getDescriptor().getMethods().get(16), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.getDefaultInstance()); @@ -15775,7 +16844,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(16), + getDescriptor().getMethods().get(17), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.getDefaultInstance()); @@ -15787,7 +16856,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(17), + getDescriptor().getMethods().get(18), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance()); @@ -15799,7 +16868,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(18), + getDescriptor().getMethods().get(19), controller, request, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance()); @@ -15811,7 +16880,7 @@ org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceRequest request) throws com.google.protobuf.ServiceException { return (org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(19), + getDescriptor().getMethods().get(20), controller, request, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceResponse.getDefaultInstance()); @@ -15861,6 +16930,16 @@ com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_MoveRegionResponse_fieldAccessorTable; private static com.google.protobuf.Descriptors.Descriptor + internal_static_MergeRegionRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_MergeRegionRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_MergeRegionResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_MergeRegionResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor internal_static_AssignRegionRequest_descriptor; private static com.google.protobuf.GeneratedMessage.FieldAccessorTable @@ -16030,65 +17109,70 @@ "hema\"\026\n\024ModifyColumnResponse\"Z\n\021MoveRegi" + "onRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecif", "ier\022#\n\016destServerName\030\002 \001(\0132\013.ServerName" + - "\"\024\n\022MoveRegionResponse\"7\n\023AssignRegionRe" + - "quest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifier\"" + - "\026\n\024AssignRegionResponse\"O\n\025UnassignRegio" + - "nRequest\022 \n\006region\030\001 \002(\0132\020.RegionSpecifi" + - "er\022\024\n\005force\030\002 \001(\010:\005false\"\030\n\026UnassignRegi" + - "onResponse\"8\n\024OfflineRegionRequest\022 \n\006re" + - "gion\030\001 \002(\0132\020.RegionSpecifier\"\027\n\025OfflineR" + - "egionResponse\"J\n\022CreateTableRequest\022!\n\013t" + - "ableSchema\030\001 \002(\0132\014.TableSchema\022\021\n\tsplitK", - "eys\030\002 \003(\014\"\025\n\023CreateTableResponse\"\'\n\022Dele" + - "teTableRequest\022\021\n\ttableName\030\001 \002(\014\"\025\n\023Del" + - "eteTableResponse\"\'\n\022EnableTableRequest\022\021" + - "\n\ttableName\030\001 \002(\014\"\025\n\023EnableTableResponse" + - "\"(\n\023DisableTableRequest\022\021\n\ttableName\030\001 \002" + - "(\014\"\026\n\024DisableTableResponse\"J\n\022ModifyTabl" + - "eRequest\022\021\n\ttableName\030\001 \002(\014\022!\n\013tableSche" + - "ma\030\002 \002(\0132\014.TableSchema\"\025\n\023ModifyTableRes" + - "ponse\"\021\n\017ShutdownRequest\"\022\n\020ShutdownResp" + - "onse\"\023\n\021StopMasterRequest\"\024\n\022StopMasterR", - "esponse\"\020\n\016BalanceRequest\"&\n\017BalanceResp" + - "onse\022\023\n\013balancerRan\030\001 \002(\010\"<\n\031SetBalancer" + - "RunningRequest\022\n\n\002on\030\001 \002(\010\022\023\n\013synchronou" + - "s\030\002 \001(\010\"6\n\032SetBalancerRunningResponse\022\030\n" + - "\020prevBalanceValue\030\001 \001(\010\"\024\n\022CatalogScanRe" + - "quest\")\n\023CatalogScanResponse\022\022\n\nscanResu" + - "lt\030\001 \001(\005\"-\n\033EnableCatalogJanitorRequest\022" + - "\016\n\006enable\030\001 \002(\010\"1\n\034EnableCatalogJanitorR" + - "esponse\022\021\n\tprevValue\030\001 \001(\010\" \n\036IsCatalogJ" + - "anitorEnabledRequest\"0\n\037IsCatalogJanitor", - "EnabledResponse\022\r\n\005value\030\001 \002(\0102\201\n\n\022Maste" + - "rAdminService\0222\n\taddColumn\022\021.AddColumnRe" + - "quest\032\022.AddColumnResponse\022;\n\014deleteColum" + - "n\022\024.DeleteColumnRequest\032\025.DeleteColumnRe" + - "sponse\022;\n\014modifyColumn\022\024.ModifyColumnReq" + - "uest\032\025.ModifyColumnResponse\0225\n\nmoveRegio" + - "n\022\022.MoveRegionRequest\032\023.MoveRegionRespon" + + "\"\024\n\022MoveRegionResponse\"u\n\022MergeRegionReq" + + "uest\022\021\n\ttableName\030\001 \002(\t\022\032\n\022regionAEncode" + + "dName\030\002 \002(\t\022\032\n\022regionBEncodedName\030\003 \002(\t\022" + + "\024\n\005force\030\004 \001(\010:\005false\"\025\n\023MergeRegionResp" + + "onse\"7\n\023AssignRegionRequest\022 \n\006region\030\001 " + + "\002(\0132\020.RegionSpecifier\"\026\n\024AssignRegionRes" + + "ponse\"O\n\025UnassignRegionRequest\022 \n\006region" + + "\030\001 \002(\0132\020.RegionSpecifier\022\024\n\005force\030\002 \001(\010:" + + "\005false\"\030\n\026UnassignRegionResponse\"8\n\024Offl", + "ineRegionRequest\022 \n\006region\030\001 \002(\0132\020.Regio" + + "nSpecifier\"\027\n\025OfflineRegionResponse\"J\n\022C" + + "reateTableRequest\022!\n\013tableSchema\030\001 \002(\0132\014" + + ".TableSchema\022\021\n\tsplitKeys\030\002 \003(\014\"\025\n\023Creat" + + "eTableResponse\"\'\n\022DeleteTableRequest\022\021\n\t" + + "tableName\030\001 \002(\014\"\025\n\023DeleteTableResponse\"\'" + + "\n\022EnableTableRequest\022\021\n\ttableName\030\001 \002(\014\"" + + "\025\n\023EnableTableResponse\"(\n\023DisableTableRe" + + "quest\022\021\n\ttableName\030\001 \002(\014\"\026\n\024DisableTable" + + "Response\"J\n\022ModifyTableRequest\022\021\n\ttableN", + "ame\030\001 \002(\014\022!\n\013tableSchema\030\002 \002(\0132\014.TableSc" + + "hema\"\025\n\023ModifyTableResponse\"\021\n\017ShutdownR" + + "equest\"\022\n\020ShutdownResponse\"\023\n\021StopMaster" + + "Request\"\024\n\022StopMasterResponse\"\020\n\016Balance" + + "Request\"&\n\017BalanceResponse\022\023\n\013balancerRa" + + "n\030\001 \002(\010\"<\n\031SetBalancerRunningRequest\022\n\n\002" + + "on\030\001 \002(\010\022\023\n\013synchronous\030\002 \001(\010\"6\n\032SetBala" + + "ncerRunningResponse\022\030\n\020prevBalanceValue\030" + + "\001 \001(\010\"\024\n\022CatalogScanRequest\")\n\023CatalogSc" + + "anResponse\022\022\n\nscanResult\030\001 \001(\005\"-\n\033Enable", + "CatalogJanitorRequest\022\016\n\006enable\030\001 \002(\010\"1\n" + + "\034EnableCatalogJanitorResponse\022\021\n\tprevVal" + + "ue\030\001 \001(\010\" \n\036IsCatalogJanitorEnabledReque" + + "st\"0\n\037IsCatalogJanitorEnabledResponse\022\r\n" + + "\005value\030\001 \002(\0102\273\n\n\022MasterAdminService\0222\n\ta" + + "ddColumn\022\021.AddColumnRequest\032\022.AddColumnR" + + "esponse\022;\n\014deleteColumn\022\024.DeleteColumnRe" + + "quest\032\025.DeleteColumnResponse\022;\n\014modifyCo" + + "lumn\022\024.ModifyColumnRequest\032\025.ModifyColum" + + "nResponse\0225\n\nmoveRegion\022\022.MoveRegionRequ", + "est\032\023.MoveRegionResponse\0228\n\013mergeRegion\022" + + "\023.MergeRegionRequest\032\024.MergeRegionRespon" + "se\022;\n\014assignRegion\022\024.AssignRegionRequest" + "\032\025.AssignRegionResponse\022A\n\016unassignRegio" + - "n\022\026.UnassignRegionRequest\032\027.UnassignRegi", + "n\022\026.UnassignRegionRequest\032\027.UnassignRegi" + "onResponse\022>\n\rofflineRegion\022\025.OfflineReg" + "ionRequest\032\026.OfflineRegionResponse\0228\n\013de" + "leteTable\022\023.DeleteTableRequest\032\024.DeleteT" + "ableResponse\0228\n\013enableTable\022\023.EnableTabl" + - "eRequest\032\024.EnableTableResponse\022;\n\014disabl" + + "eRequest\032\024.EnableTableResponse\022;\n\014disabl", "eTable\022\024.DisableTableRequest\032\025.DisableTa" + "bleResponse\0228\n\013modifyTable\022\023.ModifyTable" + "Request\032\024.ModifyTableResponse\0228\n\013createT" + "able\022\023.CreateTableRequest\032\024.CreateTableR" + - "esponse\022/\n\010shutdown\022\020.ShutdownRequest\032\021.", + "esponse\022/\n\010shutdown\022\020.ShutdownRequest\032\021." + "ShutdownResponse\0225\n\nstopMaster\022\022.StopMas" + "terRequest\032\023.StopMasterResponse\022,\n\007balan" + "ce\022\017.BalanceRequest\032\020.BalanceResponse\022M\n" + "\022setBalancerRunning\022\032.SetBalancerRunning" + - "Request\032\033.SetBalancerRunningResponse\022;\n\016" + + "Request\032\033.SetBalancerRunningResponse\022;\n\016", "runCatalogScan\022\023.CatalogScanRequest\032\024.Ca" + "talogScanResponse\022S\n\024enableCatalogJanito" + "r\022\034.EnableCatalogJanitorRequest\032\035.Enable" + "CatalogJanitorResponse\022\\\n\027isCatalogJanit" + - "orEnabled\022\037.IsCatalogJanitorEnabledReque", + "orEnabled\022\037.IsCatalogJanitorEnabledReque" + "st\032 .IsCatalogJanitorEnabledResponse\022L\n\021" + "execMasterService\022\032.CoprocessorServiceRe" + "quest\032\033.CoprocessorServiceResponseBG\n*or" + @@ -16164,8 +17248,24 @@ new java.lang.String[] { }, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionResponse.Builder.class); + internal_static_MergeRegionRequest_descriptor = + getDescriptor().getMessageTypes().get(8); + internal_static_MergeRegionRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_MergeRegionRequest_descriptor, + new java.lang.String[] { "TableName", "RegionAEncodedName", "RegionBEncodedName", "Force", }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionRequest.Builder.class); + internal_static_MergeRegionResponse_descriptor = + getDescriptor().getMessageTypes().get(9); + internal_static_MergeRegionResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_MergeRegionResponse_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MergeRegionResponse.Builder.class); internal_static_AssignRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(8); + getDescriptor().getMessageTypes().get(10); internal_static_AssignRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_AssignRegionRequest_descriptor, @@ -16173,7 +17273,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionRequest.Builder.class); internal_static_AssignRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(9); + getDescriptor().getMessageTypes().get(11); internal_static_AssignRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_AssignRegionResponse_descriptor, @@ -16181,7 +17281,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.AssignRegionResponse.Builder.class); internal_static_UnassignRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(10); + getDescriptor().getMessageTypes().get(12); internal_static_UnassignRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UnassignRegionRequest_descriptor, @@ -16189,7 +17289,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest.Builder.class); internal_static_UnassignRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(11); + getDescriptor().getMessageTypes().get(13); internal_static_UnassignRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_UnassignRegionResponse_descriptor, @@ -16197,7 +17297,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse.Builder.class); internal_static_OfflineRegionRequest_descriptor = - getDescriptor().getMessageTypes().get(12); + getDescriptor().getMessageTypes().get(14); internal_static_OfflineRegionRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OfflineRegionRequest_descriptor, @@ -16205,7 +17305,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest.Builder.class); internal_static_OfflineRegionResponse_descriptor = - getDescriptor().getMessageTypes().get(13); + getDescriptor().getMessageTypes().get(15); internal_static_OfflineRegionResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_OfflineRegionResponse_descriptor, @@ -16213,7 +17313,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionResponse.Builder.class); internal_static_CreateTableRequest_descriptor = - getDescriptor().getMessageTypes().get(14); + getDescriptor().getMessageTypes().get(16); internal_static_CreateTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CreateTableRequest_descriptor, @@ -16221,7 +17321,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableRequest.Builder.class); internal_static_CreateTableResponse_descriptor = - getDescriptor().getMessageTypes().get(15); + getDescriptor().getMessageTypes().get(17); internal_static_CreateTableResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CreateTableResponse_descriptor, @@ -16229,7 +17329,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CreateTableResponse.Builder.class); internal_static_DeleteTableRequest_descriptor = - getDescriptor().getMessageTypes().get(16); + getDescriptor().getMessageTypes().get(18); internal_static_DeleteTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DeleteTableRequest_descriptor, @@ -16237,7 +17337,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableRequest.Builder.class); internal_static_DeleteTableResponse_descriptor = - getDescriptor().getMessageTypes().get(17); + getDescriptor().getMessageTypes().get(19); internal_static_DeleteTableResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DeleteTableResponse_descriptor, @@ -16245,7 +17345,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DeleteTableResponse.Builder.class); internal_static_EnableTableRequest_descriptor = - getDescriptor().getMessageTypes().get(18); + getDescriptor().getMessageTypes().get(20); internal_static_EnableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableTableRequest_descriptor, @@ -16253,7 +17353,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest.Builder.class); internal_static_EnableTableResponse_descriptor = - getDescriptor().getMessageTypes().get(19); + getDescriptor().getMessageTypes().get(21); internal_static_EnableTableResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableTableResponse_descriptor, @@ -16261,7 +17361,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableResponse.Builder.class); internal_static_DisableTableRequest_descriptor = - getDescriptor().getMessageTypes().get(20); + getDescriptor().getMessageTypes().get(22); internal_static_DisableTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DisableTableRequest_descriptor, @@ -16269,7 +17369,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableRequest.Builder.class); internal_static_DisableTableResponse_descriptor = - getDescriptor().getMessageTypes().get(21); + getDescriptor().getMessageTypes().get(23); internal_static_DisableTableResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_DisableTableResponse_descriptor, @@ -16277,7 +17377,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.DisableTableResponse.Builder.class); internal_static_ModifyTableRequest_descriptor = - getDescriptor().getMessageTypes().get(22); + getDescriptor().getMessageTypes().get(24); internal_static_ModifyTableRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ModifyTableRequest_descriptor, @@ -16285,7 +17385,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest.Builder.class); internal_static_ModifyTableResponse_descriptor = - getDescriptor().getMessageTypes().get(23); + getDescriptor().getMessageTypes().get(25); internal_static_ModifyTableResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ModifyTableResponse_descriptor, @@ -16293,7 +17393,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableResponse.Builder.class); internal_static_ShutdownRequest_descriptor = - getDescriptor().getMessageTypes().get(24); + getDescriptor().getMessageTypes().get(26); internal_static_ShutdownRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ShutdownRequest_descriptor, @@ -16301,7 +17401,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest.Builder.class); internal_static_ShutdownResponse_descriptor = - getDescriptor().getMessageTypes().get(25); + getDescriptor().getMessageTypes().get(27); internal_static_ShutdownResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_ShutdownResponse_descriptor, @@ -16309,7 +17409,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse.Builder.class); internal_static_StopMasterRequest_descriptor = - getDescriptor().getMessageTypes().get(26); + getDescriptor().getMessageTypes().get(28); internal_static_StopMasterRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_StopMasterRequest_descriptor, @@ -16317,7 +17417,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest.Builder.class); internal_static_StopMasterResponse_descriptor = - getDescriptor().getMessageTypes().get(27); + getDescriptor().getMessageTypes().get(29); internal_static_StopMasterResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_StopMasterResponse_descriptor, @@ -16325,7 +17425,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse.Builder.class); internal_static_BalanceRequest_descriptor = - getDescriptor().getMessageTypes().get(28); + getDescriptor().getMessageTypes().get(30); internal_static_BalanceRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BalanceRequest_descriptor, @@ -16333,7 +17433,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceRequest.Builder.class); internal_static_BalanceResponse_descriptor = - getDescriptor().getMessageTypes().get(29); + getDescriptor().getMessageTypes().get(31); internal_static_BalanceResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_BalanceResponse_descriptor, @@ -16341,7 +17441,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.BalanceResponse.Builder.class); internal_static_SetBalancerRunningRequest_descriptor = - getDescriptor().getMessageTypes().get(30); + getDescriptor().getMessageTypes().get(32); internal_static_SetBalancerRunningRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SetBalancerRunningRequest_descriptor, @@ -16349,7 +17449,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest.Builder.class); internal_static_SetBalancerRunningResponse_descriptor = - getDescriptor().getMessageTypes().get(31); + getDescriptor().getMessageTypes().get(33); internal_static_SetBalancerRunningResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SetBalancerRunningResponse_descriptor, @@ -16357,7 +17457,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningResponse.Builder.class); internal_static_CatalogScanRequest_descriptor = - getDescriptor().getMessageTypes().get(32); + getDescriptor().getMessageTypes().get(34); internal_static_CatalogScanRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CatalogScanRequest_descriptor, @@ -16365,7 +17465,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanRequest.Builder.class); internal_static_CatalogScanResponse_descriptor = - getDescriptor().getMessageTypes().get(33); + getDescriptor().getMessageTypes().get(35); internal_static_CatalogScanResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_CatalogScanResponse_descriptor, @@ -16373,7 +17473,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.CatalogScanResponse.Builder.class); internal_static_EnableCatalogJanitorRequest_descriptor = - getDescriptor().getMessageTypes().get(34); + getDescriptor().getMessageTypes().get(36); internal_static_EnableCatalogJanitorRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableCatalogJanitorRequest_descriptor, @@ -16381,7 +17481,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.Builder.class); internal_static_EnableCatalogJanitorResponse_descriptor = - getDescriptor().getMessageTypes().get(35); + getDescriptor().getMessageTypes().get(37); internal_static_EnableCatalogJanitorResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_EnableCatalogJanitorResponse_descriptor, @@ -16389,7 +17489,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.Builder.class); internal_static_IsCatalogJanitorEnabledRequest_descriptor = - getDescriptor().getMessageTypes().get(36); + getDescriptor().getMessageTypes().get(38); internal_static_IsCatalogJanitorEnabledRequest_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_IsCatalogJanitorEnabledRequest_descriptor, @@ -16397,7 +17497,7 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.Builder.class); internal_static_IsCatalogJanitorEnabledResponse_descriptor = - getDescriptor().getMessageTypes().get(37); + getDescriptor().getMessageTypes().get(39); internal_static_IsCatalogJanitorEnabledResponse_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_IsCatalogJanitorEnabledResponse_descriptor,