You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ap...@apache.org on 2014/10/25 02:57:33 UTC
[1/2] HBASE-12142 Truncate command does not preserve ACLs table
(Vandana Ayyalasomayajula)
Repository: hbase
Updated Branches:
refs/heads/0.98 714e8368c -> 5b1380c3c
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
new file mode 100644
index 0000000..d13bca5
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/TruncateTableHandler.java
@@ -0,0 +1,247 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.handler;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.TableExistsException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.backup.HFileArchiver;
+import org.apache.hadoop.hbase.catalog.MetaEditor;
+import org.apache.hadoop.hbase.master.AssignmentManager;
+import org.apache.hadoop.hbase.master.HMaster;
+import org.apache.hadoop.hbase.master.MasterCoprocessorHost;
+import org.apache.hadoop.hbase.master.MasterFileSystem;
+import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.RegionState.State;
+import org.apache.hadoop.hbase.master.RegionStates;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.util.FSTableDescriptors;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.apache.hadoop.hbase.util.ModifyRegionUtils;
+import org.apache.zookeeper.KeeperException;
+
+/**
+ * Truncate the table by removing META and the HDFS files and recreating it.
+ * If the 'preserveSplits' option is set to true, the region splits are preserved on recreate.
+ *
+ * If the operation fails in the middle it may require hbck to fix the system state.
+ */
+@InterfaceAudience.Private
+public class TruncateTableHandler extends DeleteTableHandler {
+ private static final Log LOG = LogFactory.getLog(TruncateTableHandler.class);
+
+ private final boolean preserveSplits;
+
+ public TruncateTableHandler(final TableName tableName, final Server server,
+ final MasterServices masterServices, boolean preserveSplits) {
+ super(tableName, server, masterServices);
+ this.preserveSplits = preserveSplits;
+ }
+
+ @Override
+ protected void handleTableOperation(List<HRegionInfo> regions)
+ throws IOException, KeeperException {
+ MasterCoprocessorHost cpHost = ((HMaster) this.server).getCoprocessorHost();
+ if (cpHost != null) {
+ cpHost.preTruncateTableHandler(this.tableName);
+ }
+
+ // 1. Wait because of region in transition
+ waitRegionInTransition(regions);
+
+ // 2. Remove table from .META. and HDFS
+ removeTableData(regions);
+
+ // -----------------------------------------------------------------------
+ // PONR: At this point the table is deleted.
+ // If the recreate fails, the user can only re-create the table.
+ // -----------------------------------------------------------------------
+
+ // 3. Recreate the regions
+ recreateTable(regions);
+
+ if (cpHost != null) {
+ cpHost.postTruncateTableHandler(this.tableName);
+ }
+ }
+
+ private void recreateTable(final List<HRegionInfo> regions) throws IOException {
+ MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
+ Path tempdir = mfs.getTempDir();
+ FileSystem fs = mfs.getFileSystem();
+
+ AssignmentManager assignmentManager = this.masterServices.getAssignmentManager();
+
+ // 1. Set table znode
+ checkAndSetEnablingTable(assignmentManager, tableName);
+ try {
+ // 1. Create Table Descriptor
+ new FSTableDescriptors(server.getConfiguration())
+ .createTableDescriptorForTableDirectory(tempdir, getTableDescriptor(), false);
+ Path tempTableDir = FSUtils.getTableDir(tempdir, this.tableName);
+ Path tableDir = FSUtils.getTableDir(mfs.getRootDir(), this.tableName);
+
+ HRegionInfo[] newRegions;
+ if (this.preserveSplits) {
+ newRegions = regions.toArray(new HRegionInfo[regions.size()]);
+ LOG.info("Truncate will preserve " + newRegions.length + " regions");
+ } else {
+ newRegions = new HRegionInfo[1];
+ newRegions[0] = new HRegionInfo(this.tableName, null, null);
+ LOG.info("Truncate will not preserve the regions");
+ }
+
+ // 2. Create Regions
+ List<HRegionInfo> regionInfos = ModifyRegionUtils.createRegions(
+ masterServices.getConfiguration(), tempdir,
+ getTableDescriptor(), newRegions, null);
+
+ // 3. Move Table temp directory to the hbase root location
+ if (!fs.rename(tempTableDir, tableDir)) {
+ throw new IOException("Unable to move table from temp=" + tempTableDir +
+ " to hbase root=" + tableDir);
+ }
+
+ // 4. Add regions to META
+ MetaEditor.addRegionsToMeta(masterServices.getCatalogTracker(), regionInfos);
+
+ // 5. Trigger immediate assignment of the regions in round-robin fashion
+ ModifyRegionUtils.assignRegions(assignmentManager, regionInfos);
+
+ // 6. Set table enabled flag up in zk.
+ try {
+ assignmentManager.getZKTable().setEnabledTable(tableName);
+ } catch (KeeperException e) {
+ throw new IOException("Unable to ensure that " + tableName + " will be" +
+ " enabled because of a ZooKeeper issue", e);
+ }
+ } catch (IOException e) {
+ removeEnablingTable(assignmentManager, tableName);
+ throw e;
+ }
+ }
+
+ void checkAndSetEnablingTable(final AssignmentManager assignmentManager, final TableName tableName)
+ throws IOException {
+ // If we have multiple client threads trying to create the table at the
+ // same time, given the async nature of the operation, the table
+ // could be in a state where hbase:meta table hasn't been updated yet in
+ // the process() function.
+ // Use enabling state to tell if there is already a request for the same
+ // table in progress. This will introduce a new zookeeper call. Given
+ // createTable isn't a frequent operation, that should be ok.
+ // TODO: now that we have table locks, re-evaluate above -- table locks are not enough.
+ // We could have cleared the hbase.rootdir and not zk. How can we detect this case?
+ // Having to clean zk AND hdfs is awkward.
+ try {
+ if (!assignmentManager.getZKTable().checkAndSetEnablingTable(tableName)) {
+ throw new TableExistsException(tableName);
+ }
+ } catch (KeeperException e) {
+ throw new IOException("Unable to ensure that the table will be"
+ + " enabling because of a ZooKeeper issue", e);
+ }
+ }
+
+ void removeEnablingTable(final AssignmentManager assignmentManager, final TableName tableName) {
+ // Try deleting the enabling node in case of error
+ // If this does not happen then if the client tries to create the table
+ // again with the same Active master
+ // It will block the creation saying TableAlreadyExists.
+ try {
+ assignmentManager.getZKTable().removeEnablingTable(tableName, false);
+ } catch (KeeperException e) {
+ // Keeper exception should not happen here
+ LOG.error("Got a keeper exception while removing the ENABLING table znode " + tableName, e);
+ }
+ }
+
+ /**
+ * Removes the table from .META. and archives the HDFS files.
+ */
+ void removeTableData(final List<HRegionInfo> regions) throws IOException, KeeperException {
+ // 1. Remove regions from META
+ LOG.debug("Deleting regions from META");
+ MetaEditor.deleteRegions(this.server.getCatalogTracker(), regions);
+
+ // -----------------------------------------------------------------------
+ // NOTE: At this point we still have data on disk, but nothing in .META.
+ // if the rename below fails, hbck will report an inconsistency.
+ // -----------------------------------------------------------------------
+
+ // 2. Move the table in /hbase/.tmp
+ MasterFileSystem mfs = this.masterServices.getMasterFileSystem();
+ Path tempTableDir = mfs.moveTableToTemp(tableName);
+
+ // 3. Archive regions from FS (temp directory)
+ FileSystem fs = mfs.getFileSystem();
+ for (HRegionInfo hri : regions) {
+ LOG.debug("Archiving region " + hri.getRegionNameAsString() + " from FS");
+ HFileArchiver.archiveRegion(fs, mfs.getRootDir(), tempTableDir,
+ HRegion.getRegionDir(tempTableDir, hri.getEncodedName()));
+ }
+
+ // 4. Delete table directory from FS (temp directory)
+ if (!fs.delete(tempTableDir, true)) {
+ LOG.error("Couldn't delete " + tempTableDir);
+ }
+
+ LOG.debug("Table '" + tableName + "' archived!");
+ }
+
+ void waitRegionInTransition(final List<HRegionInfo> regions) throws IOException {
+ AssignmentManager am = this.masterServices.getAssignmentManager();
+ RegionStates states = am.getRegionStates();
+ long waitTime = server.getConfiguration().getLong("hbase.master.wait.on.region", 5 * 60 * 1000);
+ for (HRegionInfo region : regions) {
+ long done = System.currentTimeMillis() + waitTime;
+ while (System.currentTimeMillis() < done) {
+ if (states.isRegionInState(region, State.FAILED_OPEN)) {
+ am.regionOffline(region);
+ }
+ if (!states.isRegionInTransition(region)) {
+ break;
+ }
+ try {
+ Thread.sleep(waitingTimeForEvents);
+ } catch (InterruptedException e) {
+ LOG.warn("Interrupted while sleeping");
+ throw (InterruptedIOException) new InterruptedIOException().initCause(e);
+ }
+ LOG.debug("Waiting on region to clear regions in transition; "
+ + am.getRegionStates().getRegionTransitionState(region));
+ }
+ if (states.isRegionInTransition(region)) {
+ throw new IOException("Waited hbase.master.wait.on.region (" + waitTime
+ + "ms) for region to leave region " + region.getRegionNameAsString()
+ + " in transitions");
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
index 4e87bbc..40bbbad 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java
@@ -172,6 +172,8 @@ public class AccessController extends BaseMasterAndRegionObserver
private Map<InternalScanner,String> scannerOwners =
new MapMaker().weakKeys().makeMap();
+ private Map<TableName, List<UserPermission>> tableAcls;
+
// Provider for mapping principal names to Users
private UserProvider userProvider;
@@ -871,12 +873,44 @@ public class AccessController extends BaseMasterAndRegionObserver
+ " is required for " + AccessControlConstants.CF_ATTRIBUTE_EARLY_OUT
+ " to have an effect");
}
+
+ tableAcls = new MapMaker().weakValues().makeMap();
}
public void stop(CoprocessorEnvironment env) {
}
+ @Override
+ public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> c, TableName tableName)
+ throws IOException {
+ requirePermission("truncateTable", tableName, null, null, Action.ADMIN);
+ List<UserPermission> acls = AccessControlLists.getUserTablePermissions(c.getEnvironment()
+ .getConfiguration(), tableName);
+ if (acls != null) {
+ tableAcls.put(tableName, acls);
+ }
+ }
+
+ @Override
+ public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ List<UserPermission> perms = tableAcls.get(tableName);
+ if (perms != null) {
+ for (UserPermission perm : perms) {
+ AccessControlLists.addUserPermission(ctx.getEnvironment().getConfiguration(), perm);
+ }
+ }
+ tableAcls.remove(tableName);
+ }
+
+ @Override
+ public void preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+ TableName tableName) throws IOException {}
+ @Override
+ public void postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> c,
+ TableName tableName) throws IOException {}
+
@Override
public void preCreateTable(ObserverContext<MasterCoprocessorEnvironment> c,
HTableDescriptor desc, HRegionInfo[] regions) throws IOException {
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
index f74179b..98cdbba 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/security/visibility/VisibilityController.java
@@ -835,4 +835,24 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
return matchFound ? ReturnCode.INCLUDE : ReturnCode.SKIP;
}
}
+
+ @Override
+ public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
+
+ @Override
+ public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
+
+ @Override
+ public void preTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
+
+ @Override
+ public void postTruncateTableHandler(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
index 205eaea..4508835 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/ModifyRegionUtils.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.master.AssignmentManager;
/**
* Utility methods for interacting with the regions.
@@ -251,4 +252,24 @@ public abstract class ModifyRegionUtils {
});
return regionOpenAndInitThreadPool;
}
+
+ /**
+ * Triggers a bulk assignment of the specified regions
+ *
+ * @param assignmentManager the Assignment Manger
+ * @param regionInfos the list of regions to assign
+ * @throws IOException if an error occurred during the assignment
+ */
+ public static void assignRegions(final AssignmentManager assignmentManager,
+ final List<HRegionInfo> regionInfos) throws IOException {
+ try {
+ assignmentManager.getRegionStates().createRegionStates(regionInfos);
+ assignmentManager.assign(regionInfos);
+ } catch (InterruptedException e) {
+ LOG.error("Caught " + e + " during round-robin assignment");
+ InterruptedIOException ie = new InterruptedIOException(e.getMessage());
+ ie.initCause(e);
+ throw ie;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
index 351f364..053a946 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin.java
@@ -109,6 +109,9 @@ public class TestAdmin {
@After
public void tearDown() throws Exception {
+ for (HTableDescriptor htd : this.admin.listTables()) {
+ TEST_UTIL.deleteTable(htd.getName());
+ }
}
@Test (timeout=300000)
@@ -341,6 +344,48 @@ public class TestAdmin {
}
@Test (timeout=300000)
+ public void testTruncateTable() throws IOException {
+ testTruncateTable(TableName.valueOf("testTruncateTable"), false);
+ }
+
+ @Test (timeout=300000)
+ public void testTruncateTablePreservingSplits() throws IOException {
+ testTruncateTable(TableName.valueOf("testTruncateTablePreservingSplits"), true);
+ }
+
+ private void testTruncateTable(final TableName tableName, boolean preserveSplits)
+ throws IOException {
+ byte[][] splitKeys = new byte[2][];
+ splitKeys[0] = Bytes.toBytes(4);
+ splitKeys[1] = Bytes.toBytes(8);
+
+ // Create & Fill the table
+ HTable table = TEST_UTIL.createTable(tableName, HConstants.CATALOG_FAMILY, splitKeys);
+ try {
+ TEST_UTIL.loadNumericRows(table, HConstants.CATALOG_FAMILY, 0, 10);
+ assertEquals(10, TEST_UTIL.countRows(table));
+ } finally {
+ table.close();
+ }
+ assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+
+ // Truncate & Verify
+ this.admin.disableTable(tableName);
+ this.admin.truncateTable(tableName, preserveSplits);
+ table = new HTable(TEST_UTIL.getConfiguration(), tableName);
+ try {
+ assertEquals(0, TEST_UTIL.countRows(table));
+ } finally {
+ table.close();
+ }
+ if (preserveSplits) {
+ assertEquals(3, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+ } else {
+ assertEquals(1, TEST_UTIL.getHBaseCluster().getRegions(tableName).size());
+ }
+ }
+
+ @Test (timeout=300000)
public void testGetTableDescriptor() throws IOException {
HColumnDescriptor fam1 = new HColumnDescriptor("fam1");
HColumnDescriptor fam2 = new HColumnDescriptor("fam2");
@@ -366,7 +411,7 @@ public class TestAdmin {
assertTrue(exceptionThrown);
}
}
-
+
/**
* Verify schema modification takes.
* @throws IOException
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
index 9e9bd80..a241c29 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java
@@ -71,6 +71,8 @@ public class TestMasterObserver {
private boolean postCreateTableCalled;
private boolean preDeleteTableCalled;
private boolean postDeleteTableCalled;
+ private boolean preTruncateTableCalled;
+ private boolean postTruncateTableCalled;
private boolean preModifyTableCalled;
private boolean postModifyTableCalled;
private boolean preCreateNamespaceCalled;
@@ -119,6 +121,8 @@ public class TestMasterObserver {
private boolean postCreateTableHandlerCalled;
private boolean preDeleteTableHandlerCalled;
private boolean postDeleteTableHandlerCalled;
+ private boolean preTruncateTableHandlerCalled;
+ private boolean postTruncateTableHandlerCalled;
private boolean preAddColumnHandlerCalled;
private boolean postAddColumnHandlerCalled;
private boolean preModifyColumnHandlerCalled;
@@ -143,6 +147,8 @@ public class TestMasterObserver {
postCreateTableCalled = false;
preDeleteTableCalled = false;
postDeleteTableCalled = false;
+ preTruncateTableCalled = false;
+ postTruncateTableCalled = false;
preModifyTableCalled = false;
postModifyTableCalled = false;
preCreateNamespaceCalled = false;
@@ -185,6 +191,8 @@ public class TestMasterObserver {
postCreateTableHandlerCalled = false;
preDeleteTableHandlerCalled = false;
postDeleteTableHandlerCalled = false;
+ preTruncateTableHandlerCalled = false;
+ postTruncateTableHandlerCalled = false;
preModifyTableHandlerCalled = false;
postModifyTableHandlerCalled = false;
preAddColumnHandlerCalled = false;
@@ -250,6 +258,29 @@ public class TestMasterObserver {
}
@Override
+ public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> env,
+ TableName tableName) throws IOException {
+ if (bypass) {
+ env.bypass();
+ }
+ preTruncateTableCalled = true;
+ }
+
+ @Override
+ public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> env,
+ TableName tableName) throws IOException {
+ postTruncateTableCalled = true;
+ }
+
+ public boolean wasTruncateTableCalled() {
+ return preTruncateTableCalled && postTruncateTableCalled;
+ }
+
+ public boolean preTruncateTableCalledOnly() {
+ return preTruncateTableCalled && !postTruncateTableCalled;
+ }
+
+ @Override
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> env,
TableName tableName, HTableDescriptor htd) throws IOException {
if (bypass) {
@@ -768,6 +799,32 @@ public class TestMasterObserver {
public boolean wasDeleteTableHandlerCalledOnly() {
return preDeleteTableHandlerCalled && !postDeleteTableHandlerCalled;
}
+
+ @Override
+ public void preTruncateTableHandler(
+ ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName)
+ throws IOException {
+ if (bypass) {
+ env.bypass();
+ }
+ preTruncateTableHandlerCalled = true;
+ }
+
+ @Override
+ public void postTruncateTableHandler(
+ ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+ throws IOException {
+ postTruncateTableHandlerCalled = true;
+ }
+
+ public boolean wasTruncateTableHandlerCalled() {
+ return preTruncateTableHandlerCalled && postTruncateTableHandlerCalled;
+ }
+
+ public boolean wasTruncateTableHandlerCalledOnly() {
+ return preTruncateTableHandlerCalled && !postTruncateTableHandlerCalled;
+ }
+
@Override
public void preModifyTableHandler(
ObserverContext<MasterCoprocessorEnvironment> env, TableName tableName,
@@ -1050,7 +1107,12 @@ public class TestMasterObserver {
assertTrue("Second column family should be modified",
cp.preModifyColumnCalledOnly());
+ // truncate table
+ admin.truncateTable(TEST_TABLE, false);
+
// delete table
+ admin.disableTable(TEST_TABLE);
+ assertTrue(admin.isTableDisabled(TEST_TABLE));
admin.deleteTable(TEST_TABLE);
assertFalse("Test table should have been deleted",
admin.tableExists(TEST_TABLE));
@@ -1280,7 +1342,7 @@ public class TestMasterObserver {
try {
UTIL.createMultiRegions(table, TEST_FAMILY);
UTIL.waitUntilAllRegionsAssigned(TEST_TABLE);
-
+
NavigableMap<HRegionInfo, ServerName> regions = table.getRegionLocations();
Map.Entry<HRegionInfo, ServerName> firstGoodPair = null;
for (Map.Entry<HRegionInfo, ServerName> e: regions.entrySet()) {
@@ -1312,15 +1374,15 @@ public class TestMasterObserver {
firstGoodPair.getKey().getEncodedNameAsBytes(),Bytes.toBytes(destName)));
assertTrue("Coprocessor should have been called on region move",
cp.wasMoveCalled());
-
+
// make sure balancer is on
master.balanceSwitch(true);
assertTrue("Coprocessor should have been called on balance switch",
cp.wasBalanceSwitchCalled());
-
+
// turn balancer off
master.balanceSwitch(false);
-
+
// wait for assignments to finish, if any
AssignmentManager mgr = master.getAssignmentManager();
Collection<RegionState> transRegions =
@@ -1328,7 +1390,7 @@ public class TestMasterObserver {
for (RegionState state : transRegions) {
mgr.getRegionStates().waitOnRegionToClearRegionsInTransition(state.getRegion());
}
-
+
// move half the open regions from RS 0 to RS 1
HRegionServer rs = cluster.getRegionServer(0);
byte[] destRS = Bytes.toBytes(cluster.getRegionServer(1).getServerName().toString());
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
index 9037bb3..ade7635 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java
@@ -343,6 +343,10 @@ public class TestCatalogJanitor {
public void deleteTable(TableName tableName) throws IOException { }
@Override
+ public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { }
+
+
+ @Override
public void modifyTable(TableName tableName, HTableDescriptor descriptor)
throws IOException { }
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
index e9e6b1a..714c340 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/TestAccessController.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.Coprocessor;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HColumnDescriptor;
import org.apache.hadoop.hbase.HConstants;
@@ -320,6 +321,22 @@ public class TestAccessController extends SecureTestUtil {
}
@Test
+ public void testTableTruncate() throws Exception {
+ AccessTestAction truncateTable = new AccessTestAction() {
+ @Override
+ public Object run() throws Exception {
+ ACCESS_CONTROLLER
+ .preTruncateTable(ObserverContext.createAndPrepare(CP_ENV, null),
+ TEST_TABLE.getTableName());
+ return null;
+ }
+ };
+
+ verifyAllowed(truncateTable, SUPERUSER, USER_ADMIN);
+ verifyDenied(truncateTable, USER_RW, USER_RO, USER_NONE,USER_CREATE, USER_OWNER);
+ }
+
+ @Test
public void testAddColumn() throws Exception {
final HColumnDescriptor hcd = new HColumnDescriptor("fam_new");
AccessTestAction action = new AccessTestAction() {
@@ -2259,4 +2276,21 @@ public class TestAccessController extends SecureTestUtil {
TEST_UTIL.getMiniHBaseCluster().getMaster().deleteNamespace(namespace);
}
+ @Test
+ public void testTruncatePerms() throws Exception {
+ try {
+ List<UserPermission> existingPerms = AccessControlClient.getUserPermissions(conf, TEST_TABLE
+ .getTableName().getNameAsString());
+ assertTrue(existingPerms != null);
+ assertTrue(existingPerms.size() > 1);
+ TEST_UTIL.getHBaseAdmin().disableTable(TEST_TABLE.getTableName());
+ TEST_UTIL.getHBaseAdmin().truncateTable(TEST_TABLE.getTableName(), true);
+ List<UserPermission> perms = AccessControlClient.getUserPermissions(conf, TEST_TABLE
+ .getTableName().getNameAsString());
+ assertTrue(perms != null);
+ assertEquals(existingPerms.size(), perms.size());
+ } catch (Throwable e) {
+ throw new HBaseIOException(e);
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-shell/src/main/ruby/hbase/admin.rb
----------------------------------------------------------------------
diff --git a/hbase-shell/src/main/ruby/hbase/admin.rb b/hbase-shell/src/main/ruby/hbase/admin.rb
index 428d390..1b545c3 100644
--- a/hbase-shell/src/main/ruby/hbase/admin.rb
+++ b/hbase-shell/src/main/ruby/hbase/admin.rb
@@ -340,11 +340,24 @@ module Hbase
yield 'Disabling table...' if block_given?
@admin.disableTable(table_name)
- yield 'Dropping table...' if block_given?
- @admin.deleteTable(table_name)
-
- yield 'Creating table...' if block_given?
- @admin.createTable(table_description)
+ begin
+ yield 'Truncating table...' if block_given?
+ @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), false)
+ rescue => e
+ # Handle the compatibility case, where the truncate method doesn't exists on the Master
+ raise e unless e.respond_to?(:cause) && e.cause != nil
+ rootCause = e.cause
+ if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
+ # Handle the compatibility case, where the truncate method doesn't exists on the Master
+ yield 'Dropping table...' if block_given?
+ @admin.deleteTable(table_name)
+
+ yield 'Creating table...' if block_given?
+ @admin.createTable(table_description)
+ else
+ raise e
+ end
+ end
end
#----------------------------------------------------------------------------------------------
@@ -357,11 +370,24 @@ module Hbase
yield 'Disabling table...' if block_given?
disable(table_name)
- yield 'Dropping table...' if block_given?
- drop(table_name)
-
- yield 'Creating table with region boundaries...' if block_given?
- @admin.createTable(table_description, splits)
+ begin
+ yield 'Truncating table...' if block_given?
+ @admin.truncateTable(org.apache.hadoop.hbase.TableName.valueOf(table_name), true)
+ rescue => e
+ # Handle the compatibility case, where the truncate method doesn't exists on the Master
+ raise e unless e.respond_to?(:cause) && e.cause != nil
+ rootCause = e.cause
+ if rootCause.kind_of?(org.apache.hadoop.hbase.DoNotRetryIOException) then
+ # Handle the compatibility case, where the truncate method doesn't exists on the Master
+ yield 'Dropping table...' if block_given?
+ @admin.deleteTable(table_name)
+
+ yield 'Creating table with region boundaries...' if block_given?
+ @admin.createTable(table_description, splits)
+ else
+ raise e
+ end
+ end
end
#----------------------------------------------------------------------------------------------
[2/2] git commit: HBASE-12142 Truncate command does not preserve ACLs
table (Vandana Ayyalasomayajula)
Posted by ap...@apache.org.
HBASE-12142 Truncate command does not preserve ACLs table (Vandana Ayyalasomayajula)
Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5b1380c3
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5b1380c3
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5b1380c3
Branch: refs/heads/0.98
Commit: 5b1380c3c691007625d624096f57559124518288
Parents: 714e836
Author: Andrew Purtell <ap...@apache.org>
Authored: Fri Oct 24 16:33:01 2014 -0700
Committer: Andrew Purtell <ap...@apache.org>
Committed: Fri Oct 24 16:33:01 2014 -0700
----------------------------------------------------------------------
.../apache/hadoop/hbase/client/HBaseAdmin.java | 21 +
.../hadoop/hbase/client/HConnectionManager.java | 8 +
.../hadoop/hbase/protobuf/RequestConverter.java | 16 +
.../hbase/protobuf/generated/MasterProtos.java | 1248 ++++++++++++++++--
hbase-protocol/src/main/protobuf/Master.proto | 12 +
.../hbase/coprocessor/BaseMasterObserver.java | 22 +
.../hbase/coprocessor/MasterObserver.java | 46 +
.../org/apache/hadoop/hbase/master/HMaster.java | 30 +
.../hbase/master/MasterCoprocessorHost.java | 68 +
.../hadoop/hbase/master/MasterServices.java | 8 +
.../master/handler/TruncateTableHandler.java | 247 ++++
.../hbase/security/access/AccessController.java | 34 +
.../visibility/VisibilityController.java | 20 +
.../hadoop/hbase/util/ModifyRegionUtils.java | 21 +
.../apache/hadoop/hbase/client/TestAdmin.java | 47 +-
.../hbase/coprocessor/TestMasterObserver.java | 72 +-
.../hadoop/hbase/master/TestCatalogJanitor.java | 4 +
.../security/access/TestAccessController.java | 34 +
hbase-shell/src/main/ruby/hbase/admin.rb | 46 +-
19 files changed, 1910 insertions(+), 94 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
index 3cf57e5..022c382 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java
@@ -128,6 +128,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException;
import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException;
@@ -3403,4 +3404,24 @@ public class HBaseAdmin implements Abortable, Closeable {
return new RegionServerCoprocessorRpcChannel(connection, sn);
}
+ /**
+ * Truncate a table. Synchronous operation.
+ * @param tableName name of table to truncate
+ * @param preserveSplits True if the splits should be preserved
+ * @throws IOException if a remote or network exception occurs
+ */
+ public void truncateTable(final TableName tableName, final boolean preserveSplits)
+ throws IOException {
+ executeCallable(new MasterCallable<Void>(getConnection()) {
+ @Override
+ public Void call() throws ServiceException {
+ LOG.info("Started truncate of " + tableName);
+ TruncateTableRequest req = RequestConverter.buildTruncateTableRequest(
+ tableName, preserveSplits);
+ master.truncateTable(null, req);
+ return null;
+ }
+ });
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
index 30046b2..4fa5a8b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HConnectionManager.java
@@ -110,6 +110,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunnin
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ShutdownResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AddColumnRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.BalanceRequest;
@@ -2128,6 +2130,12 @@ public class HConnectionManager {
throws ServiceException {
return stub.getClusterStatus(controller, request);
}
+
+ @Override
+ public TruncateTableResponse truncateTable(RpcController controller,
+ TruncateTableRequest request) throws ServiceException {
+ return stub.truncateTable(controller, request);
+ }
};
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
index 756bc43..f5c128b 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java
@@ -98,6 +98,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MoveRegionRequest
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.OfflineRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.RunCatalogScanRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetBalancerRunningRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest;
import org.apache.hadoop.hbase.util.ByteStringer;
@@ -1142,6 +1143,21 @@ public final class RequestConverter {
}
/**
+ * Creates a protocol buffer TruncateTableRequest
+ *
+ * @param tableName name of table to truncate
+ * @param preserveSplits True if the splits should be preserved
+ * @return a TruncateTableRequest
+ */
+ public static TruncateTableRequest buildTruncateTableRequest(final TableName tableName,
+ boolean preserveSplits) {
+ TruncateTableRequest.Builder builder = TruncateTableRequest.newBuilder();
+ builder.setTableName(ProtobufUtil.toProtoTableName(tableName));
+ builder.setPreserveSplits(preserveSplits);
+ return builder.build();
+ }
+
+ /**
* Creates a protocol buffer EnableTableRequest
*
* @param tableName
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
index 83a2152..e52ad17 100644
--- a/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
+++ b/hbase-protocol/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterProtos.java
@@ -39552,6 +39552,995 @@ public final class MasterProtos {
// @@protoc_insertion_point(class_scope:IsProcedureDoneResponse)
}
+ public interface TruncateTableRequestOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+
+ // required .TableName tableName = 1;
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ boolean hasTableName();
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName();
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder();
+
+ // optional bool preserveSplits = 2 [default = false];
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ boolean hasPreserveSplits();
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ boolean getPreserveSplits();
+ }
+ /**
+ * Protobuf type {@code TruncateTableRequest}
+ */
+ public static final class TruncateTableRequest extends
+ com.google.protobuf.GeneratedMessage
+ implements TruncateTableRequestOrBuilder {
+ // Use TruncateTableRequest.newBuilder() to construct.
+ private TruncateTableRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private TruncateTableRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final TruncateTableRequest defaultInstance;
+ public static TruncateTableRequest getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public TruncateTableRequest getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private TruncateTableRequest(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ int mutable_bitField0_ = 0;
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ case 10: {
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder subBuilder = null;
+ if (((bitField0_ & 0x00000001) == 0x00000001)) {
+ subBuilder = tableName_.toBuilder();
+ }
+ tableName_ = input.readMessage(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.PARSER, extensionRegistry);
+ if (subBuilder != null) {
+ subBuilder.mergeFrom(tableName_);
+ tableName_ = subBuilder.buildPartial();
+ }
+ bitField0_ |= 0x00000001;
+ break;
+ }
+ case 16: {
+ bitField0_ |= 0x00000002;
+ preserveSplits_ = input.readBool();
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<TruncateTableRequest> PARSER =
+ new com.google.protobuf.AbstractParser<TruncateTableRequest>() {
+ public TruncateTableRequest parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new TruncateTableRequest(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<TruncateTableRequest> getParserForType() {
+ return PARSER;
+ }
+
+ private int bitField0_;
+ // required .TableName tableName = 1;
+ public static final int TABLENAME_FIELD_NUMBER = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_;
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public boolean hasTableName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+ return tableName_;
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+ return tableName_;
+ }
+
+ // optional bool preserveSplits = 2 [default = false];
+ public static final int PRESERVESPLITS_FIELD_NUMBER = 2;
+ private boolean preserveSplits_;
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public boolean hasPreserveSplits() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public boolean getPreserveSplits() {
+ return preserveSplits_;
+ }
+
+ private void initFields() {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ preserveSplits_ = 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 (!getTableName().isInitialized()) {
+ 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.writeMessage(1, tableName_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ output.writeBool(2, preserveSplits_);
+ }
+ 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
+ .computeMessageSize(1, tableName_);
+ }
+ if (((bitField0_ & 0x00000002) == 0x00000002)) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBoolSize(2, preserveSplits_);
+ }
+ 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.MasterProtos.TruncateTableRequest)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) obj;
+
+ boolean result = true;
+ result = result && (hasTableName() == other.hasTableName());
+ if (hasTableName()) {
+ result = result && getTableName()
+ .equals(other.getTableName());
+ }
+ result = result && (hasPreserveSplits() == other.hasPreserveSplits());
+ if (hasPreserveSplits()) {
+ result = result && (getPreserveSplits()
+ == other.getPreserveSplits());
+ }
+ result = result &&
+ getUnknownFields().equals(other.getUnknownFields());
+ return result;
+ }
+
+ private int memoizedHashCode = 0;
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ if (hasTableName()) {
+ hash = (37 * hash) + TABLENAME_FIELD_NUMBER;
+ hash = (53 * hash) + getTableName().hashCode();
+ }
+ if (hasPreserveSplits()) {
+ hash = (37 * hash) + PRESERVESPLITS_FIELD_NUMBER;
+ hash = (53 * hash) + hashBoolean(getPreserveSplits());
+ }
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest 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;
+ }
+ /**
+ * Protobuf type {@code TruncateTableRequest}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequestOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+ super(parent);
+ maybeForceBuilderInitialization();
+ }
+ private void maybeForceBuilderInitialization() {
+ if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+ getTableNameFieldBuilder();
+ }
+ }
+ private static Builder create() {
+ return new Builder();
+ }
+
+ public Builder clear() {
+ super.clear();
+ if (tableNameBuilder_ == null) {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ } else {
+ tableNameBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ preserveSplits_ = false;
+ bitField0_ = (bitField0_ & ~0x00000002);
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(buildPartial());
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableRequest_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest build() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest(this);
+ int from_bitField0_ = bitField0_;
+ int to_bitField0_ = 0;
+ if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+ to_bitField0_ |= 0x00000001;
+ }
+ if (tableNameBuilder_ == null) {
+ result.tableName_ = tableName_;
+ } else {
+ result.tableName_ = tableNameBuilder_.build();
+ }
+ if (((from_bitField0_ & 0x00000002) == 0x00000002)) {
+ to_bitField0_ |= 0x00000002;
+ }
+ result.preserveSplits_ = preserveSplits_;
+ result.bitField0_ = to_bitField0_;
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance()) return this;
+ if (other.hasTableName()) {
+ mergeTableName(other.getTableName());
+ }
+ if (other.hasPreserveSplits()) {
+ setPreserveSplits(other.getPreserveSplits());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public final boolean isInitialized() {
+ if (!hasTableName()) {
+
+ return false;
+ }
+ if (!getTableName().isInitialized()) {
+
+ return false;
+ }
+ return true;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+ private int bitField0_;
+
+ // required .TableName tableName = 1;
+ private org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder> tableNameBuilder_;
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public boolean hasTableName() {
+ return ((bitField0_ & 0x00000001) == 0x00000001);
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName getTableName() {
+ if (tableNameBuilder_ == null) {
+ return tableName_;
+ } else {
+ return tableNameBuilder_.getMessage();
+ }
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public Builder setTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+ if (tableNameBuilder_ == null) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ tableName_ = value;
+ onChanged();
+ } else {
+ tableNameBuilder_.setMessage(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public Builder setTableName(
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder builderForValue) {
+ if (tableNameBuilder_ == null) {
+ tableName_ = builderForValue.build();
+ onChanged();
+ } else {
+ tableNameBuilder_.setMessage(builderForValue.build());
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public Builder mergeTableName(org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName value) {
+ if (tableNameBuilder_ == null) {
+ if (((bitField0_ & 0x00000001) == 0x00000001) &&
+ tableName_ != org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance()) {
+ tableName_ =
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.newBuilder(tableName_).mergeFrom(value).buildPartial();
+ } else {
+ tableName_ = value;
+ }
+ onChanged();
+ } else {
+ tableNameBuilder_.mergeFrom(value);
+ }
+ bitField0_ |= 0x00000001;
+ return this;
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public Builder clearTableName() {
+ if (tableNameBuilder_ == null) {
+ tableName_ = org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.getDefaultInstance();
+ onChanged();
+ } else {
+ tableNameBuilder_.clear();
+ }
+ bitField0_ = (bitField0_ & ~0x00000001);
+ return this;
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder getTableNameBuilder() {
+ bitField0_ |= 0x00000001;
+ onChanged();
+ return getTableNameFieldBuilder().getBuilder();
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ public org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder getTableNameOrBuilder() {
+ if (tableNameBuilder_ != null) {
+ return tableNameBuilder_.getMessageOrBuilder();
+ } else {
+ return tableName_;
+ }
+ }
+ /**
+ * <code>required .TableName tableName = 1;</code>
+ */
+ private com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>
+ getTableNameFieldBuilder() {
+ if (tableNameBuilder_ == null) {
+ tableNameBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+ org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableName.Builder, org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableNameOrBuilder>(
+ tableName_,
+ getParentForChildren(),
+ isClean());
+ tableName_ = null;
+ }
+ return tableNameBuilder_;
+ }
+
+ // optional bool preserveSplits = 2 [default = false];
+ private boolean preserveSplits_ ;
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public boolean hasPreserveSplits() {
+ return ((bitField0_ & 0x00000002) == 0x00000002);
+ }
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public boolean getPreserveSplits() {
+ return preserveSplits_;
+ }
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public Builder setPreserveSplits(boolean value) {
+ bitField0_ |= 0x00000002;
+ preserveSplits_ = value;
+ onChanged();
+ return this;
+ }
+ /**
+ * <code>optional bool preserveSplits = 2 [default = false];</code>
+ */
+ public Builder clearPreserveSplits() {
+ bitField0_ = (bitField0_ & ~0x00000002);
+ preserveSplits_ = false;
+ onChanged();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:TruncateTableRequest)
+ }
+
+ static {
+ defaultInstance = new TruncateTableRequest(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:TruncateTableRequest)
+ }
+
+ public interface TruncateTableResponseOrBuilder
+ extends com.google.protobuf.MessageOrBuilder {
+ }
+ /**
+ * Protobuf type {@code TruncateTableResponse}
+ */
+ public static final class TruncateTableResponse extends
+ com.google.protobuf.GeneratedMessage
+ implements TruncateTableResponseOrBuilder {
+ // Use TruncateTableResponse.newBuilder() to construct.
+ private TruncateTableResponse(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+ super(builder);
+ this.unknownFields = builder.getUnknownFields();
+ }
+ private TruncateTableResponse(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+ private static final TruncateTableResponse defaultInstance;
+ public static TruncateTableResponse getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public TruncateTableResponse getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ private final com.google.protobuf.UnknownFieldSet unknownFields;
+ @java.lang.Override
+ public final com.google.protobuf.UnknownFieldSet
+ getUnknownFields() {
+ return this.unknownFields;
+ }
+ private TruncateTableResponse(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ initFields();
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder();
+ try {
+ boolean done = false;
+ while (!done) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ done = true;
+ break;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ done = true;
+ }
+ break;
+ }
+ }
+ }
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ throw e.setUnfinishedMessage(this);
+ } catch (java.io.IOException e) {
+ throw new com.google.protobuf.InvalidProtocolBufferException(
+ e.getMessage()).setUnfinishedMessage(this);
+ } finally {
+ this.unknownFields = unknownFields.build();
+ makeExtensionsImmutable();
+ }
+ }
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.Builder.class);
+ }
+
+ public static com.google.protobuf.Parser<TruncateTableResponse> PARSER =
+ new com.google.protobuf.AbstractParser<TruncateTableResponse>() {
+ public TruncateTableResponse parsePartialFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return new TruncateTableResponse(input, extensionRegistry);
+ }
+ };
+
+ @java.lang.Override
+ public com.google.protobuf.Parser<TruncateTableResponse> getParserForType() {
+ return PARSER;
+ }
+
+ 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.MasterProtos.TruncateTableResponse)) {
+ return super.equals(obj);
+ }
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) obj;
+
+ boolean result = true;
+ result = result &&
+ getUnknownFields().equals(other.getUnknownFields());
+ return result;
+ }
+
+ private int memoizedHashCode = 0;
+ @java.lang.Override
+ public int hashCode() {
+ if (memoizedHashCode != 0) {
+ return memoizedHashCode;
+ }
+ int hash = 41;
+ hash = (19 * hash) + getDescriptorForType().hashCode();
+ hash = (29 * hash) + getUnknownFields().hashCode();
+ memoizedHashCode = hash;
+ return hash;
+ }
+
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return PARSER.parseFrom(data, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseDelimitedFrom(input, extensionRegistry);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input);
+ }
+ public static org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return PARSER.parseFrom(input, extensionRegistry);
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse 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;
+ }
+ /**
+ * Protobuf type {@code TruncateTableResponse}
+ */
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder<Builder>
+ implements org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponseOrBuilder {
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.internal_static_TruncateTableResponse_fieldAccessorTable
+ .ensureFieldAccessorsInitialized(
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.Builder.class);
+ }
+
+ // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.newBuilder()
+ private Builder() {
+ maybeForceBuilderInitialization();
+ }
+
+ private Builder(
+ com.google.protobuf.GeneratedMessage.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.MasterProtos.internal_static_TruncateTableResponse_descriptor;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse getDefaultInstanceForType() {
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse build() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse result = buildPartial();
+ if (!result.isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return result;
+ }
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse buildPartial() {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse(this);
+ onBuilt();
+ return result;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) {
+ return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse other) {
+ if (other == org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.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 {
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse parsedMessage = null;
+ try {
+ parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+ } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+ parsedMessage = (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) e.getUnfinishedMessage();
+ throw e;
+ } finally {
+ if (parsedMessage != null) {
+ mergeFrom(parsedMessage);
+ }
+ }
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:TruncateTableResponse)
+ }
+
+ static {
+ defaultInstance = new TruncateTableResponse(true);
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:TruncateTableResponse)
+ }
+
/**
* Protobuf service {@code MasterService}
*/
@@ -40077,6 +41066,18 @@ public final class MasterProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse> done);
+ /**
+ * <code>rpc truncateTable(.TruncateTableRequest) returns (.TruncateTableResponse);</code>
+ *
+ * <pre>
+ ** Truncate a table
+ * </pre>
+ */
+ public abstract void truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+ com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done);
+
}
public static com.google.protobuf.Service newReflectiveService(
@@ -40410,6 +41411,14 @@ public final class MasterProtos {
impl.listTableNamesByNamespace(controller, request, done);
}
+ @java.lang.Override
+ public void truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+ com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done) {
+ impl.truncateTable(controller, request, done);
+ }
+
};
}
@@ -40514,6 +41523,8 @@ public final class MasterProtos {
return impl.listTableDescriptorsByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest)request);
case 40:
return impl.listTableNamesByNamespace(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest)request);
+ case 41:
+ return impl.truncateTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)request);
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -40610,6 +41621,8 @@ public final class MasterProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance();
case 40:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance();
+ case 41:
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -40706,6 +41719,8 @@ public final class MasterProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance();
case 40:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance();
+ case 41:
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -41231,6 +42246,18 @@ public final class MasterProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request,
com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse> done);
+ /**
+ * <code>rpc truncateTable(.TruncateTableRequest) returns (.TruncateTableResponse);</code>
+ *
+ * <pre>
+ ** Truncate a table
+ * </pre>
+ */
+ public abstract void truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+ com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done);
+
public static final
com.google.protobuf.Descriptors.ServiceDescriptor
getDescriptor() {
@@ -41458,6 +42485,11 @@ public final class MasterProtos {
com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse>specializeCallback(
done));
return;
+ case 41:
+ this.truncateTable(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest)request,
+ com.google.protobuf.RpcUtil.<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse>specializeCallback(
+ done));
+ return;
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -41554,6 +42586,8 @@ public final class MasterProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceRequest.getDefaultInstance();
case 40:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest.getDefaultInstance();
+ case 41:
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -41650,6 +42684,8 @@ public final class MasterProtos {
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableDescriptorsByNamespaceResponse.getDefaultInstance();
case 40:
return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance();
+ case 41:
+ return org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance();
default:
throw new java.lang.AssertionError("Can't get here.");
}
@@ -42285,6 +43321,21 @@ public final class MasterProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.class,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance()));
}
+
+ public void truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request,
+ com.google.protobuf.RpcCallback<org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse> done) {
+ channel.callMethod(
+ getDescriptor().getMethods().get(41),
+ controller,
+ request,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance(),
+ com.google.protobuf.RpcUtil.generalizeCallback(
+ done,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.class,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance()));
+ }
}
public static BlockingInterface newBlockingStub(
@@ -42497,6 +43548,11 @@ public final class MasterProtos {
com.google.protobuf.RpcController controller,
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceRequest request)
throws com.google.protobuf.ServiceException;
+
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request)
+ throws com.google.protobuf.ServiceException;
}
private static final class BlockingStub implements BlockingInterface {
@@ -42997,9 +44053,19 @@ public final class MasterProtos {
org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ListTableNamesByNamespaceResponse.getDefaultInstance());
}
- }
- // @@protoc_insertion_point(class_scope:MasterService)
+ public org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse truncateTable(
+ com.google.protobuf.RpcController controller,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest request)
+ throws com.google.protobuf.ServiceException {
+ return (org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse) channel.callBlockingMethod(
+ getDescriptor().getMethods().get(41),
+ controller,
+ request,
+ org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse.getDefaultInstance());
+ }
+
+ }
}
private static com.google.protobuf.Descriptors.Descriptor
@@ -43402,6 +44468,16 @@ public final class MasterProtos {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_IsProcedureDoneResponse_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_TruncateTableRequest_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_TruncateTableRequest_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_TruncateTableResponse_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_TruncateTableResponse_fieldAccessorTable;
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
@@ -43516,82 +44592,86 @@ public final class MasterProtos {
"oneRequest\022(\n\tprocedure\030\001 \001(\0132\025.Procedur" +
"eDescription\"W\n\027IsProcedureDoneResponse\022" +
"\023\n\004done\030\001 \001(\010:\005false\022\'\n\010snapshot\030\002 \001(\0132\025" +
- ".ProcedureDescription2\356\026\n\rMasterService\022" +
- "S\n\024GetSchemaAlterStatus\022\034.GetSchemaAlter" +
- "StatusRequest\032\035.GetSchemaAlterStatusResp" +
- "onse\022P\n\023GetTableDescriptors\022\033.GetTableDe" +
- "scriptorsRequest\032\034.GetTableDescriptorsRe",
- "sponse\022>\n\rGetTableNames\022\025.GetTableNamesR" +
- "equest\032\026.GetTableNamesResponse\022G\n\020GetClu" +
- "sterStatus\022\030.GetClusterStatusRequest\032\031.G" +
- "etClusterStatusResponse\022D\n\017IsMasterRunni" +
- "ng\022\027.IsMasterRunningRequest\032\030.IsMasterRu" +
- "nningResponse\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" +
- "se\022Y\n\026DispatchMergingRegions\022\036.DispatchM" +
- "ergingRegionsRequest\032\037.DispatchMergingRe" +
- "gionsResponse\022;\n\014AssignRegion\022\024.AssignRe" +
- "gionRequest\032\025.AssignRegionResponse\022A\n\016Un" +
- "assignRegion\022\026.UnassignRegionRequest\032\027.U" +
- "nassignRegionResponse\022>\n\rOfflineRegion\022\025" +
- ".OfflineRegionRequest\032\026.OfflineRegionRes" +
- "ponse\0228\n\013DeleteTable\022\023.DeleteTableReques" +
- "t\032\024.DeleteTableResponse\0228\n\013EnableTable\022\023",
- ".EnableTableRequest\032\024.EnableTableRespons" +
- "e\022;\n\014DisableTable\022\024.DisableTableRequest\032" +
- "\025.DisableTableResponse\0228\n\013ModifyTable\022\023." +
- "ModifyTableRequest\032\024.ModifyTableResponse" +
- "\0228\n\013CreateTable\022\023.CreateTableRequest\032\024.C" +
- "reateTableResponse\022/\n\010Shutdown\022\020.Shutdow" +
- "nRequest\032\021.ShutdownResponse\0225\n\nStopMaste" +
- "r\022\022.StopMasterRequest\032\023.StopMasterRespon" +
- "se\022,\n\007Balance\022\017.BalanceRequest\032\020.Balance" +
- "Response\022M\n\022SetBalancerRunning\022\032.SetBala",
- "ncerRunningRequest\032\033.SetBalancerRunningR" +
- "esponse\022A\n\016RunCatalogScan\022\026.RunCatalogSc" +
- "anRequest\032\027.RunCatalogScanResponse\022S\n\024En" +
- "ableCatalogJanitor\022\034.EnableCatalogJanito" +
- "rRequest\032\035.EnableCatalogJanitorResponse\022" +
- "\\\n\027IsCatalogJanitorEnabled\022\037.IsCatalogJa" +
- "nitorEnabledRequest\032 .IsCatalogJanitorEn" +
- "abledResponse\022L\n\021ExecMasterService\022\032.Cop" +
- "rocessorServiceRequest\032\033.CoprocessorServ" +
- "iceResponse\022/\n\010Snapshot\022\020.SnapshotReques",
- "t\032\021.SnapshotResponse\022V\n\025GetCompletedSnap" +
- "shots\022\035.GetCompletedSnapshotsRequest\032\036.G" +
- "etCompletedSnapshotsResponse\022A\n\016DeleteSn" +
- "apshot\022\026.DeleteSnapshotRequest\032\027.DeleteS" +
- "napshotResponse\022A\n\016IsSnapshotDone\022\026.IsSn" +
- "apshotDoneRequest\032\027.IsSnapshotDoneRespon" +
- "se\022D\n\017RestoreSnapshot\022\027.RestoreSnapshotR" +
- "equest\032\030.RestoreSnapshotResponse\022V\n\025IsRe" +
- "storeSnapshotDone\022\035.IsRestoreSnapshotDon" +
- "eRequest\032\036.IsRestoreSnapshotDoneResponse",
- "\022>\n\rExecProcedure\022\025.ExecProcedureRequest" +
- "\032\026.ExecProcedureResponse\022D\n\017IsProcedureD" +
- "one\022\027.IsProcedureDoneRequest\032\030.IsProcedu" +
- "reDoneResponse\022D\n\017ModifyNamespace\022\027.Modi" +
- "fyNamespaceRequest\032\030.ModifyNamespaceResp" +
- "onse\022D\n\017CreateNamespace\022\027.CreateNamespac" +
- "eRequest\032\030.CreateNamespaceResponse\022D\n\017De" +
- "leteNamespace\022\027.DeleteNamespaceRequest\032\030" +
- ".DeleteNamespaceResponse\022Y\n\026GetNamespace" +
- "Descriptor\022\036.GetNamespaceDescriptorReque",
- "st\032\037.GetNamespaceDescriptorResponse\022_\n\030L" +
- "istNamespaceDescriptors\022 .ListNamespaceD" +
- "escriptorsRequest\032!.ListNamespaceDescrip" +
- "torsResponse\022t\n\037ListTableDescriptorsByNa" +
- "mespace\022\'.ListTableDescriptorsByNamespac" +
- "eRequest\032(.ListTableDescriptorsByNamespa" +
- "ceResponse\022b\n\031ListTableNamesByNamespace\022" +
- "!.ListTableNamesByNamespaceRequest\032\".Lis" +
- "tTableNamesByNamespaceResponseBB\n*org.ap" +
- "ache.hadoop.hbase.protobuf.generatedB\014Ma",
- "sterProtosH\001\210\001\001\240\001\001"
+ ".ProcedureDescription\"T\n\024TruncateTableRe" +
+ "quest\022\035\n\ttableName\030\001 \002(\0132\n.TableName\022\035\n\016" +
+ "preserveSplits\030\002 \001(\010:\005false\"\027\n\025TruncateT" +
+ "ableResponse2\256\027\n\rMasterService\022S\n\024GetSch" +
+ "emaAlterStatus\022\034.GetSchemaAlterStatusReq",
+ "uest\032\035.GetSchemaAlterStatusResponse\022P\n\023G" +
+ "etTableDescriptors\022\033.GetTableDescriptors" +
+ "Request\032\034.GetTableDescriptorsResponse\022>\n" +
+ "\rGetTableNames\022\025.GetTableNamesRequest\032\026." +
+ "GetTableNamesResponse\022G\n\020GetClusterStatu" +
+ "s\022\030.GetClusterStatusRequest\032\031.GetCluster" +
+ "StatusResponse\022D\n\017IsMasterRunning\022\027.IsMa" +
+ "sterRunningRequest\032\030.IsMasterRunningResp" +
+ "onse\0222\n\tAddColumn\022\021.AddColumnRequest\032\022.A" +
+ "ddColumnResponse\022;\n\014DeleteColumn\022\024.Delet",
+ "eColumnRequest\032\025.DeleteColumnResponse\022;\n" +
+ "\014ModifyColumn\022\024.ModifyColumnRequest\032\025.Mo" +
+ "difyColumnResponse\0225\n\nMoveRegion\022\022.MoveR" +
+ "egionRequest\032\023.MoveRegionResponse\022Y\n\026Dis" +
+ "patchMergingRegions\022\036.DispatchMergingReg" +
+ "ionsRequest\032\037.DispatchMergingRegionsResp" +
+ "onse\022;\n\014AssignRegion\022\024.AssignRegionReque" +
+ "st\032\025.AssignRegionResponse\022A\n\016UnassignReg" +
+ "ion\022\026.UnassignRegionRequest\032\027.UnassignRe" +
+ "gionResponse\022>\n\rOfflineRegion\022\025.OfflineR",
+ "egionRequest\032\026.OfflineRegionResponse\0228\n\013" +
+ "DeleteTable\022\023.DeleteTableRequest\032\024.Delet" +
+ "eTableResponse\0228\n\013EnableTable\022\023.EnableTa" +
+ "bleRequest\032\024.EnableTableResponse\022;\n\014Disa" +
+ "bleTable\022\024.DisableTableRequest\032\025.Disable" +
+ "TableResponse\0228\n\013ModifyTable\022\023.ModifyTab" +
+ "leRequest\032\024.ModifyTableResponse\0228\n\013Creat" +
+ "eTable\022\023.CreateTableRequest\032\024.CreateTabl" +
+ "eResponse\022/\n\010Shutdown\022\020.ShutdownRequest\032" +
+ "\021.ShutdownResponse\0225\n\nStopMaster\022\022.StopM",
+ "asterRequest\032\023.StopMasterResponse\022,\n\007Bal" +
+ "ance\022\017.BalanceRequest\032\020.BalanceResponse\022" +
+ "M\n\022SetBalancerRunning\022\032.SetBalancerRunni" +
+ "ngRequest\032\033.SetBalancerRunningResponse\022A" +
+ "\n\016RunCatalogScan\022\026.RunCatalogScanRequest" +
+ "\032\027.RunCatalogScanResponse\022S\n\024EnableCatal" +
+ "ogJanitor\022\034.EnableCatalogJanitorRequest\032" +
+ "\035.EnableCatalogJanitorResponse\022\\\n\027IsCata" +
+ "logJanitorEnabled\022\037.IsCatalogJanitorEnab" +
+ "ledRequest\032 .IsCatalogJanitorEnabledResp",
+ "onse\022L\n\021ExecMasterService\022\032.CoprocessorS" +
+ "erviceRequest\032\033.CoprocessorServiceRespon" +
+ "se\022/\n\010Snapshot\022\020.SnapshotRequest\032\021.Snaps" +
+ "hotResponse\022V\n\025GetCompletedSnapshots\022\035.G" +
+ "etCompletedSnapshotsRequest\032\036.GetComplet" +
+ "edSnapshotsResponse\022A\n\016DeleteSnapshot\022\026." +
+ "DeleteSnapshotRequest\032\027.DeleteSnapshotRe" +
+ "sponse\022A\n\016IsSnapshotDone\022\026.IsSnapshotDon" +
+ "eRequest\032\027.IsSnapshotDoneResponse\022D\n\017Res" +
+ "toreSnapshot\022\027.RestoreSnapshotRequest\032\030.",
+ "RestoreSnapshotResponse\022V\n\025IsRestoreSnap" +
+ "shotDone\022\035.IsRestoreSnapshotDoneRequest\032" +
+ "\036.IsRestoreSnapshotDoneResponse\022>\n\rExecP" +
+ "rocedure\022\025.ExecProcedureRequest\032\026.ExecPr" +
+ "ocedureResponse\022D\n\017IsProcedureDone\022\027.IsP" +
+ "rocedureDoneRequest\032\030.IsProcedureDoneRes" +
+ "ponse\022D\n\017ModifyNamespace\022\027.ModifyNamespa" +
+ "ceRequest\032\030.ModifyNamespaceResponse\022D\n\017C" +
+ "reateNamespace\022\027.CreateNamespaceRequest\032" +
+ "\030.CreateNamespaceResponse\022D\n\017DeleteNames",
+ "pace\022\027.DeleteNamespaceRequest\032\030.DeleteNa" +
+ "mespaceResponse\022Y\n\026GetNamespaceDescripto" +
+ "r\022\036.GetNamespaceDescriptorRequest\032\037.GetN" +
+ "amespaceDescriptorResponse\022_\n\030ListNamesp" +
+ "aceDescriptors\022 .ListNamespaceDescriptor" +
+ "sRequest\032!.ListNamespaceDescriptorsRespo" +
+ "nse\022t\n\037ListTableDescriptorsByNamespace\022\'" +
+ ".ListTableDescriptorsByNamespaceRequest\032" +
+ "(.ListTableDescriptorsByNamespaceRespons" +
+ "e\022b\n\031ListTableNamesByNamespace\022!.ListTab",
+ "leNamesByNamespaceRequest\032\".ListTableNam" +
+ "esByNamespaceResponse\022>\n\rtruncateTable\022\025" +
+ ".TruncateTableRequest\032\026.TruncateTableRes" +
+ "ponseBB\n*org.apache.hadoop.hbase.protobu" +
+ "f.generatedB\014MasterProtosH\001\210\001\001\240\001\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -44078,6 +45158,18 @@ public final class MasterProtos {
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_IsProcedureDoneResponse_descriptor,
new java.lang.String[] { "Done", "Snapshot", });
+ internal_static_TruncateTableRequest_descriptor =
+ getDescriptor().getMessageTypes().get(80);
+ internal_static_TruncateTableRequest_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_TruncateTableRequest_descriptor,
+ new java.lang.String[] { "TableName", "PreserveSplits", });
+ internal_static_TruncateTableResponse_descriptor =
+ getDescriptor().getMessageTypes().get(81);
+ internal_static_TruncateTableResponse_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_TruncateTableResponse_descriptor,
+ new java.lang.String[] { });
return null;
}
};
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-protocol/src/main/protobuf/Master.proto
----------------------------------------------------------------------
diff --git a/hbase-protocol/src/main/protobuf/Master.proto b/hbase-protocol/src/main/protobuf/Master.proto
index b1e1a72..bdd053c 100644
--- a/hbase-protocol/src/main/protobuf/Master.proto
+++ b/hbase-protocol/src/main/protobuf/Master.proto
@@ -349,6 +349,14 @@ message IsProcedureDoneResponse {
optional ProcedureDescription snapshot = 2;
}
+message TruncateTableRequest {
+ required TableName tableName = 1;
+ optional bool preserveSplits = 2 [default = false];
+}
+
+message TruncateTableResponse {
+}
+
service MasterService {
/** Used by the client to get the number of regions that have received the updated schema */
rpc GetSchemaAlterStatus(GetSchemaAlterStatusRequest)
@@ -547,4 +555,8 @@ service MasterService {
/** returns a list of tables for a given namespace*/
rpc ListTableNamesByNamespace(ListTableNamesByNamespaceRequest)
returns(ListTableNamesByNamespaceResponse);
+
+ /** Truncate a table */
+ rpc truncateTable(TruncateTableRequest)
+ returns(TruncateTableResponse);
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
index c85f3c6..29be365 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java
@@ -83,6 +83,28 @@ public class BaseMasterObserver implements MasterObserver {
}
@Override
+ public void preTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
+
+ @Override
+ public void postTruncateTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException {
+ }
+
+ @Override
+ public void preTruncateTableHandler(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+ throws IOException {
+ }
+
+ @Override
+ public void postTruncateTableHandler(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+ throws IOException {
+ }
+
+ @Override
public void preModifyTable(ObserverContext<MasterCoprocessorEnvironment> ctx,
TableName tableName, HTableDescriptor htd) throws IOException {
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
index bd0d845..d6cb609 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java
@@ -134,6 +134,52 @@ public interface MasterObserver extends Coprocessor {
final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
throws IOException;
+
+ /**
+ * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+ * table. Called as part of truncate table RPC call.
+ * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+ * @param ctx the environment to interact with the framework and master
+ * @param tableName the name of the table
+ */
+ void preTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException;
+
+ /**
+ * Called after the truncateTable operation has been requested. Called as part
+ * of truncate table RPC call.
+ * The truncate is synchronous, so this method will be called when the
+ * truncate operation is terminated.
+ * @param ctx the environment to interact with the framework and master
+ * @param tableName the name of the table
+ */
+ void postTruncateTable(final ObserverContext<MasterCoprocessorEnvironment> ctx,
+ TableName tableName) throws IOException;
+
+ /**
+ * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+ * table. Called as part of truncate table handler and it is sync
+ * to the truncate RPC call.
+ * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+ * @param ctx the environment to interact with the framework and master
+ * @param tableName the name of the table
+ */
+ void preTruncateTableHandler(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+ throws IOException;
+
+ /**
+ * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a
+ * table. Called as part of truncate table handler and it is sync to the
+ * truncate RPC call.
+ * It can't bypass the default action, e.g., ctx.bypass() won't have effect.
+ * @param ctx the environment to interact with the framework and master
+ * @param tableName the name of the table
+ */
+ void postTruncateTableHandler(
+ final ObserverContext<MasterCoprocessorEnvironment> ctx, TableName tableName)
+ throws IOException;
+
/**
* Called prior to modifying a table's properties. Called as part of modify
* table RPC call.
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index af5e297..4e55cb4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -102,6 +102,7 @@ import org.apache.hadoop.hbase.master.handler.ModifyTableHandler;
import org.apache.hadoop.hbase.master.handler.TableAddFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableDeleteFamilyHandler;
import org.apache.hadoop.hbase.master.handler.TableModifyFamilyHandler;
+import org.apache.hadoop.hbase.master.handler.TruncateTableHandler;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.monitoring.MemoryBoundedLogMessageBuffer;
import org.apache.hadoop.hbase.monitoring.MonitoredTask;
@@ -191,6 +192,8 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.StopMasterResponse;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableRequest;
+import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.TruncateTableResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.UnassignRegionResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.ExecProcedureRequest;
@@ -3225,4 +3228,31 @@ MasterServices, Server {
}
}
+ @Override
+ public void truncateTable(TableName tableName, boolean preserveSplits) throws IOException {
+ checkInitialized();
+ if (cpHost != null) {
+ cpHost.preTruncateTable(tableName);
+ }
+ LOG.info(getClientIdAuditPrefix() + " truncate " + tableName);
+ TruncateTableHandler handler = new TruncateTableHandler(tableName, this, this, preserveSplits);
+ handler.prepare();
+ handler.process();
+ if (cpHost != null) {
+ cpHost.postTruncateTable(tableName);
+ }
+ }
+
+
+ @Override
+ public TruncateTableResponse truncateTable(RpcController controller, TruncateTableRequest request)
+ throws ServiceException {
+ try {
+ truncateTable(ProtobufUtil.toTableName(request.getTableName()), request.getPreserveSplits());
+ } catch (IOException e) {
+ throw new ServiceException(e);
+ }
+ return TruncateTableResponse.newBuilder().build();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
index 60669c4..34a8b0c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java
@@ -225,6 +225,74 @@ public class MasterCoprocessorHost
});
}
+ public void preTruncateTable(TableName tableName) throws IOException {
+ ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+ for (MasterEnvironment env: coprocessors) {
+ if (env.getInstance() instanceof MasterObserver) {
+ ctx = ObserverContext.createAndPrepare(env, ctx);
+ try {
+ ((MasterObserver)env.getInstance()).preTruncateTable(ctx, tableName);
+ } catch (Throwable e) {
+ handleCoprocessorThrowable(env, e);
+ }
+ if (ctx.shouldComplete()) {
+ break;
+ }
+ }
+ }
+ }
+
+ public void postTruncateTable(TableName tableName) throws IOException {
+ ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+ for (MasterEnvironment env: coprocessors) {
+ if (env.getInstance() instanceof MasterObserver) {
+ ctx = ObserverContext.createAndPrepare(env, ctx);
+ try {
+ ((MasterObserver)env.getInstance()).postTruncateTable(ctx, tableName);
+ } catch (Throwable e) {
+ handleCoprocessorThrowable(env, e);
+ }
+ if (ctx.shouldComplete()) {
+ break;
+ }
+ }
+ }
+ }
+
+ public void preTruncateTableHandler(TableName tableName) throws IOException {
+ ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+ for (MasterEnvironment env : coprocessors) {
+ if (env.getInstance() instanceof MasterObserver) {
+ ctx = ObserverContext.createAndPrepare(env, ctx);
+ try {
+ ((MasterObserver) env.getInstance()).preTruncateTableHandler(ctx, tableName);
+ } catch (Throwable e) {
+ handleCoprocessorThrowable(env, e);
+ }
+ if (ctx.shouldComplete()) {
+ break;
+ }
+ }
+ }
+ }
+
+ public void postTruncateTableHandler(TableName tableName) throws IOException {
+ ObserverContext<MasterCoprocessorEnvironment> ctx = null;
+ for (MasterEnvironment env : coprocessors) {
+ if (env.getInstance() instanceof MasterObserver) {
+ ctx = ObserverContext.createAndPrepare(env, ctx);
+ try {
+ ((MasterObserver) env.getInstance()).postTruncateTableHandler(ctx, tableName);
+ } catch (Throwable e) {
+ handleCoprocessorThrowable(env, e);
+ }
+ if (ctx.shouldComplete()) {
+ break;
+ }
+ }
+ }
+ }
+
public void preModifyTable(final TableName tableName, final HTableDescriptor htd)
throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
http://git-wip-us.apache.org/repos/asf/hbase/blob/5b1380c3/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
index a5765ae..c402758 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java
@@ -98,6 +98,14 @@ public interface MasterServices extends Server {
void deleteTable(final TableName tableName) throws IOException;
/**
+ * Truncate a table
+ * @param tableName The table name
+ * @param preserveSplits True if the splits should be preserved
+ * @throws IOException
+ */
+ public void truncateTable(final TableName tableName, boolean preserveSplits) throws IOException;
+
+ /**
* Modify the descriptor of an existing table
* @param tableName The table name
* @param descriptor The updated table descriptor