HBASE-23329 Remove unused methods from RequestConverter (#870)

Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
Wellington Ramos Chevreuil 2019-11-22 14:33:22 +00:00 committed by GitHub
parent 54ad797abb
commit 3b0c276aa3
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 0 additions and 186 deletions

View File

@ -39,7 +39,6 @@ import org.apache.hadoop.hbase.client.Action;
import org.apache.hadoop.hbase.client.Append;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Increment;
import org.apache.hadoop.hbase.client.MasterSwitchType;
@ -87,8 +86,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.GetRequest
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MultiRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.ColumnValue.QualifierValue;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutationProto.MutationType;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.RegionAction;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ScanRequest;
@ -112,10 +109,8 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableCata
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.EnableTableRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetClusterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetNamespaceDescriptorRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetSchemaAlterStatusRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableDescriptorsRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableNamesRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.GetTableStateRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsBalancerEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCatalogJanitorEnabledRequest;
import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProtos.IsCleanerChoreEnabledRequest;
@ -187,48 +182,6 @@ public final class RequestConverter {
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a client increment
*
* @param regionName
* @param row
* @param family
* @param qualifier
* @param amount
* @param durability
* @return a mutate request
*/
public static MutateRequest buildIncrementRequest(
final byte[] regionName, final byte[] row, final byte[] family, final byte[] qualifier,
final long amount, final Durability durability, long nonceGroup, long nonce) {
MutateRequest.Builder builder = MutateRequest.newBuilder();
RegionSpecifier region = buildRegionSpecifier(
RegionSpecifierType.REGION_NAME, regionName);
builder.setRegion(region);
MutationProto.Builder mutateBuilder = MutationProto.newBuilder();
mutateBuilder.setRow(UnsafeByteOperations.unsafeWrap(row));
mutateBuilder.setMutateType(MutationType.INCREMENT);
mutateBuilder.setDurability(ProtobufUtil.toDurability(durability));
ColumnValue.Builder columnBuilder = ColumnValue.newBuilder();
columnBuilder.setFamily(UnsafeByteOperations.unsafeWrap(family));
QualifierValue.Builder valueBuilder = QualifierValue.newBuilder();
valueBuilder.setValue(UnsafeByteOperations.unsafeWrap(Bytes.toBytes(amount)));
valueBuilder.setQualifier(UnsafeByteOperations
.unsafeWrap(qualifier == null ? HConstants.EMPTY_BYTE_ARRAY : qualifier));
valueBuilder.setTimestamp(HConstants.LATEST_TIMESTAMP);
columnBuilder.addQualifierValue(valueBuilder.build());
mutateBuilder.addColumnValue(columnBuilder.build());
if (nonce != HConstants.NO_NONCE) {
mutateBuilder.setNonce(nonce);
}
builder.setMutation(mutateBuilder.build());
if (nonceGroup != HConstants.NO_NONCE) {
builder.setNonceGroup(nonceGroup);
}
return builder.build();
}
/**
* Create a protocol buffer MutateRequest for a conditioned put
*
@ -553,24 +506,6 @@ public final class RequestConverter {
return builder.build();
}
/**
* Create a protocol buffer bulk load request
*
* @param familyPaths
* @param regionName
* @param assignSeqNum
* @param userToken
* @param bulkToken
* @return a bulk load request
*/
public static BulkLoadHFileRequest buildBulkLoadHFileRequest(
final List<Pair<byte[], String>> familyPaths,
final byte[] regionName, boolean assignSeqNum,
final Token<?> userToken, final String bulkToken) {
return buildBulkLoadHFileRequest(familyPaths, regionName, assignSeqNum, userToken, bulkToken,
false, null, true);
}
/**
* Create a protocol buffer bulk load request
*
@ -630,101 +565,6 @@ public final class RequestConverter {
return request.build();
}
/**
* Create a protocol buffer multi request for a list of actions. Propagates Actions original
* index. The passed in multiRequestBuilder will be populated with region actions.
* @param regionName The region name of the actions.
* @param actions The actions that are grouped by the same region name.
* @param multiRequestBuilder The multiRequestBuilder to be populated with region actions.
* @param regionActionBuilder regionActionBuilder to be used to build region action.
* @param actionBuilder actionBuilder to be used to build action.
* @param mutationBuilder mutationBuilder to be used to build mutation.
* @param nonceGroup nonceGroup to be applied.
* @param rowMutationsIndexMap Map of created RegionAction to the original index for a
* RowMutations within the original list of actions
* @throws IOException
*/
public static void buildRegionActions(final byte[] regionName,
final List<Action> actions, final MultiRequest.Builder multiRequestBuilder,
final RegionAction.Builder regionActionBuilder,
final ClientProtos.Action.Builder actionBuilder,
final MutationProto.Builder mutationBuilder,
long nonceGroup, final Map<Integer, Integer> rowMutationsIndexMap) throws IOException {
regionActionBuilder.clear();
RegionAction.Builder builder = getRegionActionBuilderWithRegion(
regionActionBuilder, regionName);
ClientProtos.CoprocessorServiceCall.Builder cpBuilder = null;
boolean hasNonce = false;
List<Action> rowMutationsList = new ArrayList<>();
for (Action action: actions) {
Row row = action.getAction();
actionBuilder.clear();
actionBuilder.setIndex(action.getOriginalIndex());
mutationBuilder.clear();
if (row instanceof Get) {
Get g = (Get)row;
builder.addAction(actionBuilder.setGet(ProtobufUtil.toGet(g)));
} else if (row instanceof Put) {
builder.addAction(actionBuilder.
setMutation(ProtobufUtil.toMutation(MutationType.PUT, (Put)row, mutationBuilder)));
} else if (row instanceof Delete) {
builder.addAction(actionBuilder.
setMutation(ProtobufUtil.toMutation(MutationType.DELETE, (Delete)row, mutationBuilder)));
} else if (row instanceof Append) {
builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutation(
MutationType.APPEND, (Append)row, mutationBuilder, action.getNonce())));
hasNonce = true;
} else if (row instanceof Increment) {
builder.addAction(actionBuilder.setMutation(ProtobufUtil.toMutation(
MutationType.INCREMENT, (Increment)row, mutationBuilder, action.getNonce())));
hasNonce = true;
} else if (row instanceof RegionCoprocessorServiceExec) {
RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
// DUMB COPY!!! FIX!!! Done to copy from c.g.p.ByteString to shaded ByteString.
org.apache.hbase.thirdparty.com.google.protobuf.ByteString value =
org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations.unsafeWrap(
exec.getRequest().toByteArray());
if (cpBuilder == null) {
cpBuilder = ClientProtos.CoprocessorServiceCall.newBuilder();
} else {
cpBuilder.clear();
}
builder.addAction(actionBuilder.setServiceCall(
cpBuilder.setRow(UnsafeByteOperations.unsafeWrap(exec.getRow()))
.setServiceName(exec.getMethod().getService().getFullName())
.setMethodName(exec.getMethod().getName())
.setRequest(value)));
} else if (row instanceof RowMutations) {
rowMutationsList.add(action);
} else {
throw new DoNotRetryIOException("Multi doesn't support " + row.getClass().getName());
}
}
if (!multiRequestBuilder.hasNonceGroup() && hasNonce) {
multiRequestBuilder.setNonceGroup(nonceGroup);
}
multiRequestBuilder.addRegionAction(builder.build());
// Process RowMutations here. We can not process it in the big loop above because
// it will corrupt the sequence order maintained in cells.
// RowMutations is a set of Puts and/or Deletes all to be applied atomically
// on the one row. We do separate RegionAction for each RowMutations.
// We maintain a map to keep track of this RegionAction and the original Action index.
for (Action action : rowMutationsList) {
RowMutations rms = (RowMutations) action.getAction();
RegionAction.Builder rowMutationsRegionActionBuilder =
RequestConverter.buildRegionAction(regionName, rms);
rowMutationsRegionActionBuilder.setAtomic(true);
// Put it in the multiRequestBuilder
multiRequestBuilder.addRegionAction(rowMutationsRegionActionBuilder.build());
// This rowMutations region action is at (multiRequestBuilder.getRegionActionCount() - 1)
// in the overall multiRequest.
rowMutationsIndexMap.put(multiRequestBuilder.getRegionActionCount() - 1,
action.getOriginalIndex());
}
}
/**
* Create a protocol buffer multirequest with NO data for a list of actions (data is carried
* otherwise than via protobuf). This means it just notes attributes, whether to write the
@ -1357,19 +1197,6 @@ public final class RequestConverter {
return builder.build();
}
/**
* Creates a protocol buffer GetSchemaAlterStatusRequest
*
* @param tableName
* @return a GetSchemaAlterStatusRequest
*/
public static GetSchemaAlterStatusRequest buildGetSchemaAlterStatusRequest(
final TableName tableName) {
GetSchemaAlterStatusRequest.Builder builder = GetSchemaAlterStatusRequest.newBuilder();
builder.setTableName(ProtobufUtil.toProtoTableName((tableName)));
return builder.build();
}
/**
* Creates a protocol buffer GetTableDescriptorsRequest
*
@ -1421,19 +1248,6 @@ public final class RequestConverter {
return builder.build();
}
/**
* Creates a protocol buffer GetTableStateRequest
*
* @param tableName table to get request for
* @return a GetTableStateRequest
*/
public static GetTableStateRequest buildGetTableStateRequest(
final TableName tableName) {
return GetTableStateRequest.newBuilder()
.setTableName(ProtobufUtil.toProtoTableName(tableName))
.build();
}
/**
* Creates a protocol buffer SetTableStateInMetaRequest
* @param state table state to update in Meta