HBASE-15607 Remove PB references from Admin for 2.0 (Ram)

This commit is contained in:
Ramkrishna 2016-05-03 10:51:46 +05:30
parent d77972ff16
commit c06a976a98
45 changed files with 611 additions and 270 deletions

View File

@ -41,9 +41,6 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.security.SecurityCapability; import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.quotas.QuotaFilter; import org.apache.hadoop.hbase.quotas.QuotaFilter;
import org.apache.hadoop.hbase.quotas.QuotaRetriever; import org.apache.hadoop.hbase.quotas.QuotaRetriever;
import org.apache.hadoop.hbase.quotas.QuotaSettings; import org.apache.hadoop.hbase.quotas.QuotaSettings;
@ -1153,7 +1150,7 @@ public interface Admin extends Abortable, Closeable {
* @return the current compaction state * @return the current compaction state
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(final TableName tableName) CompactionState getCompactionState(final TableName tableName)
throws IOException; throws IOException;
/** /**
@ -1164,7 +1161,7 @@ public interface Admin extends Abortable, Closeable {
* @return the current compaction state * @return the current compaction state
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionStateForRegion( CompactionState getCompactionStateForRegion(
final byte[] regionName) throws IOException; final byte[] regionName) throws IOException;
/** /**
@ -1244,7 +1241,7 @@ public interface Admin extends Abortable, Closeable {
*/ */
void snapshot(final String snapshotName, void snapshot(final String snapshotName,
final TableName tableName, final TableName tableName,
HBaseProtos.SnapshotDescription.Type type) throws IOException, SnapshotCreationException, SnapshotType type) throws IOException, SnapshotCreationException,
IllegalArgumentException; IllegalArgumentException;
/** /**
@ -1265,7 +1262,7 @@ public interface Admin extends Abortable, Closeable {
* @throws SnapshotCreationException if snapshot failed to be taken * @throws SnapshotCreationException if snapshot failed to be taken
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/ */
void snapshot(HBaseProtos.SnapshotDescription snapshot) void snapshot(SnapshotDescription snapshot)
throws IOException, SnapshotCreationException, IllegalArgumentException; throws IOException, SnapshotCreationException, IllegalArgumentException;
/** /**
@ -1273,12 +1270,11 @@ public interface Admin extends Abortable, Closeable {
* single snapshot should be taken at a time, or results may be undefined. * single snapshot should be taken at a time, or results may be undefined.
* *
* @param snapshot snapshot to take * @param snapshot snapshot to take
* @return response from the server indicating the max time to wait for the snapshot
* @throws IOException if the snapshot did not succeed or we lose contact with the master. * @throws IOException if the snapshot did not succeed or we lose contact with the master.
* @throws SnapshotCreationException if snapshot creation failed * @throws SnapshotCreationException if snapshot creation failed
* @throws IllegalArgumentException if the snapshot request is formatted incorrectly * @throws IllegalArgumentException if the snapshot request is formatted incorrectly
*/ */
MasterProtos.SnapshotResponse takeSnapshotAsync(HBaseProtos.SnapshotDescription snapshot) void takeSnapshotAsync(SnapshotDescription snapshot)
throws IOException, SnapshotCreationException; throws IOException, SnapshotCreationException;
/** /**
@ -1297,7 +1293,7 @@ public interface Admin extends Abortable, Closeable {
* @throws org.apache.hadoop.hbase.snapshot.UnknownSnapshotException if the requested snapshot is * @throws org.apache.hadoop.hbase.snapshot.UnknownSnapshotException if the requested snapshot is
* unknown * unknown
*/ */
boolean isSnapshotFinished(final HBaseProtos.SnapshotDescription snapshot) boolean isSnapshotFinished(final SnapshotDescription snapshot)
throws IOException, HBaseSnapshotException, UnknownSnapshotException; throws IOException, HBaseSnapshotException, UnknownSnapshotException;
/** /**
@ -1470,7 +1466,7 @@ public interface Admin extends Abortable, Closeable {
* @return a list of snapshot descriptors for completed snapshots * @return a list of snapshot descriptors for completed snapshots
* @throws IOException if a network error occurs * @throws IOException if a network error occurs
*/ */
List<HBaseProtos.SnapshotDescription> listSnapshots() throws IOException; List<SnapshotDescription> listSnapshots() throws IOException;
/** /**
* List all the completed snapshots matching the given regular expression. * List all the completed snapshots matching the given regular expression.
@ -1479,7 +1475,7 @@ public interface Admin extends Abortable, Closeable {
* @return - returns a List of SnapshotDescription * @return - returns a List of SnapshotDescription
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
List<HBaseProtos.SnapshotDescription> listSnapshots(String regex) throws IOException; List<SnapshotDescription> listSnapshots(String regex) throws IOException;
/** /**
* List all the completed snapshots matching the given pattern. * List all the completed snapshots matching the given pattern.
@ -1488,7 +1484,7 @@ public interface Admin extends Abortable, Closeable {
* @return - returns a List of SnapshotDescription * @return - returns a List of SnapshotDescription
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
List<HBaseProtos.SnapshotDescription> listSnapshots(Pattern pattern) throws IOException; List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException;
/** /**
* List all the completed snapshots matching the given table name regular expression and snapshot * List all the completed snapshots matching the given table name regular expression and snapshot
@ -1498,7 +1494,7 @@ public interface Admin extends Abortable, Closeable {
* @return - returns a List of completed SnapshotDescription * @return - returns a List of completed SnapshotDescription
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
List<HBaseProtos.SnapshotDescription> listTableSnapshots(String tableNameRegex, List<SnapshotDescription> listTableSnapshots(String tableNameRegex,
String snapshotNameRegex) throws IOException; String snapshotNameRegex) throws IOException;
/** /**
@ -1509,7 +1505,7 @@ public interface Admin extends Abortable, Closeable {
* @return - returns a List of completed SnapshotDescription * @return - returns a List of completed SnapshotDescription
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
List<HBaseProtos.SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern,
Pattern snapshotNamePattern) throws IOException; Pattern snapshotNamePattern) throws IOException;
/** /**
@ -1651,7 +1647,7 @@ public interface Admin extends Abortable, Closeable {
* Compact a table. Asynchronous operation. * Compact a table. Asynchronous operation.
* *
* @param tableName table to compact * @param tableName table to compact
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType} * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
* @throws IOException * @throws IOException
* @throws InterruptedException * @throws InterruptedException
*/ */
@ -1663,7 +1659,7 @@ public interface Admin extends Abortable, Closeable {
* *
* @param tableName table to compact * @param tableName table to compact
* @param columnFamily column family within a table * @param columnFamily column family within a table
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType} * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
* @throws IOException if not a mob column family or if a remote or network exception occurs * @throws IOException if not a mob column family or if a remote or network exception occurs
* @throws InterruptedException * @throws InterruptedException
*/ */
@ -1674,7 +1670,7 @@ public interface Admin extends Abortable, Closeable {
* Major compact a table. Asynchronous operation. * Major compact a table. Asynchronous operation.
* *
* @param tableName table to compact * @param tableName table to compact
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType} * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
* @throws IOException * @throws IOException
* @throws InterruptedException * @throws InterruptedException
*/ */
@ -1686,7 +1682,7 @@ public interface Admin extends Abortable, Closeable {
* *
* @param tableName table to compact * @param tableName table to compact
* @param columnFamily column family within a table * @param columnFamily column family within a table
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType} * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
* @throws IOException if not a mob column family or if a remote or network exception occurs * @throws IOException if not a mob column family or if a remote or network exception occurs
* @throws InterruptedException * @throws InterruptedException
*/ */
@ -1697,11 +1693,11 @@ public interface Admin extends Abortable, Closeable {
* Get the current compaction state of a table. It could be in a compaction, or none. * Get the current compaction state of a table. It could be in a compaction, or none.
* *
* @param tableName table to examine * @param tableName table to examine
* @param compactType {@link org.apache.hadoop.hbase.client.Admin.CompactType} * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
* @return the current compaction state * @return the current compaction state
* @throws IOException if a remote or network exception occurs * @throws IOException if a remote or network exception occurs
*/ */
AdminProtos.GetRegionInfoResponse.CompactionState getCompactionState(final TableName tableName, CompactionState getCompactionState(final TableName tableName,
CompactType compactType) throws IOException; CompactType compactType) throws IOException;
/** /**
@ -1741,27 +1737,4 @@ public interface Admin extends Abortable, Closeable {
* and rollback the switch state to be original state before you change switch * and rollback the switch state to be original state before you change switch
* */ * */
void releaseSplitOrMergeLockAndRollback() throws IOException; void releaseSplitOrMergeLockAndRollback() throws IOException;
/**
* Currently, there are only two compact types:
* {@code NORMAL} means do store files compaction;
* {@code MOB} means do mob files compaction.
* */
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum CompactType {
NORMAL (0),
MOB (1);
CompactType(int value) {}
}
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum MasterSwitchType {
SPLIT,
MERGE
}
} }

View File

@ -0,0 +1,35 @@
/**
* 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.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Currently, there are only two compact types:
* {@code NORMAL} means do store files compaction;
* {@code MOB} means do mob files compaction.
* */
@InterfaceAudience.Public
@InterfaceStability.Unstable
public enum CompactType {
NORMAL (0),
MOB (1);
CompactType(int value) {}
}

View File

@ -0,0 +1,29 @@
/**
* 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.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* POJO representing the compaction state
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum CompactionState {
NONE, MINOR, MAJOR, MAJOR_AND_MINOR;
}

View File

@ -73,6 +73,7 @@ import org.apache.hadoop.hbase.ipc.RegionServerCoprocessorRpcChannel;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.RequestConverter; import org.apache.hadoop.hbase.protobuf.RequestConverter;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CloseRegionResponse;
@ -80,7 +81,6 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.CompactRegionReque
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.FlushRegionRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterRequest;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.RollWALWriterResponse;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.StopServerRequest;
@ -89,7 +89,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.ProcedureDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.RegionSpecifier.RegionSpecifierType;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.TableSchema;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.AbortProcedureRequest;
@ -2265,7 +2264,10 @@ public class HBaseAdmin implements Admin {
PayloadCarryingRpcController controller = rpcControllerFactory.newController(); PayloadCarryingRpcController controller = rpcControllerFactory.newController();
// TODO: this does not do retries, it should. Set priority and timeout in controller // TODO: this does not do retries, it should. Set priority and timeout in controller
GetRegionInfoResponse response = admin.getRegionInfo(controller, request); GetRegionInfoResponse response = admin.getRegionInfo(controller, request);
return response.getCompactionState(); if (response.getCompactionState() != null) {
return ProtobufUtil.createCompactionState(response.getCompactionState());
}
return null;
} catch (ServiceException se) { } catch (ServiceException se) {
throw ProtobufUtil.getRemoteException(se); throw ProtobufUtil.getRemoteException(se);
} }
@ -2275,33 +2277,30 @@ public class HBaseAdmin implements Admin {
public void snapshot(final String snapshotName, public void snapshot(final String snapshotName,
final TableName tableName) throws IOException, final TableName tableName) throws IOException,
SnapshotCreationException, IllegalArgumentException { SnapshotCreationException, IllegalArgumentException {
snapshot(snapshotName, tableName, SnapshotDescription.Type.FLUSH); snapshot(snapshotName, tableName, SnapshotType.FLUSH);
} }
@Override @Override
public void snapshot(final byte[] snapshotName, final TableName tableName) public void snapshot(final byte[] snapshotName, final TableName tableName)
throws IOException, SnapshotCreationException, IllegalArgumentException { throws IOException, SnapshotCreationException, IllegalArgumentException {
snapshot(Bytes.toString(snapshotName), tableName, SnapshotDescription.Type.FLUSH); snapshot(Bytes.toString(snapshotName), tableName, SnapshotType.FLUSH);
} }
@Override @Override
public void snapshot(final String snapshotName, final TableName tableName, public void snapshot(final String snapshotName, final TableName tableName,
SnapshotDescription.Type type) SnapshotType type)
throws IOException, SnapshotCreationException, IllegalArgumentException { throws IOException, SnapshotCreationException, IllegalArgumentException {
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); snapshot(new SnapshotDescription(snapshotName, tableName.getNameAsString(), type));
builder.setTable(tableName.getNameAsString());
builder.setName(snapshotName);
builder.setType(type);
snapshot(builder.build());
} }
@Override @Override
public void snapshot(SnapshotDescription snapshot) throws IOException, SnapshotCreationException, public void snapshot(SnapshotDescription snapshotDesc)
IllegalArgumentException { throws IOException, SnapshotCreationException, IllegalArgumentException {
// actually take the snapshot // actually take the snapshot
SnapshotResponse response = takeSnapshotAsync(snapshot); HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot) SnapshotResponse response = asyncSnapshot(snapshot);
.build(); final IsSnapshotDoneRequest request =
IsSnapshotDoneRequest.newBuilder().setSnapshot(snapshot).build();
IsSnapshotDoneResponse done = null; IsSnapshotDoneResponse done = null;
long start = EnvironmentEdgeManager.currentTime(); long start = EnvironmentEdgeManager.currentTime();
long max = response.getExpectedTimeout(); long max = response.getExpectedTimeout();
@ -2339,8 +2338,37 @@ public class HBaseAdmin implements Admin {
} }
@Override @Override
public SnapshotResponse takeSnapshotAsync(SnapshotDescription snapshot) throws IOException, public void takeSnapshotAsync(SnapshotDescription snapshotDesc) throws IOException,
SnapshotCreationException { SnapshotCreationException {
HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
asyncSnapshot(snapshot);
}
private HBaseProtos.SnapshotDescription
createHBaseProtosSnapshotDesc(SnapshotDescription snapshotDesc) {
HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
if (snapshotDesc.getTable() != null) {
builder.setTable(snapshotDesc.getTable());
}
if (snapshotDesc.getName() != null) {
builder.setName(snapshotDesc.getName());
}
if (snapshotDesc.getOwner() != null) {
builder.setOwner(snapshotDesc.getOwner());
}
if (snapshotDesc.getCreationTime() != -1) {
builder.setCreationTime(snapshotDesc.getCreationTime());
}
if (snapshotDesc.getVersion() != -1) {
builder.setVersion(snapshotDesc.getVersion());
}
builder.setType(ProtobufUtil.createProtosSnapShotDescType(snapshotDesc.getType()));
HBaseProtos.SnapshotDescription snapshot = builder.build();
return snapshot;
}
private SnapshotResponse asyncSnapshot(HBaseProtos.SnapshotDescription snapshot)
throws IOException {
ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot); ClientSnapshotDescriptionUtils.assertSnapshotRequestIsValid(snapshot);
final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot) final SnapshotRequest request = SnapshotRequest.newBuilder().setSnapshot(snapshot)
.build(); .build();
@ -2356,9 +2384,9 @@ public class HBaseAdmin implements Admin {
} }
@Override @Override
public boolean isSnapshotFinished(final SnapshotDescription snapshot) public boolean isSnapshotFinished(final SnapshotDescription snapshotDesc)
throws IOException, HBaseSnapshotException, UnknownSnapshotException { throws IOException, HBaseSnapshotException, UnknownSnapshotException {
final HBaseProtos.SnapshotDescription snapshot = createHBaseProtosSnapshotDesc(snapshotDesc);
return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) { return executeCallable(new MasterCallable<IsSnapshotDoneResponse>(getConnection()) {
@Override @Override
public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException { public IsSnapshotDoneResponse call(int callTimeout) throws ServiceException {
@ -2643,7 +2671,7 @@ public class HBaseAdmin implements Admin {
private Future<Void> internalRestoreSnapshotAsync( private Future<Void> internalRestoreSnapshotAsync(
final String snapshotName, final String snapshotName,
final TableName tableName) throws IOException, RestoreSnapshotException { final TableName tableName) throws IOException, RestoreSnapshotException {
final SnapshotDescription snapshot = SnapshotDescription.newBuilder() final HBaseProtos.SnapshotDescription snapshot = HBaseProtos.SnapshotDescription.newBuilder()
.setName(snapshotName).setTable(tableName.getNameAsString()).build(); .setName(snapshotName).setTable(tableName.getNameAsString()).build();
// actually restore the snapshot // actually restore the snapshot
@ -2671,7 +2699,7 @@ public class HBaseAdmin implements Admin {
private static class RestoreSnapshotFuture extends TableFuture<Void> { private static class RestoreSnapshotFuture extends TableFuture<Void> {
public RestoreSnapshotFuture( public RestoreSnapshotFuture(
final HBaseAdmin admin, final HBaseAdmin admin,
final SnapshotDescription snapshot, final HBaseProtos.SnapshotDescription snapshot,
final TableName tableName, final TableName tableName,
final RestoreSnapshotResponse response) { final RestoreSnapshotResponse response) {
super(admin, tableName, super(admin, tableName,
@ -2702,8 +2730,16 @@ public class HBaseAdmin implements Admin {
public List<SnapshotDescription> call(int callTimeout) throws ServiceException { public List<SnapshotDescription> call(int callTimeout) throws ServiceException {
PayloadCarryingRpcController controller = rpcControllerFactory.newController(); PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setCallTimeout(callTimeout); controller.setCallTimeout(callTimeout);
return master.getCompletedSnapshots(controller, List<HBaseProtos.SnapshotDescription> snapshotsList = master
GetCompletedSnapshotsRequest.newBuilder().build()).getSnapshotsList(); .getCompletedSnapshots(controller, GetCompletedSnapshotsRequest.newBuilder().build())
.getSnapshotsList();
List<SnapshotDescription> result = new ArrayList<SnapshotDescription>(snapshotsList.size());
for (HBaseProtos.SnapshotDescription snapshot : snapshotsList) {
result.add(new SnapshotDescription(snapshot.getName(), snapshot.getTable(),
ProtobufUtil.createSnapshotType(snapshot.getType()), snapshot.getOwner(),
snapshot.getCreationTime(), snapshot.getVersion()));
}
return result;
} }
}); });
} }
@ -2765,7 +2801,9 @@ public class HBaseAdmin implements Admin {
controller.setCallTimeout(callTimeout); controller.setCallTimeout(callTimeout);
master.deleteSnapshot(controller, master.deleteSnapshot(controller,
DeleteSnapshotRequest.newBuilder(). DeleteSnapshotRequest.newBuilder().
setSnapshot(SnapshotDescription.newBuilder().setName(snapshotName).build()).build() setSnapshot(
HBaseProtos.SnapshotDescription.newBuilder().setName(snapshotName).build())
.build()
); );
return null; return null;
} }
@ -2798,7 +2836,7 @@ public class HBaseAdmin implements Admin {
PayloadCarryingRpcController controller = rpcControllerFactory.newController(); PayloadCarryingRpcController controller = rpcControllerFactory.newController();
controller.setCallTimeout(callTimeout); controller.setCallTimeout(callTimeout);
this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder() this.master.deleteSnapshot(controller, DeleteSnapshotRequest.newBuilder()
.setSnapshot(snapshot).build()); .setSnapshot(createHBaseProtosSnapshotDesc(snapshot)).build());
return null; return null;
} }
}); });
@ -3002,7 +3040,8 @@ public class HBaseAdmin implements Admin {
@Override @Override
public CompactionState getCompactionState(TableName tableName, public CompactionState getCompactionState(TableName tableName,
CompactType compactType) throws IOException { CompactType compactType) throws IOException {
CompactionState state = CompactionState.NONE; AdminProtos.GetRegionInfoResponse.CompactionState state =
AdminProtos.GetRegionInfoResponse.CompactionState.NONE;
checkTableExists(tableName); checkTableExists(tableName);
PayloadCarryingRpcController controller = rpcControllerFactory.newController(); PayloadCarryingRpcController controller = rpcControllerFactory.newController();
switch (compactType) { switch (compactType) {
@ -3044,16 +3083,16 @@ public class HBaseAdmin implements Admin {
case MAJOR_AND_MINOR: case MAJOR_AND_MINOR:
return CompactionState.MAJOR_AND_MINOR; return CompactionState.MAJOR_AND_MINOR;
case MAJOR: case MAJOR:
if (state == CompactionState.MINOR) { if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MINOR) {
return CompactionState.MAJOR_AND_MINOR; return CompactionState.MAJOR_AND_MINOR;
} }
state = CompactionState.MAJOR; state = AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR;
break; break;
case MINOR: case MINOR:
if (state == CompactionState.MAJOR) { if (state == AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR) {
return CompactionState.MAJOR_AND_MINOR; return CompactionState.MAJOR_AND_MINOR;
} }
state = CompactionState.MINOR; state = AdminProtos.GetRegionInfoResponse.CompactionState.MINOR;
break; break;
case NONE: case NONE:
default: // nothing, continue default: // nothing, continue
@ -3084,7 +3123,10 @@ public class HBaseAdmin implements Admin {
} }
break; break;
} }
return state; if(state != null) {
return ProtobufUtil.createCompactionState(state);
}
return null;
} }
/** /**

View File

@ -0,0 +1,29 @@
/**
* 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.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Represents the master switch type
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum MasterSwitchType {
SPLIT,
MERGE
}

View File

@ -0,0 +1,84 @@
/**
* 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.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* The POJO equivalent of HBaseProtos.SnapshotDescription
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public class SnapshotDescription {
private String name;
private String table;
private SnapshotType snapShotType = SnapshotType.DISABLED;
private String owner;
private long creationTime = -1L;
private int version = -1;
public SnapshotDescription(String name) {
this(name, null);
}
public SnapshotDescription(String name, String table) {
this(name, table, SnapshotType.DISABLED, null);
}
public SnapshotDescription(String name, String table, SnapshotType type) {
this(name, table, type, null);
}
public SnapshotDescription(String name, String table, SnapshotType type, String owner) {
this(name, table, type, owner, -1, -1);
}
public SnapshotDescription(String name, String table, SnapshotType type, String owner,
long creationTime, int version) {
this.name = name;
this.table = table;
this.snapShotType = type;
this.owner = owner;
this.creationTime = creationTime;
this.version = version;
}
public String getName() {
return this.name;
}
public String getTable() {
return this.table;
}
public SnapshotType getType() {
return this.snapShotType;
}
public String getOwner() {
return this.owner;
}
public long getCreationTime() {
return this.creationTime;
}
public int getVersion() {
return this.version;
}
}

View File

@ -0,0 +1,29 @@
/**
* 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.client;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* POJO representing the snapshot type
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public enum SnapshotType {
DISABLED, FLUSH, SKIPFLUSH;
}

View File

@ -58,6 +58,7 @@ import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Consistency; import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
@ -67,6 +68,8 @@ import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan; import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.client.SnapshotType;
import org.apache.hadoop.hbase.client.metrics.ScanMetrics; import org.apache.hadoop.hbase.client.metrics.ScanMetrics;
import org.apache.hadoop.hbase.client.security.SecurityCapability; import org.apache.hadoop.hbase.client.security.SecurityCapability;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -3397,4 +3400,76 @@ public final class ProtobufUtil {
} }
return htd; return htd;
} }
/**
* Creates {@link CompactionState} from
* {@link org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState}
* state
* @param state the protobuf CompactionState
* @return CompactionState
*/
public static CompactionState createCompactionState(GetRegionInfoResponse.CompactionState state) {
return CompactionState.valueOf(state.toString());
}
/**
* Creates {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type}
* from {@link SnapshotType}
* @param type the SnapshotDescription type
* @return the protobuf SnapshotDescription type
*/
public static HBaseProtos.SnapshotDescription.Type
createProtosSnapShotDescType(SnapshotType type) {
return HBaseProtos.SnapshotDescription.Type.valueOf(type.name());
}
/**
* Creates {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription.Type}
* from the type of SnapshotDescription string
* @param snapshotDesc string representing the snapshot description type
* @return the protobuf SnapshotDescription type
*/
public static HBaseProtos.SnapshotDescription.Type
createProtosSnapShotDescType(String snapshotDesc) {
return HBaseProtos.SnapshotDescription.Type.valueOf(snapshotDesc.toUpperCase());
}
/**
* Creates {@link SnapshotType} from the type of
* {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription}
* @param type the snapshot description type
* @return the protobuf SnapshotDescription type
*/
public static SnapshotType createSnapshotType(HBaseProtos.SnapshotDescription.Type type) {
return SnapshotType.valueOf(type.toString());
}
/**
* Convert from {@link SnapshotDescription} to
* {@link org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription}
* @param snapshotDesc the POJO SnapshotDescription
* @return the protobuf SnapshotDescription
*/
public static HBaseProtos.SnapshotDescription
createHBaseProtosSnapshotDesc(SnapshotDescription snapshotDesc) {
HBaseProtos.SnapshotDescription.Builder builder = HBaseProtos.SnapshotDescription.newBuilder();
if (snapshotDesc.getTable() != null) {
builder.setTable(snapshotDesc.getTable());
}
if (snapshotDesc.getName() != null) {
builder.setName(snapshotDesc.getName());
}
if (snapshotDesc.getOwner() != null) {
builder.setOwner(snapshotDesc.getOwner());
}
if (snapshotDesc.getCreationTime() != -1L) {
builder.setCreationTime(snapshotDesc.getCreationTime());
}
if (snapshotDesc.getVersion() != -1) {
builder.setVersion(snapshotDesc.getVersion());
}
builder.setType(ProtobufUtil.createProtosSnapShotDescType(snapshotDesc.getType()));
HBaseProtos.SnapshotDescription snapshot = builder.build();
return snapshot;
}
} }

View File

@ -31,12 +31,12 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Action; import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec; import org.apache.hadoop.hbase.client.RegionCoprocessorServiceExec;
@ -1701,11 +1701,11 @@ public final class RequestConverter {
/** /**
* Creates a protocol buffer IsSplitOrMergeEnabledRequest * Creates a protocol buffer IsSplitOrMergeEnabledRequest
* *
* @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType} * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
* @return a IsSplitOrMergeEnabledRequest * @return a IsSplitOrMergeEnabledRequest
*/ */
public static IsSplitOrMergeEnabledRequest buildIsSplitOrMergeEnabledRequest( public static IsSplitOrMergeEnabledRequest buildIsSplitOrMergeEnabledRequest(
Admin.MasterSwitchType switchType) { MasterSwitchType switchType) {
IsSplitOrMergeEnabledRequest.Builder builder = IsSplitOrMergeEnabledRequest.newBuilder(); IsSplitOrMergeEnabledRequest.Builder builder = IsSplitOrMergeEnabledRequest.newBuilder();
builder.setSwitchType(convert(switchType)); builder.setSwitchType(convert(switchType));
return builder.build(); return builder.build();
@ -1723,23 +1723,23 @@ public final class RequestConverter {
* *
* @param enabled switch is enabled or not * @param enabled switch is enabled or not
* @param synchronous set switch sync? * @param synchronous set switch sync?
* @param switchTypes see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType}, it is * @param switchTypes see {@link org.apache.hadoop.hbase.client.MasterSwitchType}, it is
* a list. * a list.
* @return a SetSplitOrMergeEnabledRequest * @return a SetSplitOrMergeEnabledRequest
*/ */
public static SetSplitOrMergeEnabledRequest buildSetSplitOrMergeEnabledRequest(boolean enabled, public static SetSplitOrMergeEnabledRequest buildSetSplitOrMergeEnabledRequest(boolean enabled,
boolean synchronous, boolean skipLock, Admin.MasterSwitchType... switchTypes) { boolean synchronous, boolean skipLock, MasterSwitchType... switchTypes) {
SetSplitOrMergeEnabledRequest.Builder builder = SetSplitOrMergeEnabledRequest.newBuilder(); SetSplitOrMergeEnabledRequest.Builder builder = SetSplitOrMergeEnabledRequest.newBuilder();
builder.setEnabled(enabled); builder.setEnabled(enabled);
builder.setSynchronous(synchronous); builder.setSynchronous(synchronous);
builder.setSkipLock(skipLock); builder.setSkipLock(skipLock);
for (Admin.MasterSwitchType switchType : switchTypes) { for (MasterSwitchType switchType : switchTypes) {
builder.addSwitchTypes(convert(switchType)); builder.addSwitchTypes(convert(switchType));
} }
return builder.build(); return builder.build();
} }
private static MasterProtos.MasterSwitchType convert(Admin.MasterSwitchType switchType) { private static MasterProtos.MasterSwitchType convert(MasterSwitchType switchType) {
switch (switchType) { switch (switchType) {
case SPLIT: case SPLIT:
return MasterProtos.MasterSwitchType.SPLIT; return MasterProtos.MasterSwitchType.SPLIT;

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController; import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SnapshotRequest;
@ -141,18 +140,17 @@ public class TestSnapshotFromAdmin {
Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory); Mockito.when(mockConnection.getRpcRetryingCallerFactory()).thenReturn(callerFactory);
Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory); Mockito.when(mockConnection.getRpcControllerFactory()).thenReturn(controllerFactory);
Admin admin = new HBaseAdmin(mockConnection); Admin admin = new HBaseAdmin(mockConnection);
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder();
// check that invalid snapshot names fail // check that invalid snapshot names fail
failSnapshotStart(admin, builder.setName(HConstants.SNAPSHOT_DIR_NAME).build()); failSnapshotStart(admin, new SnapshotDescription(HConstants.SNAPSHOT_DIR_NAME));
failSnapshotStart(admin, builder.setName("-snapshot").build()); failSnapshotStart(admin, new SnapshotDescription("-snapshot"));
failSnapshotStart(admin, builder.setName("snapshot fails").build()); failSnapshotStart(admin, new SnapshotDescription("snapshot fails"));
failSnapshotStart(admin, builder.setName("snap$hot").build()); failSnapshotStart(admin, new SnapshotDescription("snap$hot"));
failSnapshotStart(admin, builder.setName("snap:hot").build()); failSnapshotStart(admin, new SnapshotDescription("snap:hot"));
// check the table name also get verified // check the table name also get verified
failSnapshotStart(admin, builder.setName("snapshot").setTable(".table").build()); failSnapshotStart(admin, new SnapshotDescription("snapshot", ".table"));
failSnapshotStart(admin, builder.setName("snapshot").setTable("-table").build()); failSnapshotStart(admin, new SnapshotDescription("snapshot", "-table"));
failSnapshotStart(admin, builder.setName("snapshot").setTable("table fails").build()); failSnapshotStart(admin, new SnapshotDescription("snapshot", "table fails"));
failSnapshotStart(admin, builder.setName("snapshot").setTable("tab%le").build()); failSnapshotStart(admin, new SnapshotDescription("snapshot", "tab%le"));
// mock the master connection // mock the master connection
MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class); MasterKeepAliveConnection master = Mockito.mock(MasterKeepAliveConnection.class);
@ -167,10 +165,11 @@ public class TestSnapshotFromAdmin {
Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse); Mockito.any(IsSnapshotDoneRequest.class))).thenReturn(doneResponse);
// make sure that we can use valid names // make sure that we can use valid names
admin.snapshot(builder.setName("snapshot").setTable("table").build()); admin.snapshot(new SnapshotDescription("snapshot", "table"));
} }
private void failSnapshotStart(Admin admin, SnapshotDescription snapshot) throws IOException { private void failSnapshotStart(Admin admin, SnapshotDescription snapshot)
throws IOException {
try { try {
admin.snapshot(snapshot); admin.snapshot(snapshot);
fail("Snapshot should not have succeed with name:" + snapshot.getName()); fail("Snapshot should not have succeed with name:" + snapshot.getName());

View File

@ -22,6 +22,7 @@ import org.apache.commons.lang.math.RandomUtils;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.CompactType;
/** /**
* Action that queues a table compaction. * Action that queues a table compaction.
@ -56,9 +57,9 @@ public class CompactMobAction extends Action {
LOG.info("Performing action: Compact mob of table " + tableName + ", major=" + major); LOG.info("Performing action: Compact mob of table " + tableName + ", major=" + major);
try { try {
if (major) { if (major) {
admin.majorCompact(tableName, Admin.CompactType.MOB); admin.majorCompact(tableName, CompactType.MOB);
} else { } else {
admin.compact(tableName, Admin.CompactType.MOB); admin.compact(tableName, CompactType.MOB);
} }
} catch (Exception ex) { } catch (Exception ex) {
LOG.warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage()); LOG.warn("Mob Compaction failed, might be caused by other chaos: " + ex.getMessage());

View File

@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.ProcedureInfo; import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.constraint.ConstraintException; import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@ -827,13 +827,13 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService
@Override @Override
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
return false; return false;
} }
@Override @Override
public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
} }
@Override @Override

View File

@ -39,13 +39,13 @@ org.apache.hadoop.hbase.ServerLoad;
org.apache.hadoop.hbase.ServerName; org.apache.hadoop.hbase.ServerName;
org.apache.hadoop.hbase.TableName; org.apache.hadoop.hbase.TableName;
org.apache.hadoop.hbase.client.Admin; org.apache.hadoop.hbase.client.Admin;
org.apache.hadoop.hbase.client.SnapshotDescription;
org.apache.hadoop.hbase.master.AssignmentManager; org.apache.hadoop.hbase.master.AssignmentManager;
org.apache.hadoop.hbase.master.DeadServer; org.apache.hadoop.hbase.master.DeadServer;
org.apache.hadoop.hbase.master.HMaster; org.apache.hadoop.hbase.master.HMaster;
org.apache.hadoop.hbase.master.RegionState; org.apache.hadoop.hbase.master.RegionState;
org.apache.hadoop.hbase.master.ServerManager; org.apache.hadoop.hbase.master.ServerManager;
org.apache.hadoop.hbase.protobuf.ProtobufUtil; org.apache.hadoop.hbase.protobuf.ProtobufUtil;
org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
org.apache.hadoop.hbase.quotas.QuotaUtil; org.apache.hadoop.hbase.quotas.QuotaUtil;
org.apache.hadoop.hbase.security.access.AccessControlLists; org.apache.hadoop.hbase.security.access.AccessControlLists;
org.apache.hadoop.hbase.security.visibility.VisibilityConstants; org.apache.hadoop.hbase.security.visibility.VisibilityConstants;

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -450,7 +450,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
@Override @Override
public boolean preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue, boolean newValue,
Admin.MasterSwitchType switchType) MasterSwitchType switchType)
throws IOException { throws IOException {
return false; return false;
} }
@ -458,7 +458,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver
@Override @Override
public void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(ObserverContext<MasterCoprocessorEnvironment> ctx,
boolean newValue, boolean newValue,
Admin.MasterSwitchType switchType) MasterSwitchType switchType)
throws IOException { throws IOException {
} }

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -442,13 +442,13 @@ public class BaseMasterObserver implements MasterObserver {
@Override @Override
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
return false; return false;
} }
@Override @Override
public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
} }
@Override @Override

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.master.RegionPlan; import org.apache.hadoop.hbase.master.RegionPlan;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv; import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@ -807,7 +807,7 @@ public interface MasterObserver extends Coprocessor {
* @param switchType type of switch * @param switchType type of switch
*/ */
boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException; final boolean newValue, final MasterSwitchType switchType) throws IOException;
/** /**
* Called after setting split / merge switch * Called after setting split / merge switch
@ -816,7 +816,7 @@ public interface MasterObserver extends Coprocessor {
* @param switchType type of switch * @param switchType type of switch
*/ */
void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException; final boolean newValue, final MasterSwitchType switchType) throws IOException;
/** /**
* Called prior to modifying the flag used to enable/disable region balancing. * Called prior to modifying the flag used to enable/disable region balancing.

View File

@ -62,7 +62,7 @@ import org.apache.hadoop.hbase.RegionStateListener;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
@ -2364,7 +2364,7 @@ public class AssignmentManager {
} }
if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled( if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
Admin.MasterSwitchType.SPLIT)) { MasterSwitchType.SPLIT)) {
return "split switch is off!"; return "split switch is off!";
} }
@ -2527,7 +2527,7 @@ public class AssignmentManager {
} }
if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled( if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
Admin.MasterSwitchType.MERGE)) { MasterSwitchType.MERGE)) {
return "merge switch is off!"; return "merge switch is off!";
} }
// Just return in case of retrying // Just return in case of retrying

View File

@ -83,7 +83,7 @@ import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
@ -2821,10 +2821,10 @@ public class HMaster extends HRegionServer implements MasterServices {
/** /**
* Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized, * Queries the state of the {@link SplitOrMergeTracker}. If it is not initialized,
* false is returned. If switchType is illegal, false will return. * false is returned. If switchType is illegal, false will return.
* @param switchType see {@link org.apache.hadoop.hbase.client.Admin.MasterSwitchType} * @param switchType see {@link org.apache.hadoop.hbase.client.MasterSwitchType}
* @return The state of the switch * @return The state of the switch
*/ */
public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) { public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
if (null == splitOrMergeTracker) { if (null == splitOrMergeTracker) {
return false; return false;
} }

View File

@ -38,7 +38,7 @@ import org.apache.hadoop.hbase.ProcedureInfo;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.CoprocessorService; import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.MasterCoprocessorEnvironment;
@ -779,7 +779,7 @@ public class MasterCoprocessorHost
} }
public boolean preSetSplitOrMergeEnabled(final boolean newValue, public boolean preSetSplitOrMergeEnabled(final boolean newValue,
final Admin.MasterSwitchType switchType) throws IOException { final MasterSwitchType switchType) throws IOException {
return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override @Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx) public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)
@ -790,7 +790,7 @@ public class MasterCoprocessorHost
} }
public void postSetSplitOrMergeEnabled(final boolean newValue, public void postSetSplitOrMergeEnabled(final boolean newValue,
final Admin.MasterSwitchType switchType) throws IOException { final MasterSwitchType switchType) throws IOException {
execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() {
@Override @Override
public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx) public void call(MasterObserver oserver, ObserverContext<MasterCoprocessorEnvironment> ctx)

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.UnknownRegionException; import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.TableState; import org.apache.hadoop.hbase.client.TableState;
import org.apache.hadoop.hbase.errorhandling.ForeignException; import org.apache.hadoop.hbase.errorhandling.ForeignException;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
@ -1500,8 +1500,8 @@ public class MasterRpcServices extends RSRpcServices
if (!master.getSplitOrMergeTracker().lock(skipLock)) { if (!master.getSplitOrMergeTracker().lock(skipLock)) {
throw new DoNotRetryIOException("can't set splitOrMerge switch due to lock"); throw new DoNotRetryIOException("can't set splitOrMerge switch due to lock");
} }
for (MasterSwitchType masterSwitchType : request.getSwitchTypesList()) { for (MasterProtos.MasterSwitchType masterSwitchType : request.getSwitchTypesList()) {
Admin.MasterSwitchType switchType = convert(masterSwitchType); MasterSwitchType switchType = convert(masterSwitchType);
boolean oldValue = master.isSplitOrMergeEnabled(switchType); boolean oldValue = master.isSplitOrMergeEnabled(switchType);
response.addPrevValue(oldValue); response.addPrevValue(oldValue);
boolean bypass = false; boolean bypass = false;
@ -1619,12 +1619,12 @@ public class MasterRpcServices extends RSRpcServices
return response.build(); return response.build();
} }
private Admin.MasterSwitchType convert(MasterSwitchType switchType) { private MasterSwitchType convert(MasterProtos.MasterSwitchType switchType) {
switch (switchType) { switch (switchType) {
case SPLIT: case SPLIT:
return Admin.MasterSwitchType.SPLIT; return MasterSwitchType.SPLIT;
case MERGE: case MERGE:
return Admin.MasterSwitchType.MERGE; return MasterSwitchType.MERGE;
default: default:
break; break;
} }

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.RegionLoad;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin.MasterSwitchType; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.master.MasterRpcServices; import org.apache.hadoop.hbase.master.MasterRpcServices;
import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType; import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;

View File

@ -60,12 +60,12 @@ import org.apache.hadoop.hbase.Tag;
import org.apache.hadoop.hbase.TagRewriteCell; import org.apache.hadoop.hbase.TagRewriteCell;
import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Query; import org.apache.hadoop.hbase.client.Query;
@ -1262,14 +1262,14 @@ public class AccessController extends BaseMasterAndRegionObserver
@Override @Override
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
requirePermission("setSplitOrMergeEnabled", Action.ADMIN); requirePermission("setSplitOrMergeEnabled", Action.ADMIN);
return false; return false;
} }
@Override @Override
public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
} }
@Override @Override

View File

@ -51,11 +51,11 @@ import org.apache.hadoop.hbase.TagRewriteCell;
import org.apache.hadoop.hbase.TagType; import org.apache.hadoop.hbase.TagType;
import org.apache.hadoop.hbase.TagUtil; import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -310,13 +310,13 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
@Override @Override
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
return false; return false;
} }
@Override @Override
public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
} }
@Override @Override

View File

@ -23,6 +23,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.AbstractHBaseTool;
import java.util.Arrays; import java.util.Arrays;
@ -66,10 +68,10 @@ public class CreateSnapshot extends AbstractHBaseTool {
admin = connection.getAdmin(); admin = connection.getAdmin();
HBaseProtos.SnapshotDescription.Type type = HBaseProtos.SnapshotDescription.Type.FLUSH; HBaseProtos.SnapshotDescription.Type type = HBaseProtos.SnapshotDescription.Type.FLUSH;
if (snapshotType != null) { if (snapshotType != null) {
type = HBaseProtos.SnapshotDescription.Type.valueOf(snapshotName.toUpperCase()); type = ProtobufUtil.createProtosSnapShotDescType(snapshotName);
} }
admin.snapshot(new SnapshotDescription(snapshotName, tableName,
admin.snapshot(snapshotName, TableName.valueOf(tableName), type); ProtobufUtil.createSnapshotType(type)));
} catch (Exception e) { } catch (Exception e) {
return -1; return -1;
} finally { } finally {

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.classification.InterfaceStability;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.conf.Configured; import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.HRegionInfo; import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -47,7 +48,8 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.io.HFileLink; import org.apache.hadoop.hbase.io.HFileLink;
import org.apache.hadoop.hbase.io.WALLink; import org.apache.hadoop.hbase.io.WALLink;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.FSUtils;
@ -126,14 +128,15 @@ public final class SnapshotInfo extends Configured implements Tool {
private AtomicLong hfilesMobSize = new AtomicLong(); private AtomicLong hfilesMobSize = new AtomicLong();
private AtomicLong logSize = new AtomicLong(); private AtomicLong logSize = new AtomicLong();
private final SnapshotDescription snapshot; private final HBaseProtos.SnapshotDescription snapshot;
private final TableName snapshotTable; private final TableName snapshotTable;
private final Configuration conf; private final Configuration conf;
private final FileSystem fs; private final FileSystem fs;
SnapshotStats(final Configuration conf, final FileSystem fs, final SnapshotDescription snapshot) SnapshotStats(final Configuration conf, final FileSystem fs,
final SnapshotDescription snapshot)
{ {
this.snapshot = snapshot; this.snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
this.snapshotTable = TableName.valueOf(snapshot.getTable()); this.snapshotTable = TableName.valueOf(snapshot.getTable());
this.conf = conf; this.conf = conf;
this.fs = fs; this.fs = fs;
@ -141,7 +144,9 @@ public final class SnapshotInfo extends Configured implements Tool {
/** @return the snapshot descriptor */ /** @return the snapshot descriptor */
public SnapshotDescription getSnapshotDescription() { public SnapshotDescription getSnapshotDescription() {
return this.snapshot; return new SnapshotDescription(this.snapshot.getName(), this.snapshot.getTable(),
ProtobufUtil.createSnapshotType(this.snapshot.getType()), this.snapshot.getOwner(),
this.snapshot.getCreationTime(), this.snapshot.getVersion());
} }
/** @return true if the snapshot is corrupted */ /** @return true if the snapshot is corrupted */
@ -371,7 +376,8 @@ public final class SnapshotInfo extends Configured implements Tool {
return false; return false;
} }
SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); HBaseProtos.SnapshotDescription snapshotDesc =
SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
snapshotManifest = SnapshotManifest.open(getConf(), fs, snapshotDir, snapshotDesc); snapshotManifest = SnapshotManifest.open(getConf(), fs, snapshotDir, snapshotDesc);
return true; return true;
} }
@ -380,7 +386,7 @@ public final class SnapshotInfo extends Configured implements Tool {
* Dump the {@link SnapshotDescription} * Dump the {@link SnapshotDescription}
*/ */
private void printInfo() { private void printInfo() {
SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription(); HBaseProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss"); SimpleDateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
System.out.println("Snapshot Info"); System.out.println("Snapshot Info");
System.out.println("----------------------------------------"); System.out.println("----------------------------------------");
@ -413,9 +419,12 @@ public final class SnapshotInfo extends Configured implements Tool {
} }
// Collect information about hfiles and logs in the snapshot // Collect information about hfiles and logs in the snapshot
final SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription(); final HBaseProtos.SnapshotDescription snapshotDesc = snapshotManifest.getSnapshotDescription();
final String table = snapshotDesc.getTable(); final String table = snapshotDesc.getTable();
final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, snapshotDesc); SnapshotDescription desc = new SnapshotDescription(snapshotDesc.getName(),
snapshotDesc.getTable(), ProtobufUtil.createSnapshotType(snapshotDesc.getType()),
snapshotDesc.getOwner(), snapshotDesc.getCreationTime(), snapshotDesc.getVersion());
final SnapshotStats stats = new SnapshotStats(this.getConf(), this.fs, desc);
SnapshotReferenceUtil.concurrentVisitReferencedFiles(getConf(), fs, snapshotManifest, SnapshotReferenceUtil.concurrentVisitReferencedFiles(getConf(), fs, snapshotManifest,
new SnapshotReferenceUtil.SnapshotVisitor() { new SnapshotReferenceUtil.SnapshotVisitor() {
@Override @Override
@ -492,10 +501,11 @@ public final class SnapshotInfo extends Configured implements Tool {
*/ */
public static SnapshotStats getSnapshotStats(final Configuration conf, public static SnapshotStats getSnapshotStats(final Configuration conf,
final SnapshotDescription snapshot) throws IOException { final SnapshotDescription snapshot) throws IOException {
HBaseProtos.SnapshotDescription snapshotDesc = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshot);
Path rootDir = FSUtils.getRootDir(conf); Path rootDir = FSUtils.getRootDir(conf);
FileSystem fs = FileSystem.get(rootDir.toUri(), conf); FileSystem fs = FileSystem.get(rootDir.toUri(), conf);
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshot, rootDir); Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotDesc, rootDir);
SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshot); SnapshotManifest manifest = SnapshotManifest.open(conf, fs, snapshotDir, snapshotDesc);
final SnapshotStats stats = new SnapshotStats(conf, fs, snapshot); final SnapshotStats stats = new SnapshotStats(conf, fs, snapshot);
SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest, SnapshotReferenceUtil.concurrentVisitReferencedFiles(conf, fs, manifest,
new SnapshotReferenceUtil.SnapshotVisitor() { new SnapshotReferenceUtil.SnapshotVisitor() {
@ -525,7 +535,11 @@ public final class SnapshotInfo extends Configured implements Tool {
List<SnapshotDescription> snapshotLists = List<SnapshotDescription> snapshotLists =
new ArrayList<SnapshotDescription>(snapshots.length); new ArrayList<SnapshotDescription>(snapshots.length);
for (FileStatus snapshotDirStat: snapshots) { for (FileStatus snapshotDirStat: snapshots) {
snapshotLists.add(SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDirStat.getPath())); HBaseProtos.SnapshotDescription snapshotDesc =
SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDirStat.getPath());
snapshotLists.add(new SnapshotDescription(snapshotDesc.getName(),
snapshotDesc.getTable(), ProtobufUtil.createSnapshotType(snapshotDesc.getType()),
snapshotDesc.getOwner(), snapshotDesc.getCreationTime(), snapshotDesc.getVersion()));
} }
return snapshotLists; return snapshotLists;
} }

View File

@ -104,6 +104,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.HConnection; import org.apache.hadoop.hbase.client.HConnection;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionReplicaUtil; import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -691,7 +692,7 @@ public class HBaseFsck extends Configured implements Closeable {
if (shouldDisableSplitAndMerge()) { if (shouldDisableSplitAndMerge()) {
admin.releaseSplitOrMergeLockAndRollback(); admin.releaseSplitOrMergeLockAndRollback();
oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false, false, oldSplitAndMerge = admin.setSplitOrMergeEnabled(false, false, false,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
} }
try { try {

View File

@ -24,7 +24,7 @@ import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.Abortable; import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
@ -80,7 +80,7 @@ public class SplitOrMergeTracker {
mergeStateTracker.start(); mergeStateTracker.start();
} }
public boolean isSplitOrMergeEnabled(Admin.MasterSwitchType switchType) { public boolean isSplitOrMergeEnabled(MasterSwitchType switchType) {
switch (switchType) { switch (switchType) {
case SPLIT: case SPLIT:
return splitStateTracker.isSwitchEnabled(); return splitStateTracker.isSwitchEnabled();
@ -92,7 +92,7 @@ public class SplitOrMergeTracker {
return false; return false;
} }
public void setSplitOrMergeEnabled(boolean enabled, Admin.MasterSwitchType switchType) public void setSplitOrMergeEnabled(boolean enabled, MasterSwitchType switchType)
throws KeeperException { throws KeeperException {
switch (switchType) { switch (switchType) {
case SPLIT: case SPLIT:
@ -164,8 +164,8 @@ public class SplitOrMergeTracker {
} }
private void saveOriginalState() throws KeeperException { private void saveOriginalState() throws KeeperException {
boolean splitEnabled = isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT); boolean splitEnabled = isSplitOrMergeEnabled(MasterSwitchType.SPLIT);
boolean mergeEnabled = isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE); boolean mergeEnabled = isSplitOrMergeEnabled(MasterSwitchType.MERGE);
String splitOrMergeStates = ZKUtil.joinZNode(watcher.getSwitchLockZNode(), String splitOrMergeStates = ZKUtil.joinZNode(watcher.getSwitchLockZNode(),
SplitOrMergeTracker.STATE); SplitOrMergeTracker.STATE);
ZooKeeperProtos.SplitAndMergeState.Builder builder ZooKeeperProtos.SplitAndMergeState.Builder builder

View File

@ -21,9 +21,9 @@
import="java.util.Date" import="java.util.Date"
import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.client.Admin" import="org.apache.hadoop.hbase.client.Admin"
import="org.apache.hadoop.hbase.client.SnapshotDescription"
import="org.apache.hadoop.hbase.master.HMaster" import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.snapshot.SnapshotInfo" import="org.apache.hadoop.hbase.snapshot.SnapshotInfo"
import="org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription"
import="org.apache.hadoop.util.StringUtils" import="org.apache.hadoop.util.StringUtils"
import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.TableName"
import="org.apache.hadoop.hbase.HBaseConfiguration" %> import="org.apache.hadoop.hbase.HBaseConfiguration" %>

View File

@ -28,6 +28,7 @@
import="org.apache.hadoop.conf.Configuration" import="org.apache.hadoop.conf.Configuration"
import="org.apache.hadoop.hbase.client.HTable" import="org.apache.hadoop.hbase.client.HTable"
import="org.apache.hadoop.hbase.client.Admin" import="org.apache.hadoop.hbase.client.Admin"
import="org.apache.hadoop.hbase.client.CompactionState"
import="org.apache.hadoop.hbase.client.RegionLocator" import="org.apache.hadoop.hbase.client.RegionLocator"
import="org.apache.hadoop.hbase.HRegionInfo" import="org.apache.hadoop.hbase.HRegionInfo"
import="org.apache.hadoop.hbase.HRegionLocation" import="org.apache.hadoop.hbase.HRegionLocation"
@ -39,7 +40,6 @@
import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator" import="org.apache.hadoop.hbase.zookeeper.MetaTableLocator"
import="org.apache.hadoop.hbase.util.Bytes" import="org.apache.hadoop.hbase.util.Bytes"
import="org.apache.hadoop.hbase.util.FSUtils" import="org.apache.hadoop.hbase.util.FSUtils"
import="org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState"
import="org.apache.hadoop.hbase.TableName" import="org.apache.hadoop.hbase.TableName"
import="org.apache.hadoop.hbase.HColumnDescriptor" import="org.apache.hadoop.hbase.HColumnDescriptor"
import="org.apache.hadoop.hbase.client.RegionReplicaUtil" import="org.apache.hadoop.hbase.client.RegionReplicaUtil"

View File

@ -85,7 +85,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService; import org.apache.hadoop.hbase.protobuf.generated.MultiRowMutationProtos.MultiRowMutationService;
@ -278,7 +277,7 @@ public class TestFromClientSide {
@Override @Override
public boolean evaluate() throws IOException { public boolean evaluate() throws IOException {
return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) == return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE; CompactionState.NONE;
} }
}); });

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; import org.apache.hadoop.hbase.snapshot.SnapshotCreationException;
import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException; import org.apache.hadoop.hbase.snapshot.SnapshotDoesNotExistException;
@ -212,18 +212,13 @@ public class TestSnapshotFromClient {
final String SNAPSHOT_NAME = "offlineTableSnapshot"; final String SNAPSHOT_NAME = "offlineTableSnapshot";
byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME); byte[] snapshot = Bytes.toBytes(SNAPSHOT_NAME);
SnapshotDescription desc = SnapshotDescription.newBuilder() admin.snapshot(new SnapshotDescription(SNAPSHOT_NAME, STRING_TABLE_NAME,
.setType(SnapshotDescription.Type.DISABLED) SnapshotType.DISABLED, null, -1, SnapshotManifestV1.DESCRIPTOR_VERSION));
.setTable(STRING_TABLE_NAME)
.setName(SNAPSHOT_NAME)
.setVersion(SnapshotManifestV1.DESCRIPTOR_VERSION)
.build();
admin.snapshot(desc);
LOG.debug("Snapshot completed."); LOG.debug("Snapshot completed.");
// make sure we have the snapshot // make sure we have the snapshot
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, List<SnapshotDescription> snapshots =
snapshot, TABLE_NAME); SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
// make sure its a valid snapshot // make sure its a valid snapshot
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem(); FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
@ -231,9 +226,9 @@ public class TestSnapshotFromClient {
LOG.debug("FS state after snapshot:"); LOG.debug("FS state after snapshot:");
FSUtils.logFileSystemState(UTIL.getTestFileSystem(), FSUtils.logFileSystemState(UTIL.getTestFileSystem(),
FSUtils.getRootDir(UTIL.getConfiguration()), LOG); FSUtils.getRootDir(UTIL.getConfiguration()), LOG);
SnapshotTestingUtils.confirmSnapshotValid(
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, TEST_FAM, rootDir, ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM,
admin, fs); rootDir, admin, fs);
admin.deleteSnapshot(snapshot); admin.deleteSnapshot(snapshot);
snapshots = admin.listSnapshots(); snapshots = admin.listSnapshots();
@ -292,8 +287,8 @@ public class TestSnapshotFromClient {
LOG.debug("Snapshot completed."); LOG.debug("Snapshot completed.");
// make sure we have the snapshot // make sure we have the snapshot
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, List<SnapshotDescription> snapshots =
snapshot, TABLE_NAME); SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
// make sure its a valid snapshot // make sure its a valid snapshot
FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem(); FileSystem fs = UTIL.getHBaseCluster().getMaster().getMasterFileSystem().getFileSystem();
@ -304,8 +299,9 @@ public class TestSnapshotFromClient {
List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region List<byte[]> emptyCfs = Lists.newArrayList(TEST_FAM); // no file in the region
List<byte[]> nonEmptyCfs = Lists.newArrayList(); List<byte[]> nonEmptyCfs = Lists.newArrayList();
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), TABLE_NAME, nonEmptyCfs, emptyCfs, SnapshotTestingUtils.confirmSnapshotValid(
rootDir, admin, fs); ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, nonEmptyCfs,
emptyCfs, rootDir, admin, fs);
admin.deleteSnapshot(snapshot); admin.deleteSnapshot(snapshot);
snapshots = admin.listSnapshots(); snapshots = admin.listSnapshots();
@ -375,7 +371,8 @@ public class TestSnapshotFromClient {
admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME); admin.snapshot(Bytes.toBytes(table2Snapshot1), TABLE_NAME);
LOG.debug(table2Snapshot1 + " completed."); LOG.debug(table2Snapshot1 + " completed.");
List<SnapshotDescription> listTableSnapshots = admin.listTableSnapshots("test.*", "Table1.*"); List<SnapshotDescription> listTableSnapshots =
admin.listTableSnapshots("test.*", "Table1.*");
List<String> listTableSnapshotNames = new ArrayList<String>(); List<String> listTableSnapshotNames = new ArrayList<String>();
assertEquals(2, listTableSnapshots.size()); assertEquals(2, listTableSnapshots.size());
for (SnapshotDescription s : listTableSnapshots) { for (SnapshotDescription s : listTableSnapshots) {

View File

@ -78,14 +78,14 @@ public class TestSplitOrMergeStatus {
Admin admin = TEST_UTIL.getAdmin(); Admin admin = TEST_UTIL.getAdmin();
initSwitchStatus(admin); initSwitchStatus(admin);
boolean[] results = admin.setSplitOrMergeEnabled(false, false, boolean[] results = admin.setSplitOrMergeEnabled(false, false,
true, Admin.MasterSwitchType.SPLIT); true, MasterSwitchType.SPLIT);
assertEquals(results.length, 1); assertEquals(results.length, 1);
assertTrue(results[0]); assertTrue(results[0]);
admin.split(t.getName()); admin.split(t.getName());
int count = waitOnSplitOrMerge(t).size(); int count = waitOnSplitOrMerge(t).size();
assertTrue(orignalCount == count); assertTrue(orignalCount == count);
results = admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.SPLIT); results = admin.setSplitOrMergeEnabled(true, false, true, MasterSwitchType.SPLIT);
assertEquals(results.length, 1); assertEquals(results.length, 1);
assertFalse(results[0]); assertFalse(results[0]);
admin.split(t.getName()); admin.split(t.getName());
@ -111,7 +111,7 @@ public class TestSplitOrMergeStatus {
waitForMergable(admin, name); waitForMergable(admin, name);
int orignalCount = locator.getAllRegionLocations().size(); int orignalCount = locator.getAllRegionLocations().size();
boolean[] results = admin.setSplitOrMergeEnabled(false, false, boolean[] results = admin.setSplitOrMergeEnabled(false, false,
true, Admin.MasterSwitchType.MERGE); true, MasterSwitchType.MERGE);
assertEquals(results.length, 1); assertEquals(results.length, 1);
assertTrue(results[0]); assertTrue(results[0]);
List<HRegionInfo> regions = admin.getTableRegions(t.getName()); List<HRegionInfo> regions = admin.getTableRegions(t.getName());
@ -122,7 +122,7 @@ public class TestSplitOrMergeStatus {
assertTrue(orignalCount == count); assertTrue(orignalCount == count);
waitForMergable(admin, name); waitForMergable(admin, name);
results = admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.MERGE); results = admin.setSplitOrMergeEnabled(true, false, true, MasterSwitchType.MERGE);
assertEquals(results.length, 1); assertEquals(results.length, 1);
assertFalse(results[0]); assertFalse(results[0]);
admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(), admin.mergeRegions(regions.get(0).getEncodedNameAsBytes(),
@ -136,12 +136,12 @@ public class TestSplitOrMergeStatus {
public void testMultiSwitches() throws IOException { public void testMultiSwitches() throws IOException {
Admin admin = TEST_UTIL.getAdmin(); Admin admin = TEST_UTIL.getAdmin();
boolean[] switches = admin.setSplitOrMergeEnabled(false, false, true, boolean[] switches = admin.setSplitOrMergeEnabled(false, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
for (boolean s : switches){ for (boolean s : switches){
assertTrue(s); assertTrue(s);
} }
assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)); assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
assertFalse(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)); assertFalse(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
admin.close(); admin.close();
} }
@ -149,10 +149,10 @@ public class TestSplitOrMergeStatus {
public void testSwitchLock() throws IOException { public void testSwitchLock() throws IOException {
Admin admin = TEST_UTIL.getAdmin(); Admin admin = TEST_UTIL.getAdmin();
admin.setSplitOrMergeEnabled(false, false, false, admin.setSplitOrMergeEnabled(false, false, false,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
try { try {
admin.setSplitOrMergeEnabled(false, false, true, admin.setSplitOrMergeEnabled(false, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
fail(); fail();
} catch (IOException e) { } catch (IOException e) {
LOG.info("", e); LOG.info("", e);
@ -160,7 +160,7 @@ public class TestSplitOrMergeStatus {
admin.releaseSplitOrMergeLockAndRollback(); admin.releaseSplitOrMergeLockAndRollback();
try { try {
admin.setSplitOrMergeEnabled(true, false, true, admin.setSplitOrMergeEnabled(true, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
} catch (IOException e) { } catch (IOException e) {
fail(); fail();
} }
@ -168,14 +168,14 @@ public class TestSplitOrMergeStatus {
} }
private void initSwitchStatus(Admin admin) throws IOException { private void initSwitchStatus(Admin admin) throws IOException {
if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)) { if (!admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT)) {
admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.SPLIT); admin.setSplitOrMergeEnabled(true, false, true, MasterSwitchType.SPLIT);
} }
if (!admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)) { if (!admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE)) {
admin.setSplitOrMergeEnabled(true, false, true, Admin.MasterSwitchType.MERGE); admin.setSplitOrMergeEnabled(true, false, true, MasterSwitchType.MERGE);
} }
assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT)); assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT));
assertTrue(admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE)); assertTrue(admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE));
} }
private void waitForMergable(Admin admin, TableName t) throws InterruptedException, IOException { private void waitForMergable(Admin admin, TableName t) throws InterruptedException, IOException {

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.AssignmentManager; import org.apache.hadoop.hbase.master.AssignmentManager;
@ -348,13 +349,13 @@ public class TestMasterObserver {
@Override @Override
public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public boolean preSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
return false; return false;
} }
@Override @Override
public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx, public void postSetSplitOrMergeEnabled(final ObserverContext<MasterCoprocessorEnvironment> ctx,
final boolean newValue, final Admin.MasterSwitchType switchType) throws IOException { final boolean newValue, final MasterSwitchType switchType) throws IOException {
} }
@Override @Override

View File

@ -69,6 +69,7 @@ import org.apache.hadoop.hbase.SplitLogCounters;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.ConnectionUtils; import org.apache.hadoop.hbase.client.ConnectionUtils;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
@ -87,7 +88,6 @@ import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException; import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch; import org.apache.hadoop.hbase.master.SplitLogManager.TaskBatch;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.Region;
@ -1421,7 +1421,8 @@ public class TestDistributedLogSplitting {
TEST_UTIL.waitFor(30000, 200, new Waiter.Predicate<Exception>() { TEST_UTIL.waitFor(30000, 200, new Waiter.Predicate<Exception>() {
@Override @Override
public boolean evaluate() throws Exception { public boolean evaluate() throws Exception {
return (TEST_UTIL.getHBaseAdmin().getCompactionState(tableName) == CompactionState.NONE); return (TEST_UTIL.getHBaseAdmin()
.getCompactionState(tableName) == CompactionState.NONE);
} }
}); });

View File

@ -31,7 +31,8 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter; import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -106,7 +107,7 @@ public class TestWarmupRegion {
@Override @Override
public boolean evaluate() throws IOException { public boolean evaluate() throws IOException {
return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) == return TEST_UTIL.getHBaseAdmin().getCompactionState(TABLENAME) ==
AdminProtos.GetRegionInfoResponse.CompactionState.NONE; CompactionState.NONE;
} }
}); });

View File

@ -33,7 +33,9 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.CloneSnapshotState;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -59,7 +61,7 @@ public class TestCloneSnapshotProcedure {
private static long nonceGroup = HConstants.NO_NONCE; private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE; private static long nonce = HConstants.NO_NONCE;
private static SnapshotDescription snapshot = null; private static HBaseProtos.SnapshotDescription snapshot = null;
private static void setupConf(Configuration conf) { private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1); conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
@ -99,7 +101,7 @@ public class TestCloneSnapshotProcedure {
assertTrue("expected executor to be running", procExec.isRunning()); assertTrue("expected executor to be running", procExec.isRunning());
} }
private SnapshotDescription getSnapshot() throws Exception { private HBaseProtos.SnapshotDescription getSnapshot() throws Exception {
if (snapshot == null) { if (snapshot == null) {
final TableName snapshotTableName = TableName.valueOf("testCloneSnapshot"); final TableName snapshotTableName = TableName.valueOf("testCloneSnapshot");
long tid = System.currentTimeMillis(); long tid = System.currentTimeMillis();
@ -116,7 +118,7 @@ public class TestCloneSnapshotProcedure {
admin.enableTable(snapshotTableName); admin.enableTable(snapshotTableName);
List<SnapshotDescription> snapshotList = admin.listSnapshots(); List<SnapshotDescription> snapshotList = admin.listSnapshots();
snapshot = snapshotList.get(0); snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotList.get(0));
} }
return snapshot; return snapshot;
} }

View File

@ -35,7 +35,9 @@ import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor; import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility; import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.RestoreSnapshotState;
import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils; import org.apache.hadoop.hbase.snapshot.SnapshotTestingUtils;
import org.apache.hadoop.hbase.testclassification.MasterTests; import org.apache.hadoop.hbase.testclassification.MasterTests;
@ -73,7 +75,7 @@ public class TestRestoreSnapshotProcedure {
private static long nonceGroup = HConstants.NO_NONCE; private static long nonceGroup = HConstants.NO_NONCE;
private static long nonce = HConstants.NO_NONCE; private static long nonce = HConstants.NO_NONCE;
private SnapshotDescription snapshot = null; private HBaseProtos.SnapshotDescription snapshot = null;
private HTableDescriptor snapshotHTD = null; private HTableDescriptor snapshotHTD = null;
private static void setupConf(Configuration conf) { private static void setupConf(Configuration conf) {
@ -141,7 +143,7 @@ public class TestRestoreSnapshotProcedure {
admin.snapshot(snapshotName, snapshotTableName); admin.snapshot(snapshotName, snapshotTableName);
List<SnapshotDescription> snapshotList = admin.listSnapshots(); List<SnapshotDescription> snapshotList = admin.listSnapshots();
snapshot = snapshotList.get(0); snapshot = ProtobufUtil.createHBaseProtosSnapshotDesc(snapshotList.get(0));
// modify the table // modify the table
HColumnDescriptor columnFamilyDescriptor3 = new HColumnDescriptor(CF3); HColumnDescriptor columnFamilyDescriptor3 = new HColumnDescriptor(CF3);

View File

@ -51,6 +51,8 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.CompactType;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
@ -68,7 +70,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.HFile; import org.apache.hadoop.hbase.io.hfile.HFile;
import org.apache.hadoop.hbase.mob.MobConstants; import org.apache.hadoop.hbase.mob.MobConstants;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.StoreFile; import org.apache.hadoop.hbase.regionserver.StoreFile;
@ -356,7 +357,7 @@ public class TestMobCompactor {
countFiles(tableName, false, family2)); countFiles(tableName, false, family2));
// do the major mob compaction, it will force all files to compaction // do the major mob compaction, it will force all files to compaction
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB); admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
waitUntilMobCompactionFinished(tableName); waitUntilMobCompactionFinished(tableName);
assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum), assertEquals("After compaction: mob rows count", regionNum * (rowNumPerRegion - delRowNum),
@ -399,7 +400,7 @@ public class TestMobCompactor {
Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1)); Cell cell = result.getColumnLatestCell(hcd1.getName(), Bytes.toBytes(qf1));
assertEquals("Before compaction: mob value of k0", newValue0, assertEquals("Before compaction: mob value of k0", newValue0,
Bytes.toString(CellUtil.cloneValue(cell))); Bytes.toString(CellUtil.cloneValue(cell)));
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB); admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
waitUntilMobCompactionFinished(tableName); waitUntilMobCompactionFinished(tableName);
// read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the // read the latest cell of key0, the cell seqId in bulk loaded file is not reset in the
// scanner. The cell that has "new" value is still visible. // scanner. The cell that has "new" value is still visible.
@ -449,7 +450,7 @@ public class TestMobCompactor {
loadData(admin, bufMut, tableName, new Put[] { put1 }); // now two mob files loadData(admin, bufMut, tableName, new Put[] { put1 }); // now two mob files
admin.majorCompact(tableName); admin.majorCompact(tableName);
waitUntilCompactionFinished(tableName); waitUntilCompactionFinished(tableName);
admin.majorCompact(tableName, hcd1.getName(), Admin.CompactType.MOB); admin.majorCompact(tableName, hcd1.getName(), CompactType.MOB);
waitUntilMobCompactionFinished(tableName); waitUntilMobCompactionFinished(tableName);
// read the latest cell of key1. // read the latest cell of key1.
Get get = new Get(key1); Get get = new Get(key1);
@ -475,12 +476,12 @@ public class TestMobCompactor {
private void waitUntilMobCompactionFinished(TableName tableName) throws IOException, private void waitUntilMobCompactionFinished(TableName tableName) throws IOException,
InterruptedException { InterruptedException {
long finished = EnvironmentEdgeManager.currentTime() + 60000; long finished = EnvironmentEdgeManager.currentTime() + 60000;
CompactionState state = admin.getCompactionState(tableName, Admin.CompactType.MOB); CompactionState state = admin.getCompactionState(tableName, CompactType.MOB);
while (EnvironmentEdgeManager.currentTime() < finished) { while (EnvironmentEdgeManager.currentTime() < finished) {
if (state == CompactionState.NONE) { if (state == CompactionState.NONE) {
break; break;
} }
state = admin.getCompactionState(tableName, Admin.CompactType.MOB); state = admin.getCompactionState(tableName, CompactType.MOB);
Thread.sleep(10); Thread.sleep(10);
} }
assertEquals(CompactionState.NONE, state); assertEquals(CompactionState.NONE, state);

View File

@ -31,9 +31,9 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests; import org.apache.hadoop.hbase.testclassification.VerySlowRegionServerTests;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -164,7 +164,7 @@ public class TestCompactionState {
// otherwise, the compaction should have already been done // otherwise, the compaction should have already been done
if (expectedState != state) { if (expectedState != state) {
for (Region region: regions) { for (Region region: regions) {
state = region.getCompactionState(); state = CompactionState.valueOf(region.getCompactionState().toString());
assertEquals(CompactionState.NONE, state); assertEquals(CompactionState.NONE, state);
} }
} else { } else {

View File

@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.TagUtil;
import org.apache.hadoop.hbase.ArrayBackedTag; import org.apache.hadoop.hbase.ArrayBackedTag;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Append; import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.CompactionState;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Mutation;
@ -53,7 +54,6 @@ import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
import org.apache.hadoop.hbase.coprocessor.ObserverContext; import org.apache.hadoop.hbase.coprocessor.ObserverContext;
import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;

View File

@ -62,6 +62,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment; import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -710,7 +711,7 @@ public class TestAccessController extends SecureTestUtil {
@Override @Override
public Object run() throws Exception { public Object run() throws Exception {
ACCESS_CONTROLLER.preSetSplitOrMergeEnabled(ObserverContext.createAndPrepare(CP_ENV, null), ACCESS_CONTROLLER.preSetSplitOrMergeEnabled(ObserverContext.createAndPrepare(CP_ENV, null),
true, Admin.MasterSwitchType.MERGE); true, MasterSwitchType.MERGE);
return null; return null;
} }
}; };

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.hbase.TableDescriptor;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.TableNotEnabledException; import org.apache.hadoop.hbase.TableNotEnabledException;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.SnapshotType;
import org.apache.hadoop.hbase.client.BufferedMutator; import org.apache.hadoop.hbase.client.BufferedMutator;
import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
@ -59,7 +60,8 @@ import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterFileSystem; import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.mob.MobUtils; import org.apache.hadoop.hbase.mob.MobUtils;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest; import org.apache.hadoop.hbase.protobuf.generated.SnapshotProtos.SnapshotRegionManifest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneRequest;
import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsSnapshotDoneResponse;
@ -120,7 +122,7 @@ public class SnapshotTestingUtils {
* Make sure that there is only one snapshot returned from the master * Make sure that there is only one snapshot returned from the master
*/ */
public static void assertOneSnapshotThatMatches(Admin admin, public static void assertOneSnapshotThatMatches(Admin admin,
SnapshotDescription snapshot) throws IOException { HBaseProtos.SnapshotDescription snapshot) throws IOException {
assertOneSnapshotThatMatches(admin, snapshot.getName(), assertOneSnapshotThatMatches(admin, snapshot.getName(),
TableName.valueOf(snapshot.getTable())); TableName.valueOf(snapshot.getTable()));
} }
@ -153,7 +155,7 @@ public class SnapshotTestingUtils {
} }
public static void confirmSnapshotValid(HBaseTestingUtility testUtil, public static void confirmSnapshotValid(HBaseTestingUtility testUtil,
SnapshotDescription snapshotDescriptor, TableName tableName, byte[] family) HBaseProtos.SnapshotDescription snapshotDescriptor, TableName tableName, byte[] family)
throws IOException { throws IOException {
MasterFileSystem mfs = testUtil.getHBaseCluster().getMaster().getMasterFileSystem(); MasterFileSystem mfs = testUtil.getHBaseCluster().getMaster().getMasterFileSystem();
confirmSnapshotValid(snapshotDescriptor, tableName, family, confirmSnapshotValid(snapshotDescriptor, tableName, family,
@ -165,7 +167,7 @@ public class SnapshotTestingUtils {
* be in the snapshot. * be in the snapshot.
*/ */
public static void confirmSnapshotValid( public static void confirmSnapshotValid(
SnapshotDescription snapshotDescriptor, TableName tableName, HBaseProtos.SnapshotDescription snapshotDescriptor, TableName tableName,
byte[] testFamily, Path rootDir, Admin admin, FileSystem fs) byte[] testFamily, Path rootDir, Admin admin, FileSystem fs)
throws IOException { throws IOException {
ArrayList nonEmptyTestFamilies = new ArrayList(1); ArrayList nonEmptyTestFamilies = new ArrayList(1);
@ -178,7 +180,7 @@ public class SnapshotTestingUtils {
* Confirm that the snapshot has no references files but only metadata. * Confirm that the snapshot has no references files but only metadata.
*/ */
public static void confirmEmptySnapshotValid( public static void confirmEmptySnapshotValid(
SnapshotDescription snapshotDescriptor, TableName tableName, HBaseProtos.SnapshotDescription snapshotDescriptor, TableName tableName,
byte[] testFamily, Path rootDir, Admin admin, FileSystem fs) byte[] testFamily, Path rootDir, Admin admin, FileSystem fs)
throws IOException { throws IOException {
ArrayList emptyTestFamilies = new ArrayList(1); ArrayList emptyTestFamilies = new ArrayList(1);
@ -194,7 +196,7 @@ public class SnapshotTestingUtils {
* by the MasterSnapshotVerifier, at the end of the snapshot operation. * by the MasterSnapshotVerifier, at the end of the snapshot operation.
*/ */
public static void confirmSnapshotValid( public static void confirmSnapshotValid(
SnapshotDescription snapshotDescriptor, TableName tableName, HBaseProtos.SnapshotDescription snapshotDescriptor, TableName tableName,
List<byte[]> nonEmptyTestFamilies, List<byte[]> emptyTestFamilies, List<byte[]> nonEmptyTestFamilies, List<byte[]> emptyTestFamilies,
Path rootDir, Admin admin, FileSystem fs) throws IOException { Path rootDir, Admin admin, FileSystem fs) throws IOException {
final Configuration conf = admin.getConfiguration(); final Configuration conf = admin.getConfiguration();
@ -204,7 +206,7 @@ public class SnapshotTestingUtils {
snapshotDescriptor, rootDir); snapshotDescriptor, rootDir);
assertTrue(fs.exists(snapshotDir)); assertTrue(fs.exists(snapshotDir));
SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); HBaseProtos.SnapshotDescription desc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
// Extract regions and families with store files // Extract regions and families with store files
final Set<byte[]> snapshotFamilies = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR); final Set<byte[]> snapshotFamilies = new TreeSet<byte[]>(Bytes.BYTES_COMPARATOR);
@ -265,7 +267,7 @@ public class SnapshotTestingUtils {
* @throws ServiceException if the snapshot fails * @throws ServiceException if the snapshot fails
*/ */
public static void waitForSnapshotToComplete(HMaster master, public static void waitForSnapshotToComplete(HMaster master,
SnapshotDescription snapshot, long sleep) throws ServiceException { HBaseProtos.SnapshotDescription snapshot, long sleep) throws ServiceException {
final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder() final IsSnapshotDoneRequest request = IsSnapshotDoneRequest.newBuilder()
.setSnapshot(snapshot).build(); .setSnapshot(snapshot).build();
IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder() IsSnapshotDoneResponse done = IsSnapshotDoneResponse.newBuilder()
@ -286,12 +288,13 @@ public class SnapshotTestingUtils {
*/ */
public static void snapshot(Admin admin, public static void snapshot(Admin admin,
final String snapshotName, final String tableName, final String snapshotName, final String tableName,
SnapshotDescription.Type type, int numTries) throws IOException { HBaseProtos.SnapshotDescription.Type type, int numTries) throws IOException {
int tries = 0; int tries = 0;
CorruptedSnapshotException lastEx = null; CorruptedSnapshotException lastEx = null;
while (tries++ < numTries) { while (tries++ < numTries) {
try { try {
admin.snapshot(snapshotName, TableName.valueOf(tableName), type); admin.snapshot(new SnapshotDescription(snapshotName, tableName,
SnapshotType.valueOf(type.toString())));
return; return;
} catch (CorruptedSnapshotException cse) { } catch (CorruptedSnapshotException cse) {
LOG.warn("Got CorruptedSnapshotException", cse); LOG.warn("Got CorruptedSnapshotException", cse);
@ -393,13 +396,14 @@ public class SnapshotTestingUtils {
} }
admin.snapshot(snapshotNameString, tableName); admin.snapshot(snapshotNameString, tableName);
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertExistsMatchingSnapshot(admin, List<SnapshotDescription> snapshots =
snapshotNameString, tableName); SnapshotTestingUtils.assertExistsMatchingSnapshot(admin, snapshotNameString, tableName);
if (snapshots == null || snapshots.size() != 1) { if (snapshots == null || snapshots.size() != 1) {
Assert.fail("Incorrect number of snapshots for table " + tableName); Assert.fail("Incorrect number of snapshots for table " + tableName);
} }
SnapshotTestingUtils.confirmSnapshotValid(snapshots.get(0), tableName, nonEmptyFamilyNames, SnapshotTestingUtils.confirmSnapshotValid(
ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), tableName, nonEmptyFamilyNames,
emptyFamilyNames, rootDir, admin, fs); emptyFamilyNames, rootDir, admin, fs);
} }
@ -418,7 +422,8 @@ public class SnapshotTestingUtils {
Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName, Path snapshotDir = SnapshotDescriptionUtils.getCompletedSnapshotDir(snapshotName,
mfs.getRootDir()); mfs.getRootDir());
SnapshotDescription snapshotDesc = SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir); HBaseProtos.SnapshotDescription snapshotDesc =
SnapshotDescriptionUtils.readSnapshotInfo(fs, snapshotDir);
final TableName table = TableName.valueOf(snapshotDesc.getTable()); final TableName table = TableName.valueOf(snapshotDesc.getTable());
final ArrayList corruptedFiles = new ArrayList(); final ArrayList corruptedFiles = new ArrayList();
@ -467,7 +472,7 @@ public class SnapshotTestingUtils {
public static class SnapshotBuilder { public static class SnapshotBuilder {
private final RegionData[] tableRegions; private final RegionData[] tableRegions;
private final SnapshotDescription desc; private final HBaseProtos.SnapshotDescription desc;
private final HTableDescriptor htd; private final HTableDescriptor htd;
private final Configuration conf; private final Configuration conf;
private final FileSystem fs; private final FileSystem fs;
@ -477,7 +482,7 @@ public class SnapshotTestingUtils {
public SnapshotBuilder(final Configuration conf, final FileSystem fs, public SnapshotBuilder(final Configuration conf, final FileSystem fs,
final Path rootDir, final HTableDescriptor htd, final Path rootDir, final HTableDescriptor htd,
final SnapshotDescription desc, final RegionData[] tableRegions) final HBaseProtos.SnapshotDescription desc, final RegionData[] tableRegions)
throws IOException { throws IOException {
this.fs = fs; this.fs = fs;
this.conf = conf; this.conf = conf;
@ -495,7 +500,7 @@ public class SnapshotTestingUtils {
return this.htd; return this.htd;
} }
public SnapshotDescription getSnapshotDescription() { public HBaseProtos.SnapshotDescription getSnapshotDescription() {
return this.desc; return this.desc;
} }
@ -519,7 +524,7 @@ public class SnapshotTestingUtils {
.build()); .build());
} }
private Path[] addRegion(final SnapshotDescription desc) throws IOException { private Path[] addRegion(final HBaseProtos.SnapshotDescription desc) throws IOException {
if (this.snapshotted == tableRegions.length) { if (this.snapshotted == tableRegions.length) {
throw new UnsupportedOperationException("No more regions in the table"); throw new UnsupportedOperationException("No more regions in the table");
} }
@ -648,7 +653,7 @@ public class SnapshotTestingUtils {
HTableDescriptor htd = createHtd(tableName); HTableDescriptor htd = createHtd(tableName);
RegionData[] regions = createTable(htd, numRegions); RegionData[] regions = createTable(htd, numRegions);
SnapshotDescription desc = SnapshotDescription.newBuilder() HBaseProtos.SnapshotDescription desc = HBaseProtos.SnapshotDescription.newBuilder()
.setTable(htd.getNameAsString()) .setTable(htd.getNameAsString())
.setName(snapshotName) .setName(snapshotName)
.setVersion(version) .setVersion(version)

View File

@ -44,7 +44,9 @@ import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.HMaster; import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.client.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy; import org.apache.hadoop.hbase.regionserver.ConstantSizeRegionSplitPolicy;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests; import org.apache.hadoop.hbase.testclassification.RegionServerTests;
@ -147,18 +149,20 @@ public class TestFlushSnapshotFromClient {
// take a snapshot of the enabled table // take a snapshot of the enabled table
String snapshotString = "offlineTableSnapshot"; String snapshotString = "offlineTableSnapshot";
byte[] snapshot = Bytes.toBytes(snapshotString); byte[] snapshot = Bytes.toBytes(snapshotString);
admin.snapshot(snapshotString, TABLE_NAME, SnapshotDescription.Type.FLUSH); admin.snapshot(snapshotString, TABLE_NAME,
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH));
LOG.debug("Snapshot completed."); LOG.debug("Snapshot completed.");
// make sure we have the snapshot // make sure we have the snapshot
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, List<SnapshotDescription> snapshots =
snapshot, TABLE_NAME); SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
// make sure its a valid snapshot // make sure its a valid snapshot
LOG.debug("FS state after snapshot:"); LOG.debug("FS state after snapshot:");
UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG); UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
SnapshotTestingUtils.confirmSnapshotValid(UTIL, snapshots.get(0), TABLE_NAME, TEST_FAM); SnapshotTestingUtils.confirmSnapshotValid(UTIL,
ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM);
} }
/** /**
@ -181,18 +185,20 @@ public class TestFlushSnapshotFromClient {
// take a snapshot of the enabled table // take a snapshot of the enabled table
String snapshotString = "skipFlushTableSnapshot"; String snapshotString = "skipFlushTableSnapshot";
byte[] snapshot = Bytes.toBytes(snapshotString); byte[] snapshot = Bytes.toBytes(snapshotString);
admin.snapshot(snapshotString, TABLE_NAME, SnapshotDescription.Type.SKIPFLUSH); admin.snapshot(snapshotString, TABLE_NAME,
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.SKIPFLUSH));
LOG.debug("Snapshot completed."); LOG.debug("Snapshot completed.");
// make sure we have the snapshot // make sure we have the snapshot
List<SnapshotDescription> snapshots = SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, List<SnapshotDescription> snapshots =
snapshot, TABLE_NAME); SnapshotTestingUtils.assertOneSnapshotThatMatches(admin, snapshot, TABLE_NAME);
// make sure its a valid snapshot // make sure its a valid snapshot
LOG.debug("FS state after snapshot:"); LOG.debug("FS state after snapshot:");
UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG); UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
SnapshotTestingUtils.confirmSnapshotValid(UTIL, snapshots.get(0), TABLE_NAME, TEST_FAM); SnapshotTestingUtils.confirmSnapshotValid(UTIL,
ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM);
admin.deleteSnapshot(snapshot); admin.deleteSnapshot(snapshot);
snapshots = admin.listSnapshots(); snapshots = admin.listSnapshots();
@ -234,7 +240,8 @@ public class TestFlushSnapshotFromClient {
LOG.debug("FS state after snapshot:"); LOG.debug("FS state after snapshot:");
UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG); UTIL.getHBaseCluster().getMaster().getMasterFileSystem().logFileSystemState(LOG);
SnapshotTestingUtils.confirmSnapshotValid(UTIL, snapshots.get(0), TABLE_NAME, TEST_FAM); SnapshotTestingUtils.confirmSnapshotValid(UTIL,
ProtobufUtil.createHBaseProtosSnapshotDesc(snapshots.get(0)), TABLE_NAME, TEST_FAM);
} }
@Test @Test
@ -258,7 +265,8 @@ public class TestFlushSnapshotFromClient {
// snapshot the non-existant table // snapshot the non-existant table
try { try {
admin.snapshot("fail", tableName, SnapshotDescription.Type.FLUSH); admin.snapshot("fail", tableName,
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH));
fail("Snapshot succeeded even though there is not table."); fail("Snapshot succeeded even though there is not table.");
} catch (SnapshotCreationException e) { } catch (SnapshotCreationException e) {
LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage()); LOG.info("Correctly failed to snapshot a non-existant table:" + e.getMessage());
@ -267,13 +275,14 @@ public class TestFlushSnapshotFromClient {
@Test @Test
public void testAsyncFlushSnapshot() throws Exception { public void testAsyncFlushSnapshot() throws Exception {
SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("asyncSnapshot") HBaseProtos.SnapshotDescription snapshot = HBaseProtos.SnapshotDescription.newBuilder()
.setTable(TABLE_NAME.getNameAsString()) .setName("asyncSnapshot").setTable(TABLE_NAME.getNameAsString())
.setType(SnapshotDescription.Type.FLUSH) .setType(HBaseProtos.SnapshotDescription.Type.FLUSH).build();
.build();
// take the snapshot async // take the snapshot async
admin.takeSnapshotAsync(snapshot); admin.takeSnapshotAsync(
new SnapshotDescription("asyncSnapshot", TABLE_NAME.getNameAsString(),
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH)));
// constantly loop, looking for the snapshot to complete // constantly loop, looking for the snapshot to complete
HMaster master = UTIL.getMiniHBaseCluster().getMaster(); HMaster master = UTIL.getMiniHBaseCluster().getMaster();
@ -295,7 +304,8 @@ public class TestFlushSnapshotFromClient {
// Take a snapshot // Take a snapshot
String snapshotBeforeMergeName = "snapshotBeforeMerge"; String snapshotBeforeMergeName = "snapshotBeforeMerge";
admin.snapshot(snapshotBeforeMergeName, TABLE_NAME, SnapshotDescription.Type.FLUSH); admin.snapshot(snapshotBeforeMergeName, TABLE_NAME,
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH));
// Clone the table // Clone the table
TableName cloneBeforeMergeName = TableName.valueOf("cloneBeforeMerge"); TableName cloneBeforeMergeName = TableName.valueOf("cloneBeforeMerge");
@ -364,7 +374,7 @@ public class TestFlushSnapshotFromClient {
// Take a snapshot // Take a snapshot
String snapshotName = "snapshotAfterMerge"; String snapshotName = "snapshotAfterMerge";
SnapshotTestingUtils.snapshot(admin, snapshotName, TABLE_NAME.getNameAsString(), SnapshotTestingUtils.snapshot(admin, snapshotName, TABLE_NAME.getNameAsString(),
SnapshotDescription.Type.FLUSH, 3); HBaseProtos.SnapshotDescription.Type.FLUSH, 3);
// Clone the table // Clone the table
TableName cloneName = TableName.valueOf("cloneMerge"); TableName cloneName = TableName.valueOf("cloneMerge");
@ -425,14 +435,16 @@ public class TestFlushSnapshotFromClient {
@Override @Override
public void run() { public void run() {
try { try {
LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss)); LOG.info("Submitting snapshot request: " + ClientSnapshotDescriptionUtils
.toString(ProtobufUtil.createHBaseProtosSnapshotDesc(ss)));
admin.takeSnapshotAsync(ss); admin.takeSnapshotAsync(ss);
} catch (Exception e) { } catch (Exception e) {
LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString( LOG.info("Exception during snapshot request: " + ClientSnapshotDescriptionUtils.toString(
ss) ProtobufUtil.createHBaseProtosSnapshotDesc(ss))
+ ". This is ok, we expect some", e); + ". This is ok, we expect some", e);
} }
LOG.info("Submitted snapshot request: " + ClientSnapshotDescriptionUtils.toString(ss)); LOG.info("Submitted snapshot request: " + ClientSnapshotDescriptionUtils
.toString(ProtobufUtil.createHBaseProtosSnapshotDesc(ss)));
toBeSubmitted.countDown(); toBeSubmitted.countDown();
} }
}; };
@ -440,11 +452,15 @@ public class TestFlushSnapshotFromClient {
// build descriptions // build descriptions
SnapshotDescription[] descs = new SnapshotDescription[ssNum]; SnapshotDescription[] descs = new SnapshotDescription[ssNum];
for (int i = 0; i < ssNum; i++) { for (int i = 0; i < ssNum; i++) {
SnapshotDescription.Builder builder = SnapshotDescription.newBuilder(); HBaseProtos.SnapshotDescription.Builder builder =
builder.setTable(((i % 2) == 0 ? TABLE_NAME : TABLE2_NAME).getNameAsString()); HBaseProtos.SnapshotDescription.newBuilder();
builder.setName("ss"+i); if(i %2 ==0) {
builder.setType(SnapshotDescription.Type.FLUSH); descs[i] = new SnapshotDescription("ss" + i, TABLE_NAME.getNameAsString(),
descs[i] = builder.build(); ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH));
} else {
descs[i] = new SnapshotDescription("ss" + i, TABLE2_NAME.getNameAsString(),
ProtobufUtil.createSnapshotType(HBaseProtos.SnapshotDescription.Type.FLUSH));
}
} }
// kick each off its own thread // kick each off its own thread

View File

@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin; import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.master.snapshot.SnapshotManager; import org.apache.hadoop.hbase.master.snapshot.SnapshotManager;
import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager; import org.apache.hadoop.hbase.regionserver.snapshot.RegionServerSnapshotManager;
import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.testclassification.LargeTests;
@ -116,7 +117,7 @@ public class TestRestoreFlushSnapshotFromClient {
// take a snapshot // take a snapshot
admin.snapshot(Bytes.toString(snapshotName0), tableName, admin.snapshot(Bytes.toString(snapshotName0), tableName,
SnapshotDescription.Type.FLUSH); ProtobufUtil.createSnapshotType(SnapshotDescription.Type.FLUSH));
LOG.info("=== after snapshot with 500 rows"); LOG.info("=== after snapshot with 500 rows");
logFSTree(); logFSTree();
@ -129,7 +130,7 @@ public class TestRestoreFlushSnapshotFromClient {
// take a snapshot of the updated table // take a snapshot of the updated table
admin.snapshot(Bytes.toString(snapshotName1), tableName, admin.snapshot(Bytes.toString(snapshotName1), tableName,
SnapshotDescription.Type.FLUSH); ProtobufUtil.createSnapshotType(SnapshotDescription.Type.FLUSH));
LOG.info("=== after snapshot with 1000 rows"); LOG.info("=== after snapshot with 1000 rows");
logFSTree(); logFSTree();
table.close(); table.close();
@ -194,7 +195,8 @@ public class TestRestoreFlushSnapshotFromClient {
TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis()); TableName clonedTableName = TableName.valueOf("clonedtb-" + System.currentTimeMillis());
admin.cloneSnapshot(snapshotName0, clonedTableName); admin.cloneSnapshot(snapshotName0, clonedTableName);
verifyRowCount(UTIL, clonedTableName, snapshot0Rows); verifyRowCount(UTIL, clonedTableName, snapshot0Rows);
admin.snapshot(Bytes.toString(snapshotName2), clonedTableName, SnapshotDescription.Type.FLUSH); admin.snapshot(Bytes.toString(snapshotName2), clonedTableName,
ProtobufUtil.createSnapshotType(SnapshotDescription.Type.FLUSH));
UTIL.deleteTable(clonedTableName); UTIL.deleteTable(clonedTableName);
admin.cloneSnapshot(snapshotName2, clonedTableName); admin.cloneSnapshot(snapshotName2, clonedTableName);

View File

@ -33,12 +33,12 @@ import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MiniHBaseCluster; import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.ClusterConnection;
import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Get; import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result; import org.apache.hadoop.hbase.client.Result;
@ -1856,9 +1856,9 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
@Test @Test
public void testSplitOrMergeStatWhenHBCKAbort() throws Exception { public void testSplitOrMergeStatWhenHBCKAbort() throws Exception {
admin.setSplitOrMergeEnabled(true, false, true, admin.setSplitOrMergeEnabled(true, false, true,
Admin.MasterSwitchType.SPLIT, Admin.MasterSwitchType.MERGE); MasterSwitchType.SPLIT, MasterSwitchType.MERGE);
boolean oldSplit = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT); boolean oldSplit = admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT);
boolean oldMerge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE); boolean oldMerge = admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE);
assertTrue(oldSplit); assertTrue(oldSplit);
assertTrue(oldMerge); assertTrue(oldMerge);
@ -1880,8 +1880,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
spiedHbck.onlineHbck(); spiedHbck.onlineHbck();
spiedHbck.close(); spiedHbck.close();
boolean split = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT); boolean split = admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT);
boolean merge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE); boolean merge = admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE);
assertFalse(split); assertFalse(split);
assertFalse(merge); assertFalse(merge);
@ -1892,8 +1892,8 @@ public class TestHBaseFsckOneRS extends BaseTestHBaseFsck {
hbck.onlineHbck(); hbck.onlineHbck();
hbck.close(); hbck.close();
split = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.SPLIT); split = admin.isSplitOrMergeEnabled(MasterSwitchType.SPLIT);
merge = admin.isSplitOrMergeEnabled(Admin.MasterSwitchType.MERGE); merge = admin.isSplitOrMergeEnabled(MasterSwitchType.MERGE);
assertTrue(split); assertTrue(split);
assertTrue(merge); assertTrue(merge);