diff --git a/.codespellrc b/.codespellrc
new file mode 100644
index 0000000000..4fc66cab68
--- /dev/null
+++ b/.codespellrc
@@ -0,0 +1,21 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied. See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+[codespell]
+ignore-words-list = thirdparty
diff --git a/pom.xml b/pom.xml
index d576ddbb46..4472bc85c6 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,6 +77,7 @@
1.15.0
32.0.1-jre
3.4.2
+ 1.4.0
1.19.4
1.5.4
3.0.0
@@ -101,6 +102,7 @@
${project.build.directory}/logs
1.7.9
true
+ 1.5.3
${scm.url}
@@ -773,6 +775,11 @@
protobuf-java
${protobuf.version}
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
+ ${hadoop-thirdparty.version}
+
com.google.guava
guava
@@ -1100,6 +1107,51 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+ ${maven-replacer-plugin.version}
+
+
+ replace-generated-sources
+ generate-sources
+
+ replace
+
+
+ ${project.basedir}
+
+ target/generated-sources/**/*.java
+
+
+
+ ([^\.])com.google.protobuf
+ $1org.apache.hadoop.thirdparty.protobuf
+
+
+
+
+
+ replace-generated-test-sources
+ generate-sources
+
+ replace
+
+
+ ${project.basedir}
+
+ target/generated-test-sources/**/*.java
+
+
+
+ ([^\.])com.google.protobuf
+ $1org.apache.hadoop.thirdparty.protobuf
+
+
+
+
+
+
diff --git a/tez-api/pom.xml b/tez-api/pom.xml
index a2e449f1e6..fc83710737 100644
--- a/tez-api/pom.xml
+++ b/tez-api/pom.xml
@@ -82,8 +82,8 @@
commons-collections4
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
junit
@@ -183,6 +183,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java
index dac20edc08..a255c1f633 100644
--- a/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/FrameworkClient.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
@@ -49,8 +50,6 @@
import org.apache.tez.frameworkplugins.FrameworkUtils;
import org.apache.tez.frameworkplugins.yarn.YarnClientFrameworkService;
-import com.google.protobuf.ServiceException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 8c16f6be8c..df0b0f9e60 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -44,6 +44,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -80,7 +81,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.google.protobuf.ServiceException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index fa0ea657f3..fc9b2c07dd 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -56,7 +56,7 @@
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
@@ -982,7 +982,7 @@ public static DAGClientAMProtocolBlockingPB getAMProxy(final Configuration conf,
proxy = userUgi.doAs(new PrivilegedExceptionAction() {
@Override
public DAGClientAMProtocolBlockingPB run() throws IOException {
- RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class, ProtobufRpcEngine.class);
+ RPC.setProtocolEngine(conf, DAGClientAMProtocolBlockingPB.class, ProtobufRpcEngine2.class);
return (DAGClientAMProtocolBlockingPB) RPC.getProxy(DAGClientAMProtocolBlockingPB.class,
0, serviceAddr, conf);
}
diff --git a/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java b/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java
index ab265f6ea8..d254aab67e 100644
--- a/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java
+++ b/tez-api/src/main/java/org/apache/tez/common/RPCUtil.java
@@ -23,13 +23,12 @@
import java.lang.reflect.InvocationTargetException;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.tez.dag.api.DAGNotRunningException;
import org.apache.tez.dag.api.NoCurrentDAGException;
import org.apache.tez.dag.api.SessionNotRunning;
import org.apache.tez.dag.api.TezException;
-import com.google.protobuf.ServiceException;
-
public final class RPCUtil {
private RPCUtil() {}
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java
index f376ba2384..d178ffd2ca 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezCommonUtils.java
@@ -44,6 +44,7 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.client.TezClient;
import org.apache.tez.common.security.JobTokenIdentifier;
@@ -51,8 +52,6 @@
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.TezUncheckedException;
-import com.google.protobuf.ByteString;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java
index 9797165145..ff268633f5 100644
--- a/tez-api/src/main/java/org/apache/tez/common/TezUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/common/TezUtils.java
@@ -28,15 +28,14 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
import org.apache.tez.client.TezClientUtils;
import org.apache.tez.dag.api.TezUncheckedException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.records.DAGProtos;
import org.apache.tez.runtime.api.TaskContext;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedInputStream;
-
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
import org.slf4j.Logger;
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
index 1d292e7ad6..fbfe543a78 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DagTypeConverters.java
@@ -38,6 +38,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.ByteString.Output;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
@@ -86,8 +88,6 @@
import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ByteString.Output;
@Private
public final class DagTypeConverters {
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
index 827d333371..19c55fc9e8 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientRPCImpl.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
@@ -50,7 +51,6 @@
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGRequestProto;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index 68af185cc3..d1603c16a3 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -55,6 +55,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
@@ -96,8 +98,6 @@
import org.apache.tez.serviceplugins.api.ServicePluginsDescriptor;
import com.google.common.collect.Maps;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
diff --git a/tez-api/src/test/java/org/apache/tez/common/TestRPCUtil.java b/tez-api/src/test/java/org/apache/tez/common/TestRPCUtil.java
index 507d963b77..83feaf1827 100644
--- a/tez-api/src/test/java/org/apache/tez/common/TestRPCUtil.java
+++ b/tez-api/src/test/java/org/apache/tez/common/TestRPCUtil.java
@@ -22,11 +22,10 @@
import java.io.IOException;
import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.tez.dag.api.SessionNotRunning;
import org.apache.tez.dag.api.TezException;
-import com.google.protobuf.ServiceException;
-
import org.junit.Assert;
import org.junit.Test;
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
index 6e979972a7..c846b199c8 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/client/rpc/TestDAGClient.java
@@ -40,6 +40,8 @@
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationReport;
import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -74,9 +76,6 @@
import org.apache.tez.dag.api.records.DAGProtos.VertexStatusProto;
import org.apache.tez.dag.api.records.DAGProtos.VertexStatusStateProto;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-common/pom.xml b/tez-common/pom.xml
index e9d8a907b8..49bfa05b6b 100644
--- a/tez-common/pom.xml
+++ b/tez-common/pom.xml
@@ -34,8 +34,8 @@
guava
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
org.apache.hadoop
diff --git a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
index cb7581981b..525a6b7513 100644
--- a/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
+++ b/tez-common/src/main/java/org/apache/tez/common/TezUtilsInternal.java
@@ -42,6 +42,9 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.Descriptors;
+import org.apache.hadoop.thirdparty.protobuf.TextFormat;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.log4j.Appender;
@@ -64,10 +67,6 @@
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.util.StopWatch;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.Descriptors;
-import com.google.protobuf.TextFormat;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java
index 3b9b7cae6e..d5cd7c4dd2 100644
--- a/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java
+++ b/tez-common/src/test/java/org/apache/tez/common/TestTezUtils.java
@@ -30,6 +30,7 @@
import java.util.Random;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.client.TezClientUtils;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.TezConstants;
@@ -40,8 +41,6 @@
import org.apache.tez.serviceplugins.api.TaskCommunicatorDescriptor;
import org.apache.tez.serviceplugins.api.TaskSchedulerDescriptor;
-import com.google.protobuf.ByteString;
-
import org.codehaus.jettison.json.JSONException;
import org.codehaus.jettison.json.JSONObject;
import org.junit.Assert;
diff --git a/tez-dag/pom.xml b/tez-dag/pom.xml
index bf79df2780..0bc12da4bf 100644
--- a/tez-dag/pom.xml
+++ b/tez-dag/pom.xml
@@ -99,8 +99,8 @@
slf4j-api
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
org.apache.commons
@@ -233,6 +233,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 56d9f7ad81..d29295064f 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -32,6 +32,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -72,7 +73,6 @@
import org.apache.tez.dag.app.dag.DAG;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ServiceException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java
index 4c5a0039e6..fa29d80e39 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/DAGClientServer.java
@@ -25,12 +25,13 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.RPC.Server;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.thirdparty.protobuf.BlockingService;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
import org.apache.tez.dag.api.TezConfiguration;
@@ -40,8 +41,6 @@
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.DAGClientAMProtocol;
import org.apache.tez.dag.app.security.authorize.TezAMPolicyProvider;
-import com.google.protobuf.BlockingService;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -128,7 +127,7 @@ public void setClientAMSecretKey(ByteBuffer key) {
private Server createServer(Class> pbProtocol, InetSocketAddress addr, Configuration conf,
int numHandlers,
BlockingService blockingService, String portRangeConfig) throws IOException {
- RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
+ RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine2.class);
RPC.Server server = new RPC.Builder(conf).setProtocol(pbProtocol)
.setInstance(blockingService).setBindAddress(addr.getHostString())
.setPort(addr.getPort()).setNumHandlers(numHandlers).setVerbose(false)
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
index 136d4c80eb..c0838daa8f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
@@ -27,6 +27,9 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.client.TezAppMasterStatus;
import org.apache.tez.dag.api.DagTypeConverters;
@@ -54,10 +57,6 @@
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolRPC.TryKillDAGResponseProto;
import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProtocolBlockingPB {
DAGClientHandler real;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
index 04812dff17..ca964b758e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
import org.apache.hadoop.util.functional.FutureIO;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.LocalResource;
@@ -76,7 +77,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.CodedInputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
index 411e3cd0d4..421df2d346 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TezTaskCommunicatorImpl.java
@@ -25,13 +25,14 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.ipc.ProtocolSignature;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.thirdparty.protobuf.BlockingService;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -53,6 +54,9 @@
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolBlockingPB;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolPBServerImpl;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos;
import org.apache.tez.serviceplugins.api.ContainerEndReason;
import org.apache.tez.serviceplugins.api.TaskAttemptEndReason;
import org.apache.tez.serviceplugins.api.TaskCommunicator;
@@ -150,11 +154,19 @@ protected void startRpcServer() {
JobTokenSecretManager jobTokenSecretManager = new JobTokenSecretManager(conf);
jobTokenSecretManager.addTokenForJob(tokenIdentifier, sessionToken);
+ RPC.setProtocolEngine(
+ conf, TezTaskUmbilicalProtocolBlockingPB.class, ProtobufRpcEngine2.class);
+ TezTaskUmbilicalProtocolBlockingPB service =
+ new TezTaskUmbilicalProtocolPBServerImpl(taskUmbilical);
+ BlockingService umbilicalService =
+ TezTaskUmbilicalProtocolProtos.TezTaskUmbilicalProtocol.newReflectiveBlockingService(
+ service);
+
server = new RPC.Builder(conf)
- .setProtocol(TezTaskUmbilicalProtocol.class)
+ .setProtocol(TezTaskUmbilicalProtocolBlockingPB.class)
.setBindAddress("0.0.0.0")
.setPort(0)
- .setInstance(taskUmbilical)
+ .setInstance(umbilicalService)
.setNumHandlers(
conf.getInt(TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT,
TezConfiguration.TEZ_AM_TASK_LISTENER_THREAD_COUNT_DEFAULT))
@@ -387,21 +399,6 @@ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOExce
containerInfo.usedMemory = request.getUsedMemory();
return response;
}
-
-
- // TODO Remove this method once we move to the Protobuf RPC engine
- @Override
- public long getProtocolVersion(String protocol, long clientVersion) throws IOException {
- return versionID;
- }
-
- // TODO Remove this method once we move to the Protobuf RPC engine
- @Override
- public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
- int clientMethodsHash) throws IOException {
- return ProtocolSignature.getProtocolSignature(this, protocol,
- clientVersion, clientMethodsHash);
- }
}
private ContainerTask getContainerTask(ContainerId containerId) throws IOException {
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 392739e292..3f05239ddb 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -50,6 +50,8 @@
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.service.ServiceOperations;
import org.apache.hadoop.service.ServiceStateException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
@@ -204,8 +206,6 @@
import com.google.common.util.concurrent.FutureCallback;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/security/authorize/TezAMPolicyProvider.java b/tez-dag/src/main/java/org/apache/tez/dag/app/security/authorize/TezAMPolicyProvider.java
index b2353de678..6fc6e67e99 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/security/authorize/TezAMPolicyProvider.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/security/authorize/TezAMPolicyProvider.java
@@ -21,9 +21,9 @@
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.authorize.Service;
-import org.apache.tez.common.TezTaskUmbilicalProtocol;
import org.apache.tez.dag.api.TezConstants;
import org.apache.tez.dag.api.client.rpc.DAGClientAMProtocolBlockingPB;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolBlockingPB;
/**
* {@link PolicyProvider} for YARN Tez client protocols.
@@ -36,7 +36,7 @@ public class TezAMPolicyProvider extends PolicyProvider {
new Service[] {
new Service(
TezConstants.TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_TASK_UMBILICAL,
- TezTaskUmbilicalProtocol.class),
+ TezTaskUmbilicalProtocolBlockingPB.class),
new Service(
TezConstants.TEZ_AM_SECURITY_SERVICE_AUTHORIZATION_CLIENT,
DAGClientAMProtocolBlockingPB.class)
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java
index ef7f8a0cb8..a04d10ff8f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEvent.java
@@ -20,8 +20,8 @@
import java.io.IOException;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
public interface HistoryEvent {
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 4b0f314275..9240398cc7 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
@@ -20,16 +20,15 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.recovery.records.RecoveryProtos.AMLaunchedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class AMLaunchedEvent implements HistoryEvent {
private ApplicationAttemptId applicationAttemptId;
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 cac72fccf7..6c920853e0 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
@@ -20,16 +20,15 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.recovery.records.RecoveryProtos.AMStartedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class AMStartedEvent implements HistoryEvent {
private ApplicationAttemptId applicationAttemptId;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java
index 5da61e7957..d2bf0558ca 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/AppLaunchedEvent.java
@@ -21,14 +21,13 @@
import java.io.IOException;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.common.VersionInfo;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-
public class AppLaunchedEvent implements HistoryEvent {
private ApplicationId applicationId;
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 c40ea58a19..fb2e94b18f 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.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;
@@ -27,10 +30,6 @@
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.recovery.records.RecoveryProtos.ContainerLaunchedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class ContainerLaunchedEvent implements HistoryEvent {
private ContainerId containerId;
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 85c871f517..3cbb0e5261 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.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;
@@ -27,10 +30,6 @@
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.recovery.records.RecoveryProtos.ContainerStoppedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class ContainerStoppedEvent implements HistoryEvent {
private ContainerId containerId;
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 33511d33df..94f68e7c7c 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
@@ -21,6 +21,9 @@
import java.io.IOException;
import java.io.OutputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.history.SummaryEvent;
@@ -29,10 +32,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import org.apache.tez.dag.utils.ProtoUtils;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class DAGCommitStartedEvent implements HistoryEvent, SummaryEvent {
private TezDAGID dagID;
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 f7e3a57f34..06831da127 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
@@ -22,6 +22,10 @@
import java.io.OutputStream;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.api.DagTypeConverters;
@@ -37,10 +41,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import com.google.common.primitives.Ints;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
public class DAGFinishedEvent implements HistoryEvent, SummaryEvent, DAGIDAware {
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 ef78bc4665..4450952444 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
@@ -21,6 +21,9 @@
import java.io.IOException;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.records.DAGIDAware;
@@ -29,10 +32,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos;
import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGInitializedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class DAGInitializedEvent implements HistoryEvent, DAGIDAware {
private TezDAGID dagID;
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 a77098ce60..7e018635ef 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
@@ -20,6 +20,9 @@
import java.io.IOException;
import java.io.OutputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.history.SummaryEvent;
@@ -29,10 +32,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import org.apache.tez.dag.utils.ProtoUtils;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class DAGKillRequestEvent implements HistoryEvent, SummaryEvent {
private TezDAGID dagID;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java
index dcd9103455..4c11f01c4f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/DAGRecoveredEvent.java
@@ -20,15 +20,14 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.tez.dag.app.dag.DAGState;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.records.TezDAGID;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-
public class DAGRecoveredEvent implements HistoryEvent {
private final ApplicationAttemptId applicationAttemptId;
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 5aad8bc4b6..1594bf0a73 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.app.dag.DAGState;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
@@ -27,10 +30,6 @@
import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.dag.recovery.records.RecoveryProtos.DAGStartedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class DAGStartedEvent implements HistoryEvent, DAGIDAware {
private TezDAGID dagID;
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 12ea0af6ca..8e07439e46 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
@@ -23,6 +23,9 @@
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.util.ConverterUtils;
@@ -38,10 +41,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import org.apache.tez.dag.utils.ProtoUtils;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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 9bac94d9fb..9b089d5cf2 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
@@ -23,6 +23,9 @@
import javax.annotation.Nullable;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.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;
@@ -46,9 +49,6 @@
import org.apache.tez.runtime.api.impl.TezEvent;
import com.google.common.collect.Lists;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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 06d1a36e73..5b7ec40ac4 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.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;
@@ -29,10 +32,6 @@
import org.apache.tez.dag.records.TezTaskAttemptID;
import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptStartedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class TaskAttemptStartedEvent implements HistoryEvent, TaskAttemptIDAware {
private TezTaskAttemptID taskAttemptId;
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 c95afcd8ae..cb600383a1 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.common.counters.CounterGroup;
import org.apache.tez.common.counters.TezCounter;
import org.apache.tez.common.counters.TezCounters;
@@ -31,10 +34,6 @@
import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskFinishedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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 d9b0f54fe3..88f1598d25 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.api.oldrecords.TaskState;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
@@ -27,10 +30,6 @@
import org.apache.tez.dag.records.TezTaskID;
import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskStartedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class TaskStartedEvent implements HistoryEvent, TaskIDAware {
private TezTaskID taskID;
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 8d79706be9..2d1991266d 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
@@ -22,6 +22,10 @@
import java.io.OutputStream;
import java.nio.charset.Charset;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.history.SummaryEvent;
@@ -30,11 +34,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexCommitStartedProto;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class VertexCommitStartedEvent implements HistoryEvent, SummaryEvent {
private TezVertexID vertexID;
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 2362af536c..9931893751 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
@@ -22,6 +22,9 @@
import java.util.Map;
import java.util.Map.Entry;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.api.DagTypeConverters;
import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.VertexLocationHint;
@@ -35,9 +38,6 @@
import org.apache.tez.runtime.api.InputSpecUpdate;
import com.google.common.collect.Maps;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
public class VertexConfigurationDoneEvent implements HistoryEvent, VertexIDAware {
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 1cd2eb8496..e5cda23a72 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
@@ -22,6 +22,9 @@
import java.io.OutputStream;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.common.counters.TezCounters;
import org.apache.tez.dag.app.dag.VertexState;
import org.apache.tez.dag.app.dag.impl.ServicePluginInfo;
@@ -36,10 +39,6 @@
import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishStateProto;
import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
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 6d5d2ca324..cc8878dd4a 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
@@ -22,6 +22,9 @@
import java.io.OutputStream;
import java.util.Collection;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.history.SummaryEvent;
@@ -33,9 +36,6 @@
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
public class VertexGroupCommitFinishedEvent implements HistoryEvent, SummaryEvent {
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 aa40c5d7b8..9e31e3639a 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
@@ -22,6 +22,9 @@
import java.io.OutputStream;
import java.util.Collection;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.history.SummaryEvent;
@@ -33,9 +36,6 @@
import com.google.common.base.Function;
import com.google.common.collect.Collections2;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
public class VertexGroupCommitStartedEvent implements HistoryEvent, SummaryEvent {
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 81f20f0bdd..4b02b93dfe 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
@@ -23,6 +23,9 @@
import java.util.List;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.api.DagTypeConverters;
import org.apache.tez.dag.api.InputDescriptor;
import org.apache.tez.dag.api.InputInitializerDescriptor;
@@ -40,9 +43,6 @@
import org.apache.tez.runtime.api.impl.TezEvent;
import com.google.common.collect.Lists;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
public class VertexInitializedEvent implements HistoryEvent, VertexIDAware {
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 a8bc347f94..e496b7b19e 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
@@ -20,6 +20,9 @@
import java.io.IOException;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
import org.apache.tez.dag.app.dag.VertexState;
import org.apache.tez.dag.history.HistoryEvent;
import org.apache.tez.dag.history.HistoryEventType;
@@ -27,10 +30,6 @@
import org.apache.tez.dag.records.VertexIDAware;
import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexStartedProto;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-
public class VertexStartedEvent implements HistoryEvent, VertexIDAware {
private TezVertexID vertexID;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
index b412590ed2..90674a2b3e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
@@ -33,6 +33,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.dag.api.TezConfiguration;
@@ -45,7 +46,6 @@
import org.apache.tez.dag.records.TezDAGID;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.CodedOutputStream;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java
index e179b4fd9b..5a241aaf3b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/utils/ProtoUtils.java
@@ -18,12 +18,11 @@
package org.apache.tez.dag.utils;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.dag.history.HistoryEventType;
import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.dag.recovery.records.RecoveryProtos;
-import com.google.protobuf.ByteString;
-
public final class ProtoUtils {
private ProtoUtils() {}
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
index fe33f20f0f..f43a6701e8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestDAGAppMaster.java
@@ -53,6 +53,7 @@
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.test.LambdaTestUtils;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -87,7 +88,6 @@
import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap;
-import com.google.protobuf.ByteString;
import org.junit.After;
import org.junit.Assert;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index 7a8b39a445..9fdb4f7ef8 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -46,6 +46,7 @@
import org.apache.commons.lang3.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
@@ -157,7 +158,6 @@
import com.google.common.collect.Lists;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.protobuf.ByteString;
import org.junit.After;
import org.junit.Assert;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index 833586e57f..7fc8dd396e 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -63,6 +63,7 @@
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Container;
@@ -231,7 +232,6 @@
import com.google.common.collect.Maps;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.common.util.concurrent.ListeningExecutorService;
-import com.google.protobuf.ByteString;
import org.junit.After;
import org.junit.Assert;
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 6b64df2097..29f4d1deb7 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -32,6 +32,8 @@
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
@@ -73,8 +75,6 @@
import org.apache.tez.runtime.api.impl.TezEvent;
import com.google.common.collect.Lists;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
import org.junit.Assert;
import org.junit.Test;
diff --git a/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java b/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java
index 96da31e141..0d6a3be85a 100644
--- a/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java
+++ b/tez-dag/src/test/java/org/apache/tez/test/GraceShuffleVertexManagerForTest.java
@@ -23,6 +23,7 @@
import java.util.Objects;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.TezUncheckedException;
@@ -31,8 +32,6 @@
import org.apache.tez.dag.api.event.VertexStateUpdate;
import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager;
-import com.google.protobuf.ByteString;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-ext-service-tests/pom.xml b/tez-ext-service-tests/pom.xml
index ef1d8c29d1..d5892a04d6 100644
--- a/tez-ext-service-tests/pom.xml
+++ b/tez-ext-service-tests/pom.xml
@@ -38,6 +38,10 @@
com.google.guava
guava
+
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
+
org.apache.hadoop
hadoop-common
@@ -185,6 +189,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
index df7fcbcae7..7305bb08b4 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/TezTestServiceCommunicator.java
@@ -22,6 +22,7 @@
import java.util.concurrent.Executors;
import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.thirdparty.protobuf.Message;
import org.apache.tez.common.GuavaShim;
import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
import org.apache.tez.service.impl.TezTestServiceProtocolClientImpl;
@@ -36,7 +37,6 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.google.protobuf.Message;
public class TezTestServiceCommunicator extends AbstractService {
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
index 8ea112ee59..752d6764ec 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/launcher/TezTestServiceContainerLauncher.java
@@ -18,6 +18,7 @@
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@@ -34,8 +35,6 @@
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
-import com.google.protobuf.ByteString;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
index 459e1705db..58aae461ef 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/dag/app/taskcomm/TezTestServiceTaskCommunicatorImpl.java
@@ -24,6 +24,8 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.ipc.RemoteException;
import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.dag.app.TezTaskCommunicatorImpl;
@@ -38,9 +40,6 @@
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
import org.apache.tez.util.ProtoConverters;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.ServiceException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
index ed3153f6f2..d1227a1895 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/ContainerRunnerImpl.java
@@ -34,6 +34,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataInputBuffer;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
@@ -57,6 +58,8 @@
import org.apache.tez.runtime.api.impl.ExecutionContextImpl;
import org.apache.tez.runtime.api.impl.TaskSpec;
import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolBlockingPB;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolPBClientImpl;
import org.apache.tez.runtime.task.TaskReporter;
import org.apache.tez.runtime.task.TaskRunner2Result;
import org.apache.tez.runtime.task.TezChild;
@@ -439,13 +442,20 @@ public ContainerExecutionResult call() throws Exception {
NetUtils.createSocketAddrForHost(request.getAmHost(), request.getAmPort());
SecurityUtil.setTokenService(jobToken, address);
taskOwner.addToken(jobToken);
- umbilical = taskOwner.doAs(new PrivilegedExceptionAction() {
- @Override
- public TezTaskUmbilicalProtocol run() throws Exception {
- return RPC.getProxy(TezTaskUmbilicalProtocol.class,
- TezTaskUmbilicalProtocol.versionID, address, conf);
- }
- });
+ umbilical =
+ taskOwner.doAs(
+ (PrivilegedExceptionAction)
+ () -> {
+ RPC.setProtocolEngine(
+ conf, TezTaskUmbilicalProtocolBlockingPB.class, ProtobufRpcEngine2.class);
+ TezTaskUmbilicalProtocolBlockingPB proxy =
+ RPC.getProxy(
+ TezTaskUmbilicalProtocolBlockingPB.class,
+ RPC.getProtocolVersion(TezTaskUmbilicalProtocolBlockingPB.class),
+ address,
+ conf);
+ return new TezTaskUmbilicalProtocolPBClientImpl(proxy);
+ });
// TODO Stop reading this on each request.
taskReporter = new TaskReporter(
umbilical,
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
index f18ad06889..c820e07f07 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolClientImpl.java
@@ -18,17 +18,16 @@
import java.net.InetSocketAddress;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerRequestProto;
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.RunContainerResponseProto;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
-
public class TezTestServiceProtocolClientImpl implements TezTestServiceProtocolBlockingPB {
@@ -75,7 +74,7 @@ public TezTestServiceProtocolBlockingPB getProxy() throws IOException {
public TezTestServiceProtocolBlockingPB createProxy() throws IOException {
TezTestServiceProtocolBlockingPB p;
// TODO Fix security
- RPC.setProtocolEngine(conf, TezTestServiceProtocolBlockingPB.class, ProtobufRpcEngine.class);
+ RPC.setProtocolEngine(conf, TezTestServiceProtocolBlockingPB.class, ProtobufRpcEngine2.class);
p = (TezTestServiceProtocolBlockingPB) RPC
.getProxy(TezTestServiceProtocolBlockingPB.class, 0, serverAddr, conf);
return p;
diff --git a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
index a6be6711da..d3e4aa6ff8 100644
--- a/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
+++ b/tez-ext-service-tests/src/test/java/org/apache/tez/service/impl/TezTestServiceProtocolServerImpl.java
@@ -20,10 +20,13 @@
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.ipc.ProtobufRpcEngine;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.thirdparty.protobuf.BlockingService;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.service.ContainerRunner;
import org.apache.tez.service.TezTestServiceProtocolBlockingPB;
@@ -34,9 +37,6 @@
import org.apache.tez.test.service.rpc.TezTestServiceProtocolProtos.SubmitWorkResponseProto;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.BlockingService;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -123,7 +123,7 @@ InetSocketAddress getBindAddress() {
private RPC.Server createServer(Class> pbProtocol, InetSocketAddress addr, Configuration conf,
int numHandlers, BlockingService blockingService) throws
IOException {
- RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine.class);
+ RPC.setProtocolEngine(conf, pbProtocol, ProtobufRpcEngine2.class);
RPC.Server server = new RPC.Builder(conf)
.setProtocol(pbProtocol)
.setInstance(blockingService)
diff --git a/tez-mapreduce/pom.xml b/tez-mapreduce/pom.xml
index 3ef921d5a9..f97a34e24f 100644
--- a/tez-mapreduce/pom.xml
+++ b/tez-mapreduce/pom.xml
@@ -129,8 +129,8 @@
commons-collections4
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
@@ -165,6 +165,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java
index 9abed0d125..59d6c65924 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/hadoop/MRInputHelpers.java
@@ -49,6 +49,7 @@
import org.apache.hadoop.mapreduce.JobSubmissionFiles;
import org.apache.hadoop.mapreduce.split.JobSplitWriter;
import org.apache.hadoop.mapreduce.split.TezGroupedSplit;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType;
@@ -74,7 +75,6 @@
import com.google.common.base.Strings;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java
index 64d97bf1f1..4d97d5cf38 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/input/MultiMRInput.java
@@ -35,6 +35,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.counters.TaskCounter;
import org.apache.tez.mapreduce.input.base.MRInputBase;
@@ -51,7 +52,6 @@
import com.google.common.base.Function;
import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
index 27a98b2b2d..51d08dea16 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
@@ -44,6 +44,7 @@
import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.TezUtils;
import org.apache.tez.common.counters.TaskCounter;
@@ -70,7 +71,6 @@
import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java
index b327b64bcb..b2451b4742 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputAMSplitGenerator.java
@@ -39,6 +39,7 @@
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.hadoop.mapreduce.split.TezGroupedSplit;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.dag.api.DataSourceDescriptor;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.mapreduce.TezTestUtils;
@@ -50,8 +51,6 @@
import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import com.google.protobuf.ByteString;
-
import org.junit.Test;
public class TestMRInputAMSplitGenerator {
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
index 1ad85d1303..22a591602a 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/common/TestMRInputSplitDistributor.java
@@ -31,6 +31,7 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.mapred.InputSplit;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.mapreduce.TezTestUtils;
@@ -45,8 +46,6 @@
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
import org.apache.tez.runtime.api.events.InputUpdatePayloadEvent;
-import com.google.protobuf.ByteString;
-
import org.junit.Test;
public class TestMRInputSplitDistributor {
diff --git a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java
index 92cd8d678c..b96497abb1 100644
--- a/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java
+++ b/tez-mapreduce/src/test/java/org/apache/tez/mapreduce/hadoop/TestMRInputHelpers.java
@@ -41,14 +41,13 @@
import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
import org.apache.hadoop.mapreduce.split.JobSplit;
import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.tez.dag.api.DataSourceDescriptor;
import org.apache.tez.dag.api.TaskLocationHint;
import org.apache.tez.mapreduce.protos.MRRuntimeProtos.MRSplitProto;
import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
diff --git a/tez-plugins/tez-aux-services/pom.xml b/tez-plugins/tez-aux-services/pom.xml
index d935d8ef74..3f88631ac6 100644
--- a/tez-plugins/tez-aux-services/pom.xml
+++ b/tez-plugins/tez-aux-services/pom.xml
@@ -100,11 +100,6 @@
org.slf4j
slf4j-api
-
- com.google.protobuf
- protobuf-java
- provided
-
org.fusesource.leveldbjni
leveldbjni-all
diff --git a/tez-plugins/tez-protobuf-history-plugin/pom.xml b/tez-plugins/tez-protobuf-history-plugin/pom.xml
index b43ac0a92c..0b33ea45bc 100644
--- a/tez-plugins/tez-protobuf-history-plugin/pom.xml
+++ b/tez-plugins/tez-protobuf-history-plugin/pom.xml
@@ -37,8 +37,8 @@
hadoop-common
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
junit
@@ -83,6 +83,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java
index f5386a81b9..3bfbfd8f3c 100644
--- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java
+++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/DatePartitionedLogger.java
@@ -34,11 +34,10 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.protobuf.MessageLite;
+import org.apache.hadoop.thirdparty.protobuf.Parser;
import org.apache.hadoop.yarn.util.Clock;
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java
index 2cac4d8764..de75beb882 100644
--- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java
+++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageReader.java
@@ -25,9 +25,8 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.SequenceFile.Reader;
-
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
+import org.apache.hadoop.thirdparty.protobuf.MessageLite;
+import org.apache.hadoop.thirdparty.protobuf.Parser;
public class ProtoMessageReader implements Closeable {
private final Path filePath;
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 7d20256b67..d2d30fee90 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
@@ -26,12 +26,11 @@
import java.io.OutputStream;
import org.apache.hadoop.io.Writable;
-
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-import com.google.protobuf.ExtensionRegistry;
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
+import org.apache.hadoop.thirdparty.protobuf.ExtensionRegistry;
+import org.apache.hadoop.thirdparty.protobuf.MessageLite;
+import org.apache.hadoop.thirdparty.protobuf.Parser;
public class ProtoMessageWritable implements Writable {
private T message;
diff --git a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java
index 604fd10e55..60a65df190 100644
--- a/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java
+++ b/tez-plugins/tez-protobuf-history-plugin/src/main/java/org/apache/tez/dag/history/logging/proto/ProtoMessageWriter.java
@@ -27,11 +27,10 @@
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.SequenceFile.CompressionType;
import org.apache.hadoop.io.SequenceFile.Writer;
+import org.apache.hadoop.thirdparty.protobuf.MessageLite;
+import org.apache.hadoop.thirdparty.protobuf.Parser;
import org.apache.tez.common.StreamHelper;
-import com.google.protobuf.MessageLite;
-import com.google.protobuf.Parser;
-
public class ProtoMessageWriter implements Closeable {
private final Path filePath;
private final Writer writer;
diff --git a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java
index 8445f0b5ad..fa316b937c 100644
--- a/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java
+++ b/tez-plugins/tez-protobuf-history-plugin/src/test/java/org/apache/tez/dag/history/logging/proto/TestProtoHistoryLoggingService.java
@@ -34,6 +34,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -64,8 +65,6 @@
import org.apache.tez.dag.records.TezVertexID;
import org.apache.tez.hadoop.shim.HadoopShim;
-import com.google.protobuf.CodedInputStream;
-
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
diff --git a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java
index a8ed9ff185..80a461bc99 100644
--- a/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java
+++ b/tez-plugins/tez-yarn-timeline-history-with-fs/src/test/java/org/apache/tez/dag/history/ats/acls/TestATSHistoryV15.java
@@ -32,6 +32,8 @@
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
@@ -55,9 +57,6 @@
import org.apache.tez.runtime.library.processor.SleepProcessor.SleepProcessorConfig;
import org.apache.tez.tests.MiniTezClusterWithTimeline;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
diff --git a/tez-runtime-internals/findbugs-exclude.xml b/tez-runtime-internals/findbugs-exclude.xml
index 259734fad0..c625ccc986 100644
--- a/tez-runtime-internals/findbugs-exclude.xml
+++ b/tez-runtime-internals/findbugs-exclude.xml
@@ -43,6 +43,10 @@
+
+
+
+
diff --git a/tez-runtime-internals/pom.xml b/tez-runtime-internals/pom.xml
index d776cbfd03..2ae347eead 100644
--- a/tez-runtime-internals/pom.xml
+++ b/tez-runtime-internals/pom.xml
@@ -75,8 +75,8 @@
hadoop-yarn-common
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
com.google.guava
@@ -119,6 +119,9 @@
${basedir}/src/main/proto
+
+ ${basedir}/../tez-api/src/main/proto
+
${project.build.directory}/generated-sources/java
@@ -128,6 +131,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java
index 77b9cbe8fb..cc1b017872 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/ProtoConverters.java
@@ -20,6 +20,7 @@
import java.nio.ByteBuffer;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
import org.apache.tez.runtime.api.events.CompositeRoutedDataMovementEvent;
import org.apache.tez.runtime.api.events.CustomProcessorEvent;
@@ -31,7 +32,6 @@
import org.apache.tez.runtime.api.events.VertexManagerEvent;
import com.google.common.base.Charsets;
-import com.google.protobuf.ByteString;
public final class ProtoConverters {
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezPBConverters.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezPBConverters.java
new file mode 100644
index 0000000000..22269d51d8
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezPBConverters.java
@@ -0,0 +1,572 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.tez.common;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.EntityDescriptor;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.records.DAGProtos.TezCountersProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezUserPayloadProto;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.impl.EventType;
+import org.apache.tez.runtime.api.impl.GroupInputSpec;
+import org.apache.tez.runtime.api.impl.IOStatistics;
+import org.apache.tez.runtime.api.impl.InputSpec;
+import org.apache.tez.runtime.api.impl.OutputSpec;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TaskStatistics;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.ContainerTaskProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.EventMetaDataProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.EventProducerConsumerTypeProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.EventTypeProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.GroupInputSpecProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.HeartbeatRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.HeartbeatResponseProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.IOStatisticsProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.InputSpecProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.OutputSpecProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TaskSpecProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TaskStatisticsProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TaskStatusUpdateEventProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezDAGIDProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezEventProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezLocalResourceProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezTaskAttemptIDProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezTaskIDProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezVertexIDProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolUtils;
+
+/**
+ * Utility class for converting between Tez Java domain objects and their Protobuf counterparts used
+ * in the TaskUmbilicalProtocol.
+ */
+@Private
+public final class TezPBConverters {
+
+ private TezPBConverters() {}
+
+ // --- ID Converters ---
+
+ public static TezDAGIDProto convertToProto(TezDAGID id) {
+ return TezDAGIDProto.newBuilder()
+ .setClusterTimestamp(id.getApplicationId().getClusterTimestamp())
+ .setAppId(id.getApplicationId().getId())
+ .setId(id.getId())
+ .build();
+ }
+
+ public static TezDAGID convertFromProto(TezDAGIDProto proto) {
+ return TezDAGID.getInstance(
+ ApplicationId.newInstance(proto.getClusterTimestamp(), proto.getAppId()), proto.getId());
+ }
+
+ public static TezVertexIDProto convertToProto(TezVertexID id) {
+ return TezVertexIDProto.newBuilder()
+ .setDagId(convertToProto(id.getDAGID()))
+ .setId(id.getId())
+ .build();
+ }
+
+ public static TezVertexID convertFromProto(TezVertexIDProto proto) {
+ return TezVertexID.getInstance(convertFromProto(proto.getDagId()), proto.getId());
+ }
+
+ public static TezTaskIDProto convertToProto(TezTaskID id) {
+ return TezTaskIDProto.newBuilder()
+ .setVertexId(convertToProto(id.getVertexID()))
+ .setId(id.getId())
+ .build();
+ }
+
+ public static TezTaskID convertFromProto(TezTaskIDProto proto) {
+ return TezTaskID.getInstance(convertFromProto(proto.getVertexId()), proto.getId());
+ }
+
+ public static TezTaskAttemptIDProto convertToProto(TezTaskAttemptID id) {
+ return TezTaskAttemptIDProto.newBuilder()
+ .setTaskId(convertToProto(id.getTaskID()))
+ .setId(id.getId())
+ .build();
+ }
+
+ public static TezTaskAttemptID convertFromProto(TezTaskAttemptIDProto proto) {
+ return TezTaskAttemptID.getInstance(convertFromProto(proto.getTaskId()), proto.getId());
+ }
+
+ // --- Entity Converters ---
+
+ public static TezEntityDescriptorProto convertToProto(EntityDescriptor> descriptor) {
+ TezEntityDescriptorProto.Builder builder = TezEntityDescriptorProto.newBuilder();
+ builder.setClassName(descriptor.getClassName());
+ if (descriptor.getUserPayload() != null) {
+ TezUserPayloadProto.Builder payloadBuilder = TezUserPayloadProto.newBuilder();
+ if (descriptor.getUserPayload().getPayload() != null) {
+ payloadBuilder.setUserPayload(
+ ByteString.copyFrom(descriptor.getUserPayload().getPayload()));
+ }
+ payloadBuilder.setVersion(descriptor.getUserPayload().getVersion());
+ builder.setTezUserPayload(payloadBuilder.build());
+ }
+ if (descriptor.getHistoryText() != null) {
+ builder.setHistoryText(ByteString.copyFromUtf8(descriptor.getHistoryText()));
+ }
+ return builder.build();
+ }
+
+ private static void fillDescriptorFromProto(
+ EntityDescriptor> descriptor, TezEntityDescriptorProto proto) {
+ if (proto.hasTezUserPayload()) {
+ descriptor.setUserPayload(
+ UserPayload.create(
+ !proto.getTezUserPayload().getUserPayload().isEmpty()
+ ? proto.getTezUserPayload().getUserPayload().asReadOnlyByteBuffer()
+ : null,
+ proto.getTezUserPayload().getVersion()));
+ }
+ if (!proto.getHistoryText().isEmpty()) {
+ descriptor.setHistoryText(proto.getHistoryText().toStringUtf8());
+ }
+ }
+
+ public static ProcessorDescriptor convertProcessorDescriptorFromProto(
+ TezEntityDescriptorProto proto) {
+ ProcessorDescriptor descriptor = ProcessorDescriptor.create(proto.getClassName());
+ fillDescriptorFromProto(descriptor, proto);
+ return descriptor;
+ }
+
+ public static InputDescriptor convertInputDescriptorFromProto(TezEntityDescriptorProto proto) {
+ InputDescriptor descriptor = InputDescriptor.create(proto.getClassName());
+ fillDescriptorFromProto(descriptor, proto);
+ return descriptor;
+ }
+
+ public static OutputDescriptor convertOutputDescriptorFromProto(TezEntityDescriptorProto proto) {
+ OutputDescriptor descriptor = OutputDescriptor.create(proto.getClassName());
+ fillDescriptorFromProto(descriptor, proto);
+ return descriptor;
+ }
+
+ // --- Spec Converters ---
+
+ public static InputSpecProto convertToProto(InputSpec spec) {
+ return InputSpecProto.newBuilder()
+ .setSourceVertexName(spec.getSourceVertexName())
+ .setPhysicalEdgeCount(spec.getPhysicalEdgeCount())
+ .setInputDescriptor(convertToProto(spec.getInputDescriptor()))
+ .build();
+ }
+
+ public static InputSpec convertFromProto(InputSpecProto proto) {
+ return new InputSpec(
+ proto.getSourceVertexName(),
+ convertInputDescriptorFromProto(proto.getInputDescriptor()),
+ proto.getPhysicalEdgeCount());
+ }
+
+ public static OutputSpecProto convertToProto(OutputSpec spec) {
+ return OutputSpecProto.newBuilder()
+ .setDestinationVertexName(spec.getDestinationVertexName())
+ .setPhysicalEdgeCount(spec.getPhysicalEdgeCount())
+ .setOutputDescriptor(convertToProto(spec.getOutputDescriptor()))
+ .build();
+ }
+
+ public static OutputSpec convertFromProto(OutputSpecProto proto) {
+ return new OutputSpec(
+ proto.getDestinationVertexName(),
+ convertOutputDescriptorFromProto(proto.getOutputDescriptor()),
+ proto.getPhysicalEdgeCount());
+ }
+
+ public static GroupInputSpecProto convertToProto(GroupInputSpec spec) {
+ return GroupInputSpecProto.newBuilder()
+ .setGroupName(spec.getGroupName())
+ .addAllGroupVertices(spec.getGroupVertices())
+ .setMergedInputDescriptor(convertToProto(spec.getMergedInputDescriptor()))
+ .build();
+ }
+
+ public static GroupInputSpec convertFromProto(GroupInputSpecProto proto) {
+ return new GroupInputSpec(
+ proto.getGroupName(),
+ new ArrayList<>(proto.getGroupVerticesList()),
+ convertInputDescriptorFromProto(proto.getMergedInputDescriptor()));
+ }
+
+ public static TaskSpecProto convertToProto(TaskSpec spec) {
+ TaskSpecProto.Builder builder =
+ TaskSpecProto.newBuilder()
+ .setTaskAttemptId(convertToProto(spec.getTaskAttemptID()))
+ .setDagName(spec.getDAGName())
+ .setVertexName(spec.getVertexName())
+ .setVertexParallelism(spec.getVertexParallelism())
+ .setProcessorDescriptor(convertToProto(spec.getProcessorDescriptor()));
+ for (InputSpec inputSpec : spec.getInputs()) {
+ builder.addInputSpecs(convertToProto(inputSpec));
+ }
+ for (OutputSpec outputSpec : spec.getOutputs()) {
+ builder.addOutputSpecs(convertToProto(outputSpec));
+ }
+ if (spec.getGroupInputs() != null) {
+ for (GroupInputSpec group : spec.getGroupInputs()) {
+ builder.addGroupInputSpecs(convertToProto(group));
+ }
+ }
+ if (spec.getTaskConf() != null) {
+ try {
+ builder.setTaskConfBytes(
+ TezTaskUmbilicalProtocolUtils.serializeToByteString(spec.getTaskConf()));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return builder.build();
+ }
+
+ public static TaskSpec convertFromProto(TaskSpecProto proto) {
+ List inputs = new ArrayList<>(proto.getInputSpecsCount());
+ for (InputSpecProto isp : proto.getInputSpecsList()) {
+ inputs.add(convertFromProto(isp));
+ }
+ List outputs = new ArrayList<>(proto.getOutputSpecsCount());
+ for (OutputSpecProto osp : proto.getOutputSpecsList()) {
+ outputs.add(convertFromProto(osp));
+ }
+ List groups = null;
+ if (proto.getGroupInputSpecsCount() > 0) {
+ groups = new ArrayList<>(proto.getGroupInputSpecsCount());
+ for (GroupInputSpecProto gsp : proto.getGroupInputSpecsList()) {
+ groups.add(convertFromProto(gsp));
+ }
+ }
+ Configuration conf = null;
+ if (!proto.getTaskConfBytes().isEmpty()) {
+ conf = new Configuration(false);
+ try {
+ TezTaskUmbilicalProtocolUtils.deserializeFromByteString(proto.getTaskConfBytes(), conf);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return new TaskSpec(
+ convertFromProto(proto.getTaskAttemptId()),
+ proto.getDagName(),
+ proto.getVertexName(),
+ proto.getVertexParallelism(),
+ convertProcessorDescriptorFromProto(proto.getProcessorDescriptor()),
+ inputs,
+ outputs,
+ groups,
+ conf);
+ }
+
+ public static TezLocalResourceProto convertToProto(TezLocalResource lr) {
+ return TezLocalResourceProto.newBuilder()
+ .setUri(lr.getUri().toString())
+ .setSize(lr.getSize())
+ .setTimestamp(lr.getTimestamp())
+ .build();
+ }
+
+ public static TezLocalResource convertFromProto(TezLocalResourceProto proto) {
+ try {
+ return new TezLocalResource(new URI(proto.getUri()), proto.getSize(), proto.getTimestamp());
+ } catch (java.net.URISyntaxException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ public static ContainerTaskProto convertToProto(ContainerTask task) {
+ ContainerTaskProto.Builder builder =
+ ContainerTaskProto.newBuilder()
+ .setShouldDie(task.shouldDie())
+ .setCredentialsChanged(task.haveCredentialsChanged());
+ if (task.getTaskSpec() != null) {
+ builder.setTaskSpec(convertToProto(task.getTaskSpec()));
+ }
+ if (task.getAdditionalResources() != null) {
+ for (Map.Entry entry : task.getAdditionalResources().entrySet()) {
+ builder.putAdditionalResources(entry.getKey(), convertToProto(entry.getValue()));
+ }
+ }
+ if (task.getCredentials() != null) {
+ builder.setCredentialsBinary(
+ DagTypeConverters.convertCredentialsToProto(task.getCredentials()));
+ }
+ return builder.build();
+ }
+
+ public static ContainerTask convertFromProto(ContainerTaskProto proto) {
+ Map resources = new HashMap<>();
+ for (Map.Entry entry :
+ proto.getAdditionalResourcesMap().entrySet()) {
+ resources.put(entry.getKey(), convertFromProto(entry.getValue()));
+ }
+ return new ContainerTask(
+ proto.hasTaskSpec() ? convertFromProto(proto.getTaskSpec()) : null,
+ proto.getShouldDie(),
+ resources,
+ DagTypeConverters.convertByteStringToCredentials(proto.getCredentialsBinary()),
+ proto.getCredentialsChanged());
+ }
+
+ // --- Heartbeat & Event Converters ---
+
+ public static EventProducerConsumerTypeProto convertToProto(EventProducerConsumerType type) {
+ return EventProducerConsumerTypeProto.valueOf("EPC_" + type.name());
+ }
+
+ public static EventProducerConsumerType convertFromProto(EventProducerConsumerTypeProto proto) {
+ return EventProducerConsumerType.valueOf(proto.name().substring(4));
+ }
+
+ public static EventMetaDataProto convertToProto(EventMetaData meta) {
+ EventMetaDataProto.Builder builder =
+ EventMetaDataProto.newBuilder()
+ .setProducerConsumerType(convertToProto(meta.getEventGenerator()))
+ .setTaskVertexName(meta.getTaskVertexName());
+ if (meta.getEdgeVertexName() != null) {
+ builder.setEdgeVertexName(meta.getEdgeVertexName());
+ }
+ if (meta.getTaskAttemptID() != null) {
+ builder.setTaskAttemptId(convertToProto(meta.getTaskAttemptID()));
+ }
+ return builder.build();
+ }
+
+ public static EventMetaData convertFromProto(EventMetaDataProto proto) {
+ return new EventMetaData(
+ convertFromProto(proto.getProducerConsumerType()),
+ proto.getTaskVertexName(),
+ proto.getEdgeVertexName().isEmpty() ? null : proto.getEdgeVertexName(),
+ proto.hasTaskAttemptId() ? convertFromProto(proto.getTaskAttemptId()) : null);
+ }
+
+ public static IOStatisticsProto convertToProto(IOStatistics stats) {
+ return IOStatisticsProto.newBuilder()
+ .setDataSize(stats.getDataSize())
+ .setItemsProcessed(stats.getItemsProcessed())
+ .build();
+ }
+
+ public static IOStatistics convertFromProto(IOStatisticsProto proto) {
+ IOStatistics stats = new IOStatistics();
+ stats.setDataSize(proto.getDataSize());
+ stats.setItemsProcessed(proto.getItemsProcessed());
+ return stats;
+ }
+
+ public static TaskStatisticsProto convertToProto(TaskStatistics stats) {
+ TaskStatisticsProto.Builder builder = TaskStatisticsProto.newBuilder();
+ for (Map.Entry entry : stats.getIOStatistics().entrySet()) {
+ builder.putIoStatistics(entry.getKey(), convertToProto(entry.getValue()));
+ }
+ return builder.build();
+ }
+
+ public static TaskStatistics convertFromProto(TaskStatisticsProto proto) {
+ TaskStatistics stats = new TaskStatistics();
+ for (Map.Entry entry : proto.getIoStatisticsMap().entrySet()) {
+ stats.addIO(entry.getKey(), convertFromProto(entry.getValue()));
+ }
+ return stats;
+ }
+
+ public static TezCountersProto convertTezCountersToProto(TezCounters counters) {
+ return DagTypeConverters.convertTezCountersToProto(counters);
+ }
+
+ public static TezCounters convertTezCountersFromProto(TezCountersProto proto) {
+ return DagTypeConverters.convertTezCountersFromProto(proto);
+ }
+
+ public static TaskStatusUpdateEventProto convertToProto(TaskStatusUpdateEvent event) {
+ TaskStatusUpdateEventProto.Builder builder =
+ TaskStatusUpdateEventProto.newBuilder()
+ .setProgress(event.getProgress())
+ .setProgressNotified(event.getProgressNotified());
+ if (event.getCounters() != null) {
+ builder.setCounters(convertTezCountersToProto(event.getCounters()));
+ }
+ if (event.getStatistics() != null) {
+ builder.setStatistics(convertToProto(event.getStatistics()));
+ }
+ return builder.build();
+ }
+
+ public static TaskStatusUpdateEvent convertFromProto(TaskStatusUpdateEventProto proto) {
+ return new TaskStatusUpdateEvent(
+ proto.hasCounters() ? convertTezCountersFromProto(proto.getCounters()) : null,
+ proto.getProgress(),
+ proto.hasStatistics() ? convertFromProto(proto.getStatistics()) : null,
+ proto.getProgressNotified());
+ }
+
+ public static TezEventProto convertToProto(TezEvent event) {
+ TezEventProto.Builder builder =
+ TezEventProto.newBuilder()
+ .setEventType(EventTypeProto.valueOf(event.getEventType().name()))
+ .setEventReceivedTime(event.getEventReceivedTime());
+ if (event.getSourceInfo() != null) {
+ builder.setSourceInfo(convertToProto(event.getSourceInfo()));
+ }
+ if (event.getDestinationInfo() != null) {
+ builder.setDestinationInfo(convertToProto(event.getDestinationInfo()));
+ }
+
+ // Payload handling
+ if (event.getEventType() == EventType.TASK_STATUS_UPDATE_EVENT) {
+ builder.setEventPayload(
+ convertToProto((TaskStatusUpdateEvent) event.getEvent()).toByteString());
+ } else {
+ try {
+ builder.setEventPayload(TezTaskUmbilicalProtocolUtils.serializeToByteString(event));
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ return builder.build();
+ }
+
+ public static TezEvent convertFromProto(TezEventProto proto) {
+ TezEvent event = new TezEvent();
+ event.setEventReceivedTime(proto.getEventReceivedTime());
+ event.setEventType(EventType.valueOf(proto.getEventType().name()));
+ if (proto.hasSourceInfo()) {
+ event.setSourceInfo(convertFromProto(proto.getSourceInfo()));
+ }
+ if (proto.hasDestinationInfo()) {
+ event.setDestinationInfo(convertFromProto(proto.getDestinationInfo()));
+ }
+
+ if (!proto.getEventPayload().isEmpty()) {
+ if (proto.getEventType() == EventTypeProto.TASK_STATUS_UPDATE_EVENT) {
+ try {
+ TaskStatusUpdateEventProto payloadProto =
+ TaskStatusUpdateEventProto.parseFrom(proto.getEventPayload());
+ event.setEvent(convertFromProto(payloadProto));
+ } catch (org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException e) {
+ throw new RuntimeException(e);
+ }
+ } else {
+ try {
+ TezTaskUmbilicalProtocolUtils.deserializeFromByteString(proto.getEventPayload(), event);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ }
+ }
+ return event;
+ }
+
+ // --- Heartbeat Converters ---
+
+ public static HeartbeatRequestProto convertToProto(TezHeartbeatRequest request) {
+ HeartbeatRequestProto.Builder builder =
+ HeartbeatRequestProto.newBuilder()
+ .setRequestId(request.getRequestId())
+ .setPreRoutedStartIndex(request.getPreRoutedStartIndex())
+ .setContainerIdentifier(request.getContainerIdentifier())
+ .setStartIndex(request.getStartIndex())
+ .setMaxEvents(request.getMaxEvents())
+ .setUsedMemory(request.getUsedMemory());
+ if (request.getCurrentTaskAttemptID() != null) {
+ builder.setCurrentTaskAttemptId(convertToProto(request.getCurrentTaskAttemptID()));
+ }
+ if (request.getEvents() != null) {
+ for (TezEvent event : request.getEvents()) {
+ builder.addEvents(convertToProto(event));
+ }
+ }
+ return builder.build();
+ }
+
+ public static TezHeartbeatRequest convertFromProto(HeartbeatRequestProto proto) {
+ List events = new ArrayList<>(proto.getEventsCount());
+ for (TezEventProto ep : proto.getEventsList()) {
+ events.add(convertFromProto(ep));
+ }
+ return new TezHeartbeatRequest(
+ proto.getRequestId(),
+ events,
+ proto.getPreRoutedStartIndex(),
+ proto.getContainerIdentifier(),
+ proto.hasCurrentTaskAttemptId() ? convertFromProto(proto.getCurrentTaskAttemptId()) : null,
+ proto.getStartIndex(),
+ proto.getMaxEvents(),
+ proto.getUsedMemory());
+ }
+
+ public static HeartbeatResponseProto convertToProto(TezHeartbeatResponse response) {
+ HeartbeatResponseProto.Builder builder =
+ HeartbeatResponseProto.newBuilder()
+ .setLastRequestId(response.getLastRequestId())
+ .setShouldDie(response.shouldDie())
+ .setNextFromEventId(response.getNextFromEventId())
+ .setNextPreRoutedEventId(response.getNextPreRoutedEventId());
+ if (response.getEvents() != null) {
+ for (TezEvent event : response.getEvents()) {
+ builder.addEvents(convertToProto(event));
+ }
+ }
+ return builder.build();
+ }
+
+ public static TezHeartbeatResponse convertFromProto(HeartbeatResponseProto proto) {
+ List events = new ArrayList<>(proto.getEventsCount());
+ for (TezEventProto ep : proto.getEventsList()) {
+ events.add(convertFromProto(ep));
+ }
+ TezHeartbeatResponse response = new TezHeartbeatResponse(events);
+ response.setLastRequestId(proto.getLastRequestId());
+ if (proto.getShouldDie()) {
+ response.setShouldDie();
+ }
+ response.setNextFromEventId(proto.getNextFromEventId());
+ response.setNextPreRoutedEventId(proto.getNextPreRoutedEventId());
+ return response;
+ }
+}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
index e3bd589b44..b6afd870cc 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/TezTaskUmbilicalProtocol.java
@@ -15,14 +15,12 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.tez.common;
import java.io.IOException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.ipc.VersionedProtocol;
import org.apache.hadoop.security.token.TokenInfo;
import org.apache.tez.dag.api.TezException;
import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -30,27 +28,18 @@
import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
import org.apache.tez.runtime.common.security.JobTokenSelector;
-/** Protocol that task child process uses to contact its parent process. The
- * parent is a daemon which which polls the central master for a new map or
- * reduce task and runs it as a child process. All communication between child
- * and parent is via this protocol. */
+/**
+ * Protocol that task child process uses to contact its parent process. All communication between
+ * child and parent is via this protocol using Protobuf RPC.
+ */
@TokenInfo(JobTokenSelector.class)
@InterfaceAudience.Private
@InterfaceStability.Stable
-// ProtocolInfo will be required once we move to Hadoop PB RPC
-//@ProtocolInfo(protocolName = "TezTaskUmbilicalProtocol", protocolVersion = 1)
-public interface TezTaskUmbilicalProtocol extends VersionedProtocol {
-
- public static final long versionID = 19L;
+public interface TezTaskUmbilicalProtocol {
ContainerTask getTask(ContainerContext containerContext) throws IOException;
boolean canCommit(TezTaskAttemptID taskid) throws IOException;
- /// Copies from TezUmbilical until complete re-factor is done
- // TODONEWTEZ
-
- public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
- throws IOException, TezException;
-
+ TezHeartbeatResponse heartbeat(TezHeartbeatRequest request) throws IOException, TezException;
}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/common/package-info.java b/tez-runtime-internals/src/main/java/org/apache/tez/common/package-info.java
new file mode 100644
index 0000000000..f452f32c77
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/common/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+@Private
+package org.apache.tez.common;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
index e9d4f1127e..a3fec6ce77 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
@@ -27,6 +27,9 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.thirdparty.protobuf.AbstractMessage;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.tez.common.ProtoConverters;
import org.apache.tez.common.TezConverterUtils;
import org.apache.tez.dag.api.TezUncheckedException;
@@ -56,10 +59,6 @@
import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptFailedEventProto;
import org.apache.tez.runtime.internals.api.events.SystemEventProtos.TaskAttemptKilledEventProto;
-import com.google.protobuf.AbstractMessage;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-
public class TezEvent implements Writable {
private EventType eventType;
@@ -119,6 +118,14 @@ public Event getEvent() {
return event;
}
+ public void setEvent(Event event) {
+ this.event = event;
+ }
+
+ public void setEventType(EventType eventType) {
+ this.eventType = eventType;
+ }
+
public void setEventReceivedTime(long eventReceivedTime) { // TODO save
this.eventReceivedTime = eventReceivedTime;
}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolBlockingPB.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolBlockingPB.java
new file mode 100644
index 0000000000..a108b905ef
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolBlockingPB.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tez.runtime.internals.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.ipc.ProtocolInfo;
+import org.apache.hadoop.security.token.TokenInfo;
+import org.apache.tez.runtime.common.security.JobTokenSelector;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.TezTaskUmbilicalProtocol;
+
+@Private
+@TokenInfo(JobTokenSelector.class)
+@ProtocolInfo(
+ protocolName = "org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolBlockingPB",
+ protocolVersion = 1)
+public interface TezTaskUmbilicalProtocolBlockingPB
+ extends TezTaskUmbilicalProtocol.BlockingInterface {}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBClientImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBClientImpl.java
new file mode 100644
index 0000000000..cb497c71c7
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBClientImpl.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tez.runtime.internals.protocolPB;
+
+import static org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolUtils.service;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezPBConverters;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.CanCommitRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.ContainerContextProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.GetTaskRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.GetTaskResponseProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.HeartbeatRequestProto;
+
+/**
+ * Protobuf-based client implementation for the TezTaskUmbilicalProtocol. This class handles the
+ * translation between Java API calls and remote RPC calls.
+ */
+@Private
+public class TezTaskUmbilicalProtocolPBClientImpl implements TezTaskUmbilicalProtocol, Closeable {
+
+ private final TezTaskUmbilicalProtocolBlockingPB proxy;
+
+ public TezTaskUmbilicalProtocolPBClientImpl(TezTaskUmbilicalProtocolBlockingPB proxy) {
+ this.proxy = proxy;
+ }
+
+ @Override
+ public void close() throws IOException {
+ if (proxy != null) {
+ RPC.stopProxy(proxy);
+ }
+ }
+
+ /** Called by the Task process to retrieve the work from the AM. */
+ @Override
+ public ContainerTask getTask(ContainerContext containerContext) throws IOException {
+ return service(
+ () -> {
+ GetTaskRequestProto.Builder builder = GetTaskRequestProto.newBuilder();
+ if (containerContext != null) {
+ builder.setContainerContext(
+ ContainerContextProto.newBuilder()
+ .setContainerIdentifier(containerContext.getContainerIdentifier())
+ .build());
+ }
+ GetTaskResponseProto response = proxy.getTask(null, builder.build());
+ return response.hasContainerTask()
+ ? TezPBConverters.convertFromProto(response.getContainerTask())
+ : null;
+ });
+ }
+
+ /** Called by the Task process to check if it's allowed to commit its outputs. */
+ @Override
+ public boolean canCommit(TezTaskAttemptID taskid) throws IOException {
+ return service(
+ () -> {
+ CanCommitRequestProto.Builder builder = CanCommitRequestProto.newBuilder();
+ if (taskid != null) {
+ builder.setTaskAttemptId(TezPBConverters.convertToProto(taskid));
+ }
+ return proxy.canCommit(null, builder.build()).getResponse();
+ });
+ }
+
+ /** Heartbeat sent by the Task to report status and retrieve events from the AM. */
+ @Override
+ public TezHeartbeatResponse heartbeat(TezHeartbeatRequest request)
+ throws IOException, TezException {
+ return service(
+ () -> {
+ HeartbeatRequestProto requestProto = TezPBConverters.convertToProto(request);
+ return TezPBConverters.convertFromProto(proxy.heartbeat(null, requestProto));
+ });
+ }
+}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBServerImpl.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBServerImpl.java
new file mode 100644
index 0000000000..e806050f94
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolPBServerImpl.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tez.runtime.internals.protocolPB;
+
+import static org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolUtils.translate;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.thirdparty.protobuf.RpcController;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.common.TezPBConverters;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.CanCommitRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.CanCommitResponseProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.GetTaskRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.GetTaskResponseProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.HeartbeatRequestProto;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolProtos.HeartbeatResponseProto;
+
+/**
+ * Protobuf-based server implementation for the TezTaskUmbilicalProtocol. This class translates
+ * incoming RPC calls into internal Java protocol method calls.
+ */
+@Private
+public class TezTaskUmbilicalProtocolPBServerImpl implements TezTaskUmbilicalProtocolBlockingPB {
+
+ private final TezTaskUmbilicalProtocol real;
+
+ public TezTaskUmbilicalProtocolPBServerImpl(TezTaskUmbilicalProtocol real) {
+ this.real = real;
+ }
+
+ /** Receives a request for work from a Task process. */
+ @Override
+ public GetTaskResponseProto getTask(RpcController controller, GetTaskRequestProto request)
+ throws ServiceException {
+ return translate(
+ () -> {
+ ContainerContext containerContext = null;
+ if (request.hasContainerContext()) {
+ containerContext =
+ new ContainerContext(request.getContainerContext().getContainerIdentifier());
+ }
+ ContainerTask response = real.getTask(containerContext);
+ GetTaskResponseProto.Builder builder = GetTaskResponseProto.newBuilder();
+ if (response != null) {
+ builder.setContainerTask(TezPBConverters.convertToProto(response));
+ }
+ return builder.build();
+ });
+ }
+
+ /** Receives a request to check if a task is allowed to commit its outputs. */
+ @Override
+ public CanCommitResponseProto canCommit(RpcController controller, CanCommitRequestProto request)
+ throws ServiceException {
+ return translate(
+ () -> {
+ TezTaskAttemptID taskAttemptID = null;
+ if (request.hasTaskAttemptId()) {
+ taskAttemptID = TezPBConverters.convertFromProto(request.getTaskAttemptId());
+ }
+ boolean response = real.canCommit(taskAttemptID);
+ return CanCommitResponseProto.newBuilder().setResponse(response).build();
+ });
+ }
+
+ /** Receives heartbeats from Task processes. */
+ @Override
+ public HeartbeatResponseProto heartbeat(RpcController controller, HeartbeatRequestProto request)
+ throws ServiceException {
+ return translate(
+ () -> {
+ TezHeartbeatRequest heartbeatRequest = TezPBConverters.convertFromProto(request);
+ TezHeartbeatResponse response = real.heartbeat(heartbeatRequest);
+ return TezPBConverters.convertToProto(response);
+ });
+ }
+}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolUtils.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolUtils.java
new file mode 100644
index 0000000000..692c94ca11
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/TezTaskUmbilicalProtocolUtils.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.tez.runtime.internals.protocolPB;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.ServiceException;
+import org.apache.tez.dag.api.TezException;
+
+@Private
+public final class TezTaskUmbilicalProtocolUtils {
+
+ private TezTaskUmbilicalProtocolUtils() {}
+
+ /** Serialize a Writable into a protobuf ByteString. */
+ public static ByteString serializeToByteString(Writable writable) throws IOException {
+ if (writable == null) {
+ return null;
+ }
+ ByteString.Output os = ByteString.newOutput();
+ try (DataOutputStream dos = new DataOutputStream(os)) {
+ writable.write(dos);
+ }
+ return os.toByteString();
+ }
+
+ /** Deserialize specific writable from bytes in a protobuf string. */
+ public static void deserializeFromByteString(ByteString byteString, Writable writable)
+ throws IOException {
+ if (byteString == null || byteString.isEmpty()) {
+ return;
+ }
+ try (InputStream is = byteString.newInput();
+ DataInputStream dis = new DataInputStream(is)) {
+ writable.readFields(dis);
+ }
+ }
+
+ /*
+ * Service invocation with exception translation.
+ */
+
+ @FunctionalInterface
+ public interface ServiceCallable {
+ T call() throws ServiceException;
+ }
+
+ /** Invoke a service. Translates ServiceException to IOException. */
+ public static T service(ServiceCallable callable) throws IOException {
+ try {
+ return callable.call();
+ } catch (ServiceException e) {
+ throw new IOException(e);
+ }
+ }
+
+ @FunctionalInterface
+ public interface ClientCallable {
+ T call() throws IOException, TezException;
+ }
+
+ /** Translates application exceptions (IOException, TezException) to RPC ServiceException. */
+ public static T translate(ClientCallable callable) throws ServiceException {
+ try {
+ return callable.call();
+ } catch (IOException | TezException e) {
+ throw new ServiceException(e);
+ }
+ }
+}
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/package-info.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/package-info.java
new file mode 100644
index 0000000000..26ad3fe002
--- /dev/null
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/internals/protocolPB/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+@Private
+package org.apache.tez.runtime.internals.protocolPB;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
index 60faec56fa..94eaf5b786 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/task/TezChild.java
@@ -41,6 +41,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.ipc.CallerContext;
+import org.apache.hadoop.ipc.ProtobufRpcEngine2;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials;
@@ -78,6 +79,8 @@
import org.apache.tez.runtime.common.objectregistry.ObjectRegistryImpl;
import org.apache.tez.runtime.hook.TezTaskAttemptHook;
import org.apache.tez.runtime.internals.api.TaskReporterInterface;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolBlockingPB;
+import org.apache.tez.runtime.internals.protocolPB.TezTaskUmbilicalProtocolPBClientImpl;
import org.apache.tez.util.LoggingUtils;
import org.apache.tez.util.TezRuntimeShutdownHandler;
@@ -196,13 +199,22 @@ public TezChild(Configuration conf, String host, int port, String containerIdent
final InetSocketAddress address = NetUtils.createSocketAddrForHost(host, port);
SecurityUtil.setTokenService(jobToken, address);
taskOwner.addToken(jobToken);
- this.umbilical = taskOwner.doAs(new PrivilegedExceptionAction() {
- @Override
- public TezTaskUmbilicalProtocol run() throws Exception {
- return RPC.getProxy(TezTaskUmbilicalProtocol.class,
- TezTaskUmbilicalProtocol.versionID, address, defaultConf);
- }
- });
+ this.umbilical =
+ taskOwner.doAs(
+ (PrivilegedExceptionAction)
+ () -> {
+ RPC.setProtocolEngine(
+ defaultConf,
+ TezTaskUmbilicalProtocolBlockingPB.class,
+ ProtobufRpcEngine2.class);
+ TezTaskUmbilicalProtocolBlockingPB proxy =
+ RPC.getProxy(
+ TezTaskUmbilicalProtocolBlockingPB.class,
+ RPC.getProtocolVersion(TezTaskUmbilicalProtocolBlockingPB.class),
+ address,
+ defaultConf);
+ return new TezTaskUmbilicalProtocolPBClientImpl(proxy);
+ });
ownUmbilical = true;
} else {
this.umbilical = umbilical;
diff --git a/tez-runtime-internals/src/main/proto/TezTaskUmbilicalProtocol.proto b/tez-runtime-internals/src/main/proto/TezTaskUmbilicalProtocol.proto
new file mode 100644
index 0000000000..09660e583f
--- /dev/null
+++ b/tez-runtime-internals/src/main/proto/TezTaskUmbilicalProtocol.proto
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+syntax = "proto3";
+
+option java_package = "org.apache.tez.runtime.internals.protocolPB";
+option java_outer_classname = "TezTaskUmbilicalProtocolProtos";
+option java_generic_services = true;
+
+import "DAGApiRecords.proto";
+
+// Imitates TezDAGID.java
+message TezDAGIDProto {
+ optional int64 cluster_timestamp = 1;
+ optional int32 app_id = 2;
+ optional int32 id = 3;
+}
+
+// Imitates TezVertexID.java
+message TezVertexIDProto {
+ optional TezDAGIDProto dag_id = 1;
+ optional int32 id = 2;
+}
+
+// Imitates TezTaskID.java
+message TezTaskIDProto {
+ optional TezVertexIDProto vertex_id = 1;
+ optional int32 id = 2;
+}
+
+// Imitates TezTaskAttemptID.java
+message TezTaskAttemptIDProto {
+ optional TezTaskIDProto task_id = 1;
+ optional int32 id = 2;
+}
+
+// Imitates ContainerContext.java
+message ContainerContextProto {
+ optional string container_identifier = 1;
+}
+
+// Imitates TezLocalResource.java
+message TezLocalResourceProto {
+ optional string uri = 1;
+ optional int64 size = 2;
+ optional int64 timestamp = 3;
+}
+
+// Imitates InputSpec.java
+message InputSpecProto {
+ optional string source_vertex_name = 1;
+ optional TezEntityDescriptorProto input_descriptor = 2;
+ optional int32 physical_edge_count = 3;
+}
+
+// Imitates OutputSpec.java
+message OutputSpecProto {
+ optional string destination_vertex_name = 1;
+ optional TezEntityDescriptorProto output_descriptor = 2;
+ optional int32 physical_edge_count = 3;
+}
+
+// Imitates GroupInputSpec.java
+message GroupInputSpecProto {
+ optional string group_name = 1;
+ repeated string group_vertices = 2;
+ optional TezEntityDescriptorProto merged_input_descriptor = 3;
+}
+
+// Imitates TaskSpec.java
+message TaskSpecProto {
+ optional TezTaskAttemptIDProto task_attempt_id = 1;
+ optional string dag_name = 2;
+ optional string vertex_name = 3;
+ optional int32 vertex_parallelism = 4;
+ optional TezEntityDescriptorProto processor_descriptor = 5;
+ repeated InputSpecProto input_specs = 6;
+ repeated OutputSpecProto output_specs = 7;
+ repeated GroupInputSpecProto group_input_specs = 8;
+ optional bytes task_conf_bytes = 9;
+}
+
+// Imitates ContainerTask.java
+message ContainerTaskProto {
+ optional TaskSpecProto task_spec = 1;
+ optional bool should_die = 2;
+ map additional_resources = 3;
+ optional bytes credentials_binary = 4;
+ optional bool credentials_changed = 5;
+}
+
+// Imitates EventMetaData.EventProducerConsumerType enum
+enum EventProducerConsumerTypeProto {
+ EPC_INPUT = 0;
+ EPC_PROCESSOR = 1;
+ EPC_OUTPUT = 2;
+ EPC_SYSTEM = 3;
+}
+
+// Imitates EventMetaData.java
+message EventMetaDataProto {
+ optional EventProducerConsumerTypeProto producer_consumer_type = 1;
+ optional string task_vertex_name = 2;
+ optional string edge_vertex_name = 3;
+ optional TezTaskAttemptIDProto task_attempt_id = 4;
+}
+
+// Imitates EventType.java enum
+enum EventTypeProto {
+ TASK_ATTEMPT_COMPLETED_EVENT = 0;
+ TASK_ATTEMPT_FAILED_EVENT = 1;
+ TASK_ATTEMPT_KILLED_EVENT = 2;
+ DATA_MOVEMENT_EVENT = 3;
+ INPUT_READ_ERROR_EVENT = 4;
+ INPUT_FAILED_EVENT = 5;
+ TASK_STATUS_UPDATE_EVENT = 6;
+ VERTEX_MANAGER_EVENT = 7;
+ ROOT_INPUT_DATA_INFORMATION_EVENT = 8;
+ COMPOSITE_DATA_MOVEMENT_EVENT = 9;
+ ROOT_INPUT_INITIALIZER_EVENT = 10;
+ COMPOSITE_ROUTED_DATA_MOVEMENT_EVENT = 11;
+ CUSTOM_PROCESSOR_EVENT = 12;
+}
+
+// Imitates IOStatistics.java
+message IOStatisticsProto {
+ optional int64 data_size = 1;
+ optional int64 items_processed = 2;
+}
+
+// Imitates TaskStatistics.java
+message TaskStatisticsProto {
+ map io_statistics = 1;
+}
+
+// Imitates TaskStatusUpdateEvent.java
+message TaskStatusUpdateEventProto {
+ optional TezCountersProto counters = 1;
+ optional float progress = 2;
+ optional TaskStatisticsProto statistics = 3;
+ optional bool progress_notified = 4;
+}
+
+// Imitates TezEvent.java
+message TezEventProto {
+ optional EventTypeProto event_type = 1;
+ optional bytes event_payload = 2;
+ optional EventMetaDataProto source_info = 3;
+ optional EventMetaDataProto destination_info = 4;
+ optional int64 event_received_time = 5;
+}
+
+message GetTaskRequestProto {
+ optional ContainerContextProto container_context = 1;
+}
+
+message GetTaskResponseProto {
+ optional ContainerTaskProto container_task = 1;
+}
+
+message CanCommitRequestProto {
+ optional TezTaskAttemptIDProto task_attempt_id = 1;
+}
+
+message CanCommitResponseProto {
+ optional bool response = 1;
+}
+
+// Imitates TezHeartbeatRequest.java
+message HeartbeatRequestProto {
+ optional int64 request_id = 1;
+ repeated TezEventProto events = 2;
+ optional int32 pre_routed_start_index = 3;
+ optional string container_identifier = 4;
+ optional TezTaskAttemptIDProto current_task_attempt_id = 5;
+ optional int32 start_index = 6;
+ optional int32 max_events = 7;
+ optional int64 used_memory = 8;
+}
+
+// Imitates TezHeartbeatResponse.java
+message HeartbeatResponseProto {
+ optional int64 last_request_id = 1;
+ optional bool should_die = 2;
+ repeated TezEventProto events = 3;
+ optional int32 next_from_event_id = 4;
+ optional int32 next_pre_routed_event_id = 5;
+}
+
+// --- RPC Service ---
+
+service TezTaskUmbilicalProtocol {
+ rpc getTask (GetTaskRequestProto) returns (GetTaskResponseProto);
+ rpc canCommit (CanCommitRequestProto) returns (CanCommitResponseProto);
+ rpc heartbeat (HeartbeatRequestProto) returns (HeartbeatResponseProto);
+}
diff --git a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
index cdab5a0165..6413a01219 100644
--- a/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
+++ b/tez-runtime-internals/src/test/java/org/apache/tez/runtime/task/TaskExecutionTestHelpers.java
@@ -26,7 +26,6 @@
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
-import org.apache.hadoop.ipc.ProtocolSignature;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -465,17 +464,6 @@ public void verifyTaskSuccessEvent() {
}
}
- @Override
- public long getProtocolVersion(String protocol, long clientVersion) {
- return 0;
- }
-
- @Override
- public ProtocolSignature getProtocolSignature(String protocol, long clientVersion,
- int clientMethodsHash) {
- return null;
- }
-
@Override
public ContainerTask getTask(ContainerContext containerContext) throws IOException {
// Return shouldDie = true
diff --git a/tez-runtime-library/pom.xml b/tez-runtime-library/pom.xml
index a09311d90a..19685346d9 100644
--- a/tez-runtime-library/pom.xml
+++ b/tez-runtime-library/pom.xml
@@ -50,8 +50,8 @@
tez-common
- com.google.protobuf
- protobuf-java
+ org.apache.hadoop.thirdparty
+ hadoop-shaded-protobuf_3_25
junit
@@ -142,6 +142,10 @@
+
+ com.google.code.maven-replacer-plugin
+ replacer
+
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairEdgeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairEdgeConfiguration.java
index b158f9f6b5..121fa71fd3 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairEdgeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairEdgeConfiguration.java
@@ -22,14 +22,13 @@
import java.util.HashMap;
import java.util.Map.Entry;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.library.vertexmanager.FairShuffleUserPayloads.FairShuffleEdgeManagerConfigPayloadProto;
import org.apache.tez.dag.library.vertexmanager.FairShuffleUserPayloads.FairShuffleEdgeManagerDestinationTaskPropProto;
import org.apache.tez.dag.library.vertexmanager.FairShuffleUserPayloads.RangeProto;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-
/**
* Handles edge configuration serialization and de-serialization between
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleEdgeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleEdgeManager.java
index 913bf625bd..585956a36d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleEdgeManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/FairShuffleEdgeManager.java
@@ -22,12 +22,11 @@
import javax.annotation.Nullable;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.dag.api.EdgeManagerPluginContext;
import org.apache.tez.dag.api.EdgeManagerPluginOnDemand;
import org.apache.tez.dag.api.UserPayload;
-import com.google.protobuf.InvalidProtocolBufferException;
-
/**
* Edge manager for fair routing. Each destination task has its
* DestinationTaskInputsProperty used to decide how to do event routing
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
index c4e2fe2a46..2392b8d467 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManager.java
@@ -32,6 +32,8 @@
import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.TezUtils;
import org.apache.tez.dag.api.EdgeManagerPluginContext;
@@ -48,8 +50,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.ShuffleEdgeManagerConfigPayloadProto;
import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java
index b38959b6a3..99dfa9f23a 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/dag/library/vertexmanager/ShuffleVertexManagerBase.java
@@ -34,6 +34,8 @@
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtils;
@@ -62,8 +64,6 @@
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java
index 88bfa46a3c..3063ab9a53 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductConfig.java
@@ -27,14 +27,14 @@
import java.util.Map;
import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.common.Preconditions;
import org.apache.tez.dag.api.TezConfiguration;
import org.apache.tez.dag.api.UserPayload;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.primitives.Ints;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
/**
* CartesianProductConfig is used to configure both
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java
index 6d6af8df80..046bcb6ffd 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductEdgeManager.java
@@ -21,12 +21,12 @@
import javax.annotation.Nullable;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.dag.api.EdgeManagerPluginContext;
import org.apache.tez.dag.api.EdgeManagerPluginOnDemand;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
/**
* This EM wrap a real edge manager implementation object. It choose whether it's partitioned or
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java
index c80182765a..0a261af726 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/CartesianProductVertexManager.java
@@ -28,6 +28,7 @@
import java.util.Map;
import java.util.Set;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
import org.apache.tez.dag.api.EdgeProperty;
@@ -41,7 +42,6 @@
import org.apache.tez.runtime.api.events.VertexManagerEvent;
import com.google.common.annotations.VisibleForTesting;
-import com.google.protobuf.ByteString;
/**
* This VM wrap a real vertex manager implementation object. It choose whether it's partitioned or
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/FairCartesianProductVertexManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/FairCartesianProductVertexManager.java
index 992c302f36..7f7f9d3549 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/FairCartesianProductVertexManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/cartesianproduct/FairCartesianProductVertexManager.java
@@ -31,6 +31,7 @@
import java.util.Map;
import java.util.Queue;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.UserPayload;
import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -44,7 +45,6 @@
import com.google.common.math.LongMath;
import com.google.common.primitives.Ints;
-import com.google.protobuf.ByteString;
import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
index fdc15dd85e..1da00ca9ac 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/ShuffleUtils.java
@@ -41,6 +41,7 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtilsInternal;
@@ -67,7 +68,6 @@
import org.apache.tez.util.FastNumberFormat;
import com.google.common.primitives.Ints;
-import com.google.protobuf.ByteString;
import org.roaringbitmap.RoaringBitmap;
import org.slf4j.Logger;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java
index d46cd86269..bdf0450513 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/impl/ShuffleInputEventHandlerImpl.java
@@ -26,6 +26,8 @@
import java.util.zip.Inflater;
import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.dag.api.TezUncheckedException;
@@ -45,9 +47,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataProto;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
index ef977137f4..79dcdd191d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/ShuffleInputEventHandlerOrderedGrouped.java
@@ -24,6 +24,8 @@
import java.util.concurrent.atomic.AtomicInteger;
import java.util.zip.Inflater;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.dag.api.TezUncheckedException;
@@ -39,9 +41,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
import org.apache.tez.util.StringInterner;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java
index 323f81c118..f31bebc7fd 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/writers/UnorderedPartitionedKVWriter.java
@@ -51,6 +51,7 @@
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.CallableWithNdc;
import org.apache.tez.common.GuavaShim;
import org.apache.tez.common.Preconditions;
@@ -84,7 +85,6 @@
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java
index 6dfefcb395..26f1111654 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/dag/library/vertexmanager/TestShuffleVertexManagerUtils.java
@@ -36,6 +36,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtils;
@@ -60,8 +61,6 @@
import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto;
-import com.google.protobuf.ByteString;
-
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
import org.roaringbitmap.RoaringBitmap;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductEdgeManagerPartitioned.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductEdgeManagerPartitioned.java
index 629ecebd1f..16d2782699 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductEdgeManagerPartitioned.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestCartesianProductEdgeManagerPartitioned.java
@@ -28,13 +28,12 @@
import java.nio.ByteBuffer;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.dag.api.EdgeManagerPluginContext;
import org.apache.tez.dag.api.EdgeManagerPluginOnDemand.CompositeEventRouteMetadata;
import org.apache.tez.dag.api.EdgeManagerPluginOnDemand.EventRouteMetadata;
import org.apache.tez.dag.api.UserPayload;
-import com.google.protobuf.ByteString;
-
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java
index 4203c94460..ce20ec847c 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/cartesianproduct/TestFairCartesianProductVertexManager.java
@@ -37,6 +37,8 @@
import java.util.List;
import java.util.Map;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
+import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException;
import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
import org.apache.tez.dag.api.EdgeProperty;
import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -54,8 +56,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.VertexManagerEventPayloadProto;
import com.google.common.primitives.Ints;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.InvalidProtocolBufferException;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
index 1d4c55dcc9..e70faae8e4 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/TestShuffleUtils.java
@@ -32,6 +32,7 @@
import org.apache.hadoop.io.compress.CompressionOutputStream;
import org.apache.hadoop.io.compress.Compressor;
import org.apache.hadoop.io.compress.Decompressor;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -53,7 +54,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
import com.google.common.collect.Lists;
-import com.google.protobuf.ByteString;
import org.junit.Assert;
import org.junit.Before;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java
index 955b1e86a5..f345c70029 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/impl/TestShuffleInputEventHandlerImpl.java
@@ -46,6 +46,7 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezExecutors;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -69,8 +70,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads.DataMovementEventPayloadProto;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java
index 6720c76e97..94a3bd0139 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/shuffle/orderedgrouped/TestShuffleInputEventHandlerOrderedGrouped.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezExecutors;
@@ -43,8 +44,6 @@
import org.apache.tez.runtime.library.common.shuffle.ShuffleUtils;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java
index 55505e2ba7..2544eaa390 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/TestIFile.java
@@ -54,6 +54,7 @@
import org.apache.hadoop.io.serializer.Deserializer;
import org.apache.hadoop.io.serializer.SerializationFactory;
import org.apache.hadoop.io.serializer.WritableSerialization;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.util.NativeCodeLoader;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
@@ -67,8 +68,6 @@
import org.apache.tez.runtime.library.utils.BufferUtils;
import org.apache.tez.runtime.library.utils.CodecUtils;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Assume;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
index 5970d8b779..39694bcdb6 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/sort/impl/dflt/TestDefaultSorter.java
@@ -47,6 +47,7 @@
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.common.TezUtils;
@@ -76,8 +77,6 @@
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
import org.apache.tez.util.StringInterner;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java
index 4f6012a5ab..62854c94aa 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/common/writers/TestUnorderedPartitionedKVWriter.java
@@ -65,6 +65,7 @@
import org.apache.hadoop.io.Writable;
import org.apache.hadoop.io.compress.CompressionCodec;
import org.apache.hadoop.io.compress.DefaultCodec;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -96,7 +97,6 @@
import com.google.common.collect.LinkedListMultimap;
import com.google.common.collect.Multimap;
import com.google.common.collect.Sets;
-import com.google.protobuf.ByteString;
import org.junit.After;
import org.junit.Before;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java
index 8735a9a948..4e9c159c3e 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileSortedOutput.java
@@ -43,6 +43,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.common.TezUtils;
import org.apache.tez.common.counters.TezCounters;
@@ -63,8 +64,6 @@
import org.apache.tez.runtime.library.partitioner.HashPartitioner;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java
index 727407349c..77397da15f 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOnFileUnorderedKVOutput.java
@@ -44,6 +44,7 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.hadoop.yarn.util.AuxiliaryServiceHelper;
import org.apache.tez.common.TezSharedExecutor;
import org.apache.tez.common.TezUtils;
@@ -73,8 +74,6 @@
import org.apache.tez.runtime.library.testutils.KVDataGen;
import org.apache.tez.runtime.library.testutils.KVDataGen.KVPair;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
index 6da45fb896..23dc283d8d 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestOrderedPartitionedKVOutput2.java
@@ -26,6 +26,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.common.TezUtilsInternal;
@@ -37,8 +38,6 @@
import org.apache.tez.runtime.library.partitioner.HashPartitioner;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
index 05849fc700..f91268c10c 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedKVOutput2.java
@@ -28,6 +28,7 @@
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.Text;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezRuntimeFrameworkConfigs;
import org.apache.tez.common.TezUtilsInternal;
@@ -38,8 +39,6 @@
import org.apache.tez.runtime.library.partitioner.HashPartitioner;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
-import com.google.protobuf.ByteString;
-
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
diff --git a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
index 86ebfdf96f..5de6248984 100644
--- a/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
+++ b/tez-runtime-library/src/test/java/org/apache/tez/runtime/library/output/TestUnorderedPartitionedKVOutput2.java
@@ -23,6 +23,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.thirdparty.protobuf.ByteString;
import org.apache.tez.common.TezCommonUtils;
import org.apache.tez.common.TezUtilsInternal;
import org.apache.tez.runtime.api.Event;
@@ -30,8 +31,6 @@
import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
import org.apache.tez.runtime.library.shuffle.impl.ShuffleUserPayloads;
-import com.google.protobuf.ByteString;
-
import org.junit.Test;
// Tests which don't require parameterization
diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
index d895f7e1de..6633e69077 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
@@ -24,6 +24,8 @@
import org.apache.commons.codec.binary.Base64;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream;
import org.apache.tez.common.Preconditions;
import org.apache.tez.common.ReflectionUtils;
import org.apache.tez.dag.api.TezReflectionException;
@@ -45,9 +47,6 @@
import org.apache.tez.dag.records.TezDAGID;
import org.apache.tez.test.RecoveryServiceWithEventHandlingHook.SimpleShutdownCondition.TIMING;
-import com.google.protobuf.CodedInputStream;
-import com.google.protobuf.CodedOutputStream;
-
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;