From 7ddae3939e38ee4a910eef63c051c9d470d32629 Mon Sep 17 00:00:00 2001 From: Esteban Gutierrez Date: Tue, 14 Jul 2015 12:53:42 -0700 Subject: [PATCH] HBASE-14076 ResultSerialization and MutationSerialization can throw InvalidProtocolBufferException when serializing a cell larger than 64MB --- .../mapreduce/MutationSerialization.java | 6 +- .../hbase/mapreduce/ResultSerialization.java | 4 +- .../hbase/mapreduce/TestSerialization.java | 129 ++++++++++++++++++ 3 files changed, 136 insertions(+), 3 deletions(-) create mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestSerialization.java diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java index b15b513c605..4d200e8064b 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/MutationSerialization.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.client.Delete; import org.apache.hadoop.hbase.client.Mutation; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto; +import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType; import org.apache.hadoop.io.serializer.Deserializer; import org.apache.hadoop.io.serializer.Serialization; @@ -57,7 +57,9 @@ public class MutationSerialization implements Serialization { @Override public Mutation deserialize(Mutation mutation) throws IOException { - MutationProto proto = MutationProto.parseDelimitedFrom(in); + ClientProtos.MutationProto.Builder builder = ClientProtos.MutationProto.newBuilder(); + ProtobufUtil.mergeDelimitedFrom(builder, in); + ClientProtos.MutationProto proto = builder.build(); return ProtobufUtil.toMutation(proto); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java index ebd366479fc..19b12c523a9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/ResultSerialization.java @@ -125,7 +125,9 @@ public class ResultSerialization extends Configured implements Serialization serializer = serialization.getSerializer(Mutation.class); + Deserializer deserializer = serialization.getDeserializer(Mutation.class); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + ByteArrayInputStream is = null; + try { + serializer.open(os); + serializer.serialize(put); + os.flush(); + is = new ByteArrayInputStream(os.toByteArray()); + deserializer.open(is); + deserializer.deserialize(null); + } catch (InvalidProtocolBufferException e) { + assertTrue("Got InvalidProtocolBufferException in " + name.getMethodName(), + e.getCause() instanceof InvalidProtocolBufferException); + } catch (Exception e) { + fail("Got an invalid exception: " + e); + } + } + @Test + public void testLargeResult() + throws Exception { + Result res = Result.create(new KeyValue[] {new KeyValue(row, family, qualifier, 0L, value)}); + + ResultSerialization serialization = new ResultSerialization(); + Serializer serializer = serialization.getSerializer(Result.class); + Deserializer deserializer = serialization.getDeserializer(Result.class); + ByteArrayOutputStream os = new ByteArrayOutputStream(); + ByteArrayInputStream is = null; + try { + serializer.open(os); + serializer.serialize(res); + os.flush(); + is = new ByteArrayInputStream(os.toByteArray()); + deserializer.open(is); + deserializer.deserialize(null); + } catch (InvalidProtocolBufferException e) { + assertTrue("Got InvalidProtocolBufferException in " + name.getMethodName(), + e.getCause() instanceof InvalidProtocolBufferException); + } catch (Exception e) { + fail("Got an invalid exception: " + e); + } + } + private static String getName() { + return "TestSerialization"; + } +}