diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java index 38ccf08..ef06dee 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/snapshot/SnapshotManifest.java @@ -28,6 +28,7 @@ import java.util.Map; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import com.google.protobuf.CodedInputStream; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; @@ -65,6 +66,7 @@ public class SnapshotManifest { private static final Log LOG = LogFactory.getLog(SnapshotManifest.class); private static final String DATA_MANIFEST_NAME = "data.manifest"; + private static final String CONF_SNAPSHOT_MANIFEST_SIZE_LIMIT = "snapshot.manifest.size.limit"; private List regionManifests; private SnapshotDescription desc; @@ -74,6 +76,7 @@ public class SnapshotManifest { private final Configuration conf; private final Path workingDir; private final FileSystem fs; + private int manifestSizeLimit; private SnapshotManifest(final Configuration conf, final FileSystem fs, final Path workingDir, final SnapshotDescription desc, @@ -83,6 +86,8 @@ public class SnapshotManifest { this.workingDir = workingDir; this.conf = conf; this.fs = fs; + + this.manifestSizeLimit = conf.getInt(CONF_SNAPSHOT_MANIFEST_SIZE_LIMIT, 67108864); } /** @@ -430,7 +435,11 @@ public class SnapshotManifest { FSDataInputStream in = null; try { in = fs.open(new Path(workingDir, DATA_MANIFEST_NAME)); - return SnapshotDataManifest.parseFrom(in); + + CodedInputStream cin = CodedInputStream.newInstance(in); + cin.setSizeLimit(manifestSizeLimit); + + return SnapshotDataManifest.parseFrom(cin); } catch (FileNotFoundException e) { return null; } finally {