HBASE-16759 Avoid ByteString.copyFrom usage wherever possible.

This commit is contained in:
anoopsamjohn 2016-10-06 00:27:34 +05:30
parent 1f1a13f2e2
commit 06758bf630
10 changed files with 27 additions and 26 deletions

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.client.replication;
import org.apache.commons.lang.StringUtils; import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.TableName;
@ -300,8 +301,8 @@ public final class ReplicationSerDeHelper {
for (Map.Entry<byte[], byte[]> entry : peerConfig.getPeerData().entrySet()) { for (Map.Entry<byte[], byte[]> entry : peerConfig.getPeerData().entrySet()) {
builder.addData(HBaseProtos.BytesBytesPair.newBuilder() builder.addData(HBaseProtos.BytesBytesPair.newBuilder()
.setFirst(ByteString.copyFrom(entry.getKey())) .setFirst(UnsafeByteOperations.unsafeWrap(entry.getKey()))
.setSecond(ByteString.copyFrom(entry.getValue())) .setSecond(UnsafeByteOperations.unsafeWrap(entry.getValue()))
.build()); .build());
} }

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter; import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest; import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceRequest;
@ -97,13 +98,13 @@ public final class CoprocessorRpcUtils {
private static CoprocessorServiceCall getCoprocessorServiceCall( private static CoprocessorServiceCall getCoprocessorServiceCall(
final Descriptors.MethodDescriptor method, final Message request, final byte [] row) { final Descriptors.MethodDescriptor method, final Message request, final byte [] row) {
return CoprocessorServiceCall.newBuilder() return CoprocessorServiceCall.newBuilder()
.setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFrom(row)) .setRow(org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations.unsafeWrap(row))
.setServiceName(CoprocessorRpcUtils.getServiceName(method.getService())) .setServiceName(CoprocessorRpcUtils.getServiceName(method.getService()))
.setMethodName(method.getName()) .setMethodName(method.getName())
// TODO!!!!! Come back here after!!!!! This is a double copy of the request if I read // TODO!!!!! Come back here after!!!!! This is a double copy of the request if I read
// it right copying from non-shaded to shaded version!!!!!! FIXXXXX!!!!! // it right copying from non-shaded to shaded version!!!!!! FIXXXXX!!!!!
.setRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString. .setRequest(org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations.
copyFrom(request.toByteArray())).build(); unsafeWrap(request.toByteArray())).build();
} }
public static MethodDescriptor getMethodDescriptor(final String methodName, public static MethodDescriptor getMethodDescriptor(final String methodName,

View File

@ -615,7 +615,7 @@ public final class RequestConverter {
RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row; RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
// DUMB COPY!!! FIX!!! Done to copy from c.g.p.ByteString to shaded ByteString. // DUMB COPY!!! FIX!!! Done to copy from c.g.p.ByteString to shaded ByteString.
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value =
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFrom( org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations.unsafeWrap(
exec.getRequest().toByteArray()); exec.getRequest().toByteArray());
regionActionBuilder.addAction(actionBuilder.setServiceCall( regionActionBuilder.addAction(actionBuilder.setServiceCall(
ClientProtos.CoprocessorServiceCall.newBuilder() ClientProtos.CoprocessorServiceCall.newBuilder()
@ -698,7 +698,7 @@ public final class RequestConverter {
RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row; RegionCoprocessorServiceExec exec = (RegionCoprocessorServiceExec) row;
// DUMB COPY!!! FIX!!! Done to copy from c.g.p.ByteString to shaded ByteString. // DUMB COPY!!! FIX!!! Done to copy from c.g.p.ByteString to shaded ByteString.
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value = org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString value =
org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString.copyFrom( org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations.unsafeWrap(
exec.getRequest().toByteArray()); exec.getRequest().toByteArray());
builder.addAction(actionBuilder.setServiceCall( builder.addAction(actionBuilder.setServiceCall(
ClientProtos.CoprocessorServiceCall.newBuilder() ClientProtos.CoprocessorServiceCall.newBuilder()

View File

@ -28,7 +28,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ProcedureInfo;
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.exceptions.TimeoutIOException; import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
@ -36,6 +35,7 @@ import org.apache.hadoop.hbase.procedure2.util.StringUtils;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState; import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos.ProcedureState;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.NonceKey; import org.apache.hadoop.hbase.util.NonceKey;
@ -815,7 +815,7 @@ public abstract class Procedure<TEnvironment> implements Comparable<Procedure> {
byte[] result = proc.getResult(); byte[] result = proc.getResult();
if (result != null) { if (result != null) {
builder.setResult(ByteString.copyFrom(result)); builder.setResult(UnsafeByteOperations.unsafeWrap(result));
} }
ByteString.Output stateStream = ByteString.newOutput(); ByteString.Output stateStream = ByteString.newOutput();

View File

@ -652,7 +652,7 @@ public class MasterRpcServices extends RSRpcServices
ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder(); ExecProcedureResponse.Builder builder = ExecProcedureResponse.newBuilder();
// set return data if available // set return data if available
if (data != null) { if (data != null) {
builder.setReturnData(ByteString.copyFrom(data)); builder.setReturnData(UnsafeByteOperations.unsafeWrap(data));
} }
return builder.build(); return builder.build();
} catch (IOException e) { } catch (IOException e) {

View File

@ -82,6 +82,7 @@ import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/** /**
* The ServerManager class manages info about region servers. * The ServerManager class manages info about region servers.
@ -475,7 +476,7 @@ public class ServerManager {
if (storeFlushedSequenceId != null) { if (storeFlushedSequenceId != null) {
for (Map.Entry<byte[], Long> entry : storeFlushedSequenceId.entrySet()) { for (Map.Entry<byte[], Long> entry : storeFlushedSequenceId.entrySet()) {
builder.addStoreSequenceId(StoreSequenceId.newBuilder() builder.addStoreSequenceId(StoreSequenceId.newBuilder()
.setFamilyName(ByteString.copyFrom(entry.getKey())) .setFamilyName(UnsafeByteOperations.unsafeWrap(entry.getKey()))
.setSequenceId(entry.getValue().longValue()).build()); .setSequenceId(entry.getValue().longValue()).build());
} }
} }

View File

@ -141,7 +141,6 @@ import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit; import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.regionserver.wal.WALUtil; import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat; import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations; import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
@ -1741,8 +1740,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// give us a sequence id that is for sure flushed. We want edit replay to start after this // give us a sequence id that is for sure flushed. We want edit replay to start after this
// sequence id in this region. If NO_SEQNUM, use the regions maximum flush id. // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id.
long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1; long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId. regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.newBuilder()
newBuilder().setFamilyName(ByteString.copyFrom(familyName)).setSequenceId(csid).build()); .setFamilyName(UnsafeByteOperations.unsafeWrap(familyName)).setSequenceId(csid).build());
} }
return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId()); return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
} }

View File

@ -80,7 +80,6 @@ import org.apache.hadoop.hbase.exceptions.OutOfOrderScannerNextException;
import org.apache.hadoop.hbase.exceptions.ScannerResetException; import org.apache.hadoop.hbase.exceptions.ScannerResetException;
import org.apache.hadoop.hbase.filter.ByteArrayComparable; import org.apache.hadoop.hbase.filter.ByteArrayComparable;
import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler; import org.apache.hadoop.hbase.ipc.HBaseRPCErrorHandler;
import org.apache.hadoop.hbase.ipc.HBaseRpcController; import org.apache.hadoop.hbase.ipc.HBaseRpcController;
import org.apache.hadoop.hbase.ipc.PriorityFunction; import org.apache.hadoop.hbase.ipc.PriorityFunction;
@ -193,13 +192,12 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog; import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message; import org.apache.hadoop.hbase.shaded.com.google.protobuf.Message;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController; import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat; import org.apache.hadoop.hbase.shaded.com.google.protobuf.TextFormat;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/** /**
* Implements the regionserver RPC services. * Implements the regionserver RPC services.
@ -362,10 +360,6 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private void incNextCallSeq() { private void incNextCallSeq() {
nextCallSeq.incrementAndGet(); nextCallSeq.incrementAndGet();
} }
private void rollbackNextCallSeq() {
nextCallSeq.decrementAndGet();
}
} }
/** /**
@ -789,7 +783,7 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
serviceResultBuilder.getValueBuilder() serviceResultBuilder.getValueBuilder()
.setName(result.getClass().getName()) .setName(result.getClass().getName())
// TODO: Copy!!! // TODO: Copy!!!
.setValue(ByteString.copyFrom(result.toByteArray())))); .setValue(UnsafeByteOperations.unsafeWrap(result.toByteArray()))));
} catch (IOException ioe) { } catch (IOException ioe) {
rpcServer.getMetrics().exception(ioe); rpcServer.getMetrics().exception(ioe);
resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe)); resultOrExceptionBuilder.setException(ResponseConverter.buildException(ioe));

View File

@ -140,12 +140,16 @@ public class WALCellCodec implements Codec {
// an array of dictionaries. // an array of dictionaries.
static class BaosAndCompressor extends ByteArrayOutputStream implements ByteStringCompressor { static class BaosAndCompressor extends ByteArrayOutputStream implements ByteStringCompressor {
public ByteString toByteString() { public ByteString toByteString() {
// We need this copy to create the ByteString as the byte[] 'buf' is not immutable. We reuse
// them.
return ByteString.copyFrom(this.buf, 0, this.count); return ByteString.copyFrom(this.buf, 0, this.count);
} }
@Override @Override
public ByteString compress(byte[] data, Dictionary dict) throws IOException { public ByteString compress(byte[] data, Dictionary dict) throws IOException {
writeCompressed(data, dict); writeCompressed(data, dict);
// We need this copy to create the ByteString as the byte[] 'buf' is not immutable. We reuse
// them.
ByteString result = ByteString.copyFrom(this.buf, 0, this.count); ByteString result = ByteString.copyFrom(this.buf, 0, this.count);
reset(); // Only resets the count - we reuse the byte array. reset(); // Only resets the count - we reuse the byte array.
return result; return result;

View File

@ -46,12 +46,12 @@ import org.apache.hadoop.hbase.regionserver.SequenceId;
// imports for things that haven't moved from regionserver.wal yet. // imports for things that haven't moved from regionserver.wal yet.
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext; import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec; import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString; import org.apache.hadoop.hbase.shaded.com.google.protobuf.ByteString;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
/** /**
* A Key for an entry in the WAL. * A Key for an entry in the WAL.
@ -650,8 +650,8 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
WALCellCodec.ByteStringCompressor compressor) throws IOException { WALCellCodec.ByteStringCompressor compressor) throws IOException {
WALProtos.WALKey.Builder builder = WALProtos.WALKey.newBuilder(); WALProtos.WALKey.Builder builder = WALProtos.WALKey.newBuilder();
if (compressionContext == null) { if (compressionContext == null) {
builder.setEncodedRegionName(ByteString.copyFrom(this.encodedRegionName)); builder.setEncodedRegionName(UnsafeByteOperations.unsafeWrap(this.encodedRegionName));
builder.setTableName(ByteString.copyFrom(this.tablename.getName())); builder.setTableName(UnsafeByteOperations.unsafeWrap(this.tablename.getName()));
} else { } else {
builder.setEncodedRegionName(compressor.compress(this.encodedRegionName, builder.setEncodedRegionName(compressor.compress(this.encodedRegionName,
compressionContext.regionDict)); compressionContext.regionDict));
@ -677,7 +677,8 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
} }
if (replicationScope != null) { if (replicationScope != null) {
for (Map.Entry<byte[], Integer> e : replicationScope.entrySet()) { for (Map.Entry<byte[], Integer> e : replicationScope.entrySet()) {
ByteString family = (compressionContext == null) ? ByteString.copyFrom(e.getKey()) ByteString family = (compressionContext == null)
? UnsafeByteOperations.unsafeWrap(e.getKey())
: compressor.compress(e.getKey(), compressionContext.familyDict); : compressor.compress(e.getKey(), compressionContext.familyDict);
builder.addScopes(FamilyScope.newBuilder() builder.addScopes(FamilyScope.newBuilder()
.setFamily(family).setScopeType(ScopeType.valueOf(e.getValue()))); .setFamily(family).setScopeType(ScopeType.valueOf(e.getValue())));