TEZ-4363: Bump protobuf dependency to 3.x (#192) (Mark Bathori reviewed by Laszlo Bodor, Aman Raj)

diff --git a/BUILDING.txt b/BUILDING.txt
index 8b89a55..a917bfa 100644
--- a/BUILDING.txt
+++ b/BUILDING.txt
@@ -9,7 +9,7 @@
 * JDK 1.8+
 * Maven 3.1 or later
 * Findbugs 2.0.2 or later (if running findbugs)
-* ProtocolBuffer 2.5.0
+* ProtocolBuffer 3.21.1
 * Internet connection for first build (to fetch all dependencies)
 * Hadoop version should be 2.7.0 or higher.
 
diff --git a/build-tools/docker/Dockerfile b/build-tools/docker/Dockerfile
index f51cc3b..a5f3e53 100644
--- a/build-tools/docker/Dockerfile
+++ b/build-tools/docker/Dockerfile
@@ -95,13 +95,13 @@
 #############
 
 ######
-# Install Google Protobuf 2.5.0
+# Install Google Protobuf 3.21.1
 ######
-FROM tezbase AS protobuf250
+FROM tezbase AS protobuf
 SHELL ["/bin/bash", "-o", "pipefail", "-c"]
 RUN mkdir -p /opt/protobuf-src \
     && curl -L -s -S \
-      https://github.com/google/protobuf/releases/download/v2.5.0/protobuf-2.5.0.tar.gz \
+      https://github.com/protocolbuffers/protobuf/releases/download/v21.1/protobuf-java-3.21.1.tar.gz \
       -o /opt/protobuf.tar.gz \
     && tar xzf /opt/protobuf.tar.gz --strip-components 1 -C /opt/protobuf-src
 WORKDIR /opt/protobuf-src
@@ -261,7 +261,7 @@
 COPY --from=shellcheck /bin/shellcheck /bin/shellcheck
 COPY --from=hadolint /bin/hadolint /bin/hadolint
 COPY --from=buf /usr/local/bin/buf /usr/local/bin/buf
-COPY --from=protobuf250 /opt/protobuf /opt/protobuf
+COPY --from=protobuf /opt/protobuf /opt/protobuf
 
 ENV PROTOBUF_HOME /opt/protobuf
 ENV PROTOC_PATH /opt/protobuf/bin/protoc
diff --git a/pom.xml b/pom.xml
index 5f381cd..2e2cd8c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -63,7 +63,7 @@
     <pig.version>0.13.0</pig.version>
     <jersey.version>1.19</jersey.version>
     <slf4j.version>1.7.36</slf4j.version>
-    <protobuf.version>2.5.0</protobuf.version>
+    <protobuf.version>3.21.1</protobuf.version>
     <roaringbitmap.version>0.7.45</roaringbitmap.version>
     <protoc.path>${env.PROTOC_PATH}</protoc.path>
     <scm.url>scm:git:https://gitbox.apache.org/repos/asf/tez.git</scm.url>
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java
index 001cbf0..bb87fdb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMLaunchedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.history.HistoryEvent;
@@ -90,7 +91,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, null);
+    AMLaunchedProto proto = inputStream.readMessage(AMLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java
index 87daba6..6be528a 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AMStartedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.history.HistoryEvent;
@@ -85,7 +86,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, null);
+    AMStartedProto proto = inputStream.readMessage(AMStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java
index 11528e2..9ec0592 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerLaunchedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -83,7 +84,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    ContainerLaunchedProto proto = inputStream.readMessage(ContainerLaunchedProto.PARSER, null);
+    ContainerLaunchedProto proto =
+        inputStream.readMessage(ContainerLaunchedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java
index 528f629..9360e49 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/ContainerStoppedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -88,7 +89,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    ContainerStoppedProto proto = inputStream.readMessage(ContainerStoppedProto.PARSER, null);
+    ContainerStoppedProto proto =
+        inputStream.readMessage(ContainerStoppedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java
index 241dada..694d947 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGCommitStartedEvent.java
@@ -23,6 +23,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
@@ -76,7 +77,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGCommitStartedProto proto = inputStream.readMessage(DAGCommitStartedProto.PARSER, null);
+    DAGCommitStartedProto proto =
+        inputStream.readMessage(DAGCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java
index 6d1d6cd..3364e1d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGFinishedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.DagTypeConverters;
@@ -129,7 +130,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, null);
+    DAGFinishedProto proto = inputStream.readMessage(DAGFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java
index 4df116a..31a53f7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGInitializedEvent.java
@@ -23,6 +23,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.records.DAGIDAware;
@@ -91,7 +92,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGInitializedProto proto = inputStream.readMessage(DAGInitializedProto.PARSER, null);
+    DAGInitializedProto proto =
+        inputStream.readMessage(DAGInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java
index c87f5ce..b9e3da8 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGKillRequestEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
@@ -76,7 +77,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGKillRequestProto proto = inputStream.readMessage(DAGKillRequestProto.PARSER, null);
+    DAGKillRequestProto proto =
+        inputStream.readMessage(DAGKillRequestProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java
index 9e1f6c4..d49ad37 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGStartedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.app.dag.DAGState;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -81,7 +82,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, null);
+    DAGStartedProto proto = inputStream.readMessage(DAGStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
index bed0f6f..ac9b923 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGSubmittedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.records.DAGIDAware;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -134,7 +135,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, null);
+    DAGSubmittedProto proto = inputStream.readMessage(DAGSubmittedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
index ce2eb72..41b0ed6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.common.TezConverterUtils;
 import org.apache.tez.common.counters.CounterGroup;
 import org.apache.tez.common.counters.TezCounter;
@@ -235,7 +236,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    TaskAttemptFinishedProto proto = inputStream.readMessage(TaskAttemptFinishedProto.PARSER, null);
+    TaskAttemptFinishedProto proto =
+        inputStream.readMessage(TaskAttemptFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java
index 10ba439..1442ff4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptStartedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -98,7 +99,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    TaskAttemptStartedProto proto = inputStream.readMessage(TaskAttemptStartedProto.PARSER, null);
+    TaskAttemptStartedProto proto =
+        inputStream.readMessage(TaskAttemptStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index 2159682..fd0a5fc 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.common.counters.CounterGroup;
 import org.apache.tez.common.counters.TezCounter;
 import org.apache.tez.dag.records.TaskIDAware;
@@ -116,7 +117,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, null);
+    TaskFinishedProto proto = inputStream.readMessage(TaskFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java
index 577aef9..1379e04 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskStartedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -90,7 +91,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, null);
+    TaskStartedProto proto = inputStream.readMessage(TaskStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java
index ca2cafc..cb05bdd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexCommitStartedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
@@ -80,7 +81,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexCommitStartedProto proto = inputStream.readMessage(VertexCommitStartedProto.PARSER, null);
+    VertexCommitStartedProto proto =
+        inputStream.readMessage(VertexCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
index 2f4fac0..d5a1ce5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.EdgeProperty;
 import org.apache.tez.dag.api.VertexLocationHint;
@@ -162,7 +163,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexConfigurationDoneProto proto = inputStream.readMessage(VertexConfigurationDoneProto.PARSER, null);
+    VertexConfigurationDoneProto proto =
+        inputStream.readMessage(VertexConfigurationDoneProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
index 24612a6..4288a2f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.app.dag.impl.ServicePluginInfo;
 import org.apache.tez.dag.records.VertexIDAware;
 import org.slf4j.Logger;
@@ -131,7 +132,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexFinishedProto proto = inputStream.readMessage(VertexFinishedProto.PARSER, null);
+    VertexFinishedProto proto =
+        inputStream.readMessage(VertexFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
index c9d5aae..7b3d0d4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
@@ -101,7 +102,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexGroupCommitFinishedProto proto = inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, null);
+    VertexGroupCommitFinishedProto proto =
+        inputStream.readMessage(VertexGroupCommitFinishedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
index cdd11bc..d615deb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
@@ -24,6 +24,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
@@ -101,7 +102,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexGroupCommitStartedProto proto = inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, null);
+    VertexGroupCommitStartedProto proto =
+        inputStream.readMessage(VertexGroupCommitStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
index a019fe3..285c520 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
@@ -25,6 +25,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.InputInitializerDescriptor;
@@ -158,7 +159,8 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexInitializedProto proto = inputStream.readMessage(VertexInitializedProto.PARSER, null);
+    VertexInitializedProto proto =
+        inputStream.readMessage(VertexInitializedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java
index cdd73b4..53f0097 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexStartedEvent.java
@@ -22,6 +22,7 @@
 
 import com.google.protobuf.CodedInputStream;
 import com.google.protobuf.CodedOutputStream;
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -81,7 +82,7 @@
 
   @Override
   public void fromProtoStream(CodedInputStream inputStream) throws IOException {
-    VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, null);
+    VertexStartedProto proto = inputStream.readMessage(VertexStartedProto.PARSER, ExtensionRegistry.getEmptyRegistry());
     if (proto == null) {
       throw new IOException("No data found in stream");
     }
diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java
index 34e4701..df57436 100644
--- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java
+++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWritable.java
@@ -25,6 +25,7 @@
 import java.io.InputStream;
 import java.io.OutputStream;
 
+import com.google.protobuf.ExtensionRegistry;
 import org.apache.hadoop.io.Writable;
 
 import com.google.protobuf.CodedInputStream;
@@ -96,6 +97,6 @@
       cin.setSizeLimit(Integer.MAX_VALUE);
     }
     din.in = in;
-    message = cin.readMessage(parser, null);
+    message = cin.readMessage(parser, ExtensionRegistry.getEmptyRegistry());
   }
 }