You are viewing a plain text version of this content. The canonical link for it is here.
Posted to yarn-dev@hadoop.apache.org by Wei-Chiu Chuang <we...@cloudera.com.INVALID> on 2020/04/29 05:46:33 UTC

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

I'm sorry for coming to this late. I missed this message. It should have
been a DISCUSS thread rather than NOTICE.

Looks like this is inevitable. But we should make the downstream developers
aware & make the update easier. As long as it is stated clearly how to
update the code to support Hadoop 3.3, I am okay with that.

Here's what I suggest:
(1) label the jira incompatible (just updated the jira) and updated the
release note to tell app developer how to update.
(2) declare ProtobufHelper a public API HADOOP-17019
<https://issues.apache.org/jira/browse/HADOOP-17019>

Tez doesn't use the removed Token API, but there's code that breaks with
the relocated protobuf class. The ProtobufHelper API will make this
transition much easier.

Other downstreamers that break with the relocated protobuf include: Ozone
and HBase. but neither of them use the removed Token API.


On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
wrote:

> Hi All,
>
>    This mail is to notify about the Removal of following public APIs from
> Hadoop Common.
>
>      ClassName: org.apache.hadoop.security.token.Token
>      APIs:
>          public Token(TokenProto tokenPB);
>          public TokenProto toTokenProto();
>
>    Reason: These APIs are having Generated protobuf classes in the
> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
> release) these APIs are breaking the downstream builds, even though
> downstreams dont use these APIs (just Loading Token class). Downstreams are
> still referencing having older version (2.5.0) of protobuf, hence build is
> being broken.
>
>     These APIs were added for the internal purpose(HADOOP-12563), to
> support serializing tokens using protobuf in UGI Credentials.
> Same purpose can be achieved using the Helper classes without introducing
> protobuf classes in API signatures.
>
> Token.java is marked as Evolving, so I believe APIs can be changed whenever
> absolute necessary.
>
>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> reported to solve downstream build failure.
>
> So since this API was added for internal purpose easy approach to solve
> this is to remove APIs and use helper classes. Otherwise, as mentioned in
> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>
> If anyone using these APIs outside hadoop project accidentally, please
> reply to this mail immediately.
>
> If no objection by next week, will go ahead with removal of above said APIs
> in HADOOP-16621.
>
> -Vinay
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Sheng Liu <li...@gmail.com>.
The  HADOOP-17046 <https://issues.apache.org/jira/browse/HADOOP-17046> has
been merged now, so is it ready to publish the  Hadoop 3.3.0 release ?

Thank you!

Brahma Reddy Battula <br...@apache.org> 于2020年6月4日周四 下午10:27写道:

> Yes, it's blocker for 3.3.0..Just I hold release for issue.
>
> On Tue, Jun 2, 2020 at 7:08 AM Akira Ajisaka <aa...@apache.org> wrote:
>
> > > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > > This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > shading and with protobuf-2.5.0 implementation) to support downstream
> > implementations.
> >
> > Thank you, Vinay. I checked the PR and it mostly looks good.
> > How do we proceed with?
> >
> > I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?
> >
> > Thanks,
> > Akira
> >
> > On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:
> >
> > > ProtobufHelper should not be a public API.  Hadoop uses protobuf
> > > serialization to expertise RPC performance with many drawbacks.  The
> > > generalized object usually require another indirection to map to usable
> > > Java object, this is making Hadoop code messy, and that is topic for
> > > another day.  The main challenges for UGI class is making the system
> > > difficult to secure.
> > >
> > > In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> > > protocol, and secured by JWT token with Google.  This means before
> > > deserializing a protobuf object on the wire, the call must deserialize
> a
> > > JSON token to determine if the call is authenticated before
> deserializing
> > > application objects.  Hence, using protobuf for RPC is no longer a good
> > > reason for performance gain over JSON because JWT token deserialization
> > > happens on every gRPC call to ensure the request is secured properly.
> > >
> > > In Hadoop world, we are not using JWT token for authentication, we have
> > > pluggable token implementation either SPNEGO, delegation token or some
> > kind
> > > of SASL.  UGI class should not allow protobuf token to be exposed as
> > public
> > > interface, otherwise down stream application can forge the protobuf
> token
> > > and it will become a privilege escalation issue.  In my opinion, UGI
> > class
> > > must be as private as possible to prevent forgery.  Down stream
> > application
> > > are discouraged from using UGI.doAs for impersonation to reduce
> > privileges
> > > escalation.  Instead, the downstream application should running like
> Unix
> > > daemon instead of root.  This will ensure that vulnerability for one
> > > application does not spill over security problems to another
> application.
> > > Some people will disagree with the statement because existing
> application
> > > is already written to take advantage of UGI.doAs, such as Hive loading
> > > external table.  Fortunately, Hive provides an option to run without
> > doAs.
> > >
> > > Protobuf is not suitable candidate for security token transport because
> > it
> > > is a strong type transport.  If multiple tokens are transported with
> UGI
> > > protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> > > conversion ambiguity that might create security holes or headache on
> type
> > > casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop
> > Token
> > > as byte array, and default to JSON serializer is probably simpler
> > solution
> > > to keep the system robust without repeating the past mistakes.
> > >
> > > regards,
> > > Eric
> > >
> > > On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <
> vinayakumarb@apache.org>
> > > wrote:
> > >
> > > > Hi Wei-chu and steve,
> > > >
> > > > Thanks for sharing insights.
> > > >
> > > > I have also tried to compile and execute ozone pointing to
> > > > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> > > >
> > > > Other than just the usage of internal protobuf APIs, because of which
> > > > compilation would break, I found another major problem was, the
> > > Hadoop-rpc
> > > > implementations in downstreams which is based on non-shaded Protobuf
> > > > classes.
> > > >
> > > > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > > > 'Message', which its expecting to be of 3.7 version and shaded
> package
> > > > (i.e. o.a.h.thirdparty.*).
> > > >
> > > > So,unless downstreams upgrade their protobuf classes to
> > > 'hadoop-thirdparty'
> > > > this issue will continue to occur, even after solving compilation
> > issues
> > > > due to internal usage of private APIs with protobuf signatures.
> > > >
> > > > I found a possible workaround for this problem.
> > > > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > > >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > > > shading and with protobuf-2.5.0 implementation) to support downstream
> > > > implementations.
> > > >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within
> > Hadoop
> > > > and later projects who wish to upgrade their protobufs to 3.x.
> > > >
> > > > For Ozone compilation:
> > > >   I have submitted to PRs to make preparations to adopt to Hadoop
> 3.3+
> > > > upgrade. These PRs will remove dependency on Hadoop for those
> internal
> > > APIs
> > > > and implemented their own copy in ozone with non-shaded protobuf.
> > > >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > > > <https://github.com/apache/hadoop-ozone/pull/933>2
> > > >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> > > >
> > > > Also, I had run some tests on Ozone after applying these PRs and
> > > > HADOOP-17046 with 3.4.0, tests seems to pass.
> > > >
> > > > Please help review these PRs.
> > > >
> > > > Thanks,
> > > > -Vinay
> > > >
> > > >
> > > > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> > > <stevel@cloudera.com.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Okay.
> > > > >
> > > > > I am not going to be a purist and say "what were they doing -using
> > our
> > > > > private APIs?" because as we all know, with things like UGI tagged
> > > > @private
> > > > > there's been no way to get something is done without getting into
> the
> > > > > private stuff.
> > > > >
> > > > > But why did we do the protobuf changes? So that we could update our
> > > > private
> > > > > copy of protobuf with out breaking every single downstream
> > application.
> > > > The
> > > > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> > > When
> > > > > was that? before hadoop-2.2 shipped? I certainly remember a couple
> of
> > > > weeks
> > > > > were absolutely nothing would build whatsoever, not until every
> > > > downstream
> > > > > project had upgraded to the same version of the library.
> > > > >
> > > > > If you ever want to see an upgrade which makes a guava update seem
> a
> > > > minor
> > > > > detail, protobuf upgrades are it. Hence the shading
> > > > >
> > > > > HBase
> > > > > =====
> > > > >
> > > > > it looks like HBase has been using deep internal stuff. That is,
> > > > > "unfortunate". I think in that world we have to look and say is
> there
> > > > > something specific we can do here to help HBase in a way we could
> > also
> > > > > backport. They shouldn't need those IPC internals.
> > > > >
> > > > > Tez & Tokens
> > > > > ============
> > > > >
> > > > > I didn't know Tez was using those protobuf APIs internally. That
> is,
> > > > > "unfortunate".
> > > > >
> > > > > What is key is this: without us moving those methods things like
> > Spark
> > > > > wouldn't work. And they weren't even using the methods, just trying
> > to
> > > > work
> > > > > with Token for job submission.
> > > > >
> > > > > All Tez should need is a byte array serialization of a token. Given
> > > Token
> > > > > is also Writable, that could be done via WritableUtils in a way
> which
> > > > will
> > > > > also work with older releases.
> > > > >
> > > > > Ozone
> > > > > =====
> > > > >
> > > > > When these were part of/in-sync with the hadoop build there
> wouldn't
> > > have
> > > > > been problems. Now there are. Again, they're going in deep, but
> here
> > > > > clearly to simulate some behaviour. Any way to do that differently?
> > > > >
> > > > > Ratis
> > > > > =====
> > > > >
> > > > > No idea.
> > > > >
> > > > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > > > <weichiu@cloudera.com.invalid
> > > > > >
> > > > > wrote:
> > > > >
> > > > > > Most of the problems are downstream applications using Hadoop's
> > > private
> > > > > > APIs.
> > > > > >
> > > > > > Tez:
> > > > > >
> > > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR
> :
> > > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > > > -------------------------------------------------------------
> > > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > > > incompatible types: com.google.protobuf.ByteString cannot be
> > > converted
> > > > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > > > >
> > > > > >
> > > > > > Tez keeps track of job tokens internally.
> > > > > > The change would look like this:
> > > > > >
> > > > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > > > >   if (stateDb != null) {
> > > > > >     TokenProto tokenProto =
> > ProtobufHelper.protoFromToken(jobToken);
> > > > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > > > >
> >  .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > > > >         .setKind(jobToken.getKind().toString())
> > > > > >         .setService(jobToken.getService().toString())
> > > > > >         .build();*/
> > > > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > > > >         .setUser(user).setJobToken(tokenProto).build();
> > > > > >     try {
> > > > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > > > >     } catch (DBException e) {
> > > > > >       throw new IOException("Error storing " + jobId, e);
> > > > > >     }
> > > > > >   }
> > > > > >   addJobToken(jobId, user, jobToken);
> > > > > > }
> > > > > >
> > > > > >
> > > > > > HBase:
> > > > > >
> > > > > >    1. HBASE-23833 <
> > https://issues.apache.org/jira/browse/HBASE-23833
> > > >
> > > > > > (this
> > > > > >    is recently fixed in the master branch)
> > > > > >    2.
> > > > > >
> > > > > >       [ERROR] Failed to execute goal
> > > > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > > > (default-compile) on project hbase-server: Compilation failure:
> > > > > > Compilation failure:
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > > > >       [ERROR]   class file for
> > > > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > > > cannot access
> > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > > > >       [ERROR]   class file for
> > > > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not
> found
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > > >       [ERROR]   class file for
> > > > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > > > cannot find symbol
> > > > > >       [ERROR]   symbol:   method
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > > >       [ERROR]   location: variable proto of type
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > > > incompatible types:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > > > incompatible types:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > > > cannot access
> > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > > > >       [ERROR]   class file for
> > > > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not
> found
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > > > cannot find symbol
> > > > > >       [ERROR]   symbol:   method
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > > >       [ERROR]   location: variable proto of type
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > > > >       [ERROR]   class file for
> > > > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > > > incompatible types:
> > > > > >
> > > >
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > > >       [ERROR]
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > > > invalid method reference
> > > > > >       [ERROR]   non-static method get() cannot be referenced
> from a
> > > > > > static context
> > > > > >
> > > > > >
> > > > > > Ozone:
> > > > > >
> > > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR
> :
> > > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > > > -------------------------------------------------------------
> > > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > > > incompatible types: com.google.protobuf.ServiceException cannot
> be
> > > > > > converted to
> org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > > > incompatible types: com.google.protobuf.ServiceException cannot
> be
> > > > > > converted to
> org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > > > >
> > > > > >
> > > > > > There's another error where Ozone uses the Hadoop RPC framework
> > which
> > > > > uses
> > > > > > the hadoop.thirdparty protobuf.
> > > > > >
> > > > > > [ERROR] Failed to execute goal
> > > > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > > > Compilation
> > > > > > failure
> > > > > > [ERROR]
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > > > converted
> > > > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > > > >
> > > > > > BlockingService scmDatanodeService =
> > > > > >     StorageContainerDatanodeProtocolService.
> > > > > >         newReflectiveBlockingService(
> > > > > >             new
> > > StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > > > >                 server,
> > Mockito.mock(ProtocolMessageMetrics.class)));
> > > > > >
> > > > > >
> > > > > >
> > > > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > > > framework
> > > > > > too.
> > > > > >
> > > > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > > > vinayakumarb@apache.org>
> > > > > > wrote:
> > > > > >
> > > > > > > hi Wei-Chiu,
> > > > > > >
> > > > > > > Can you elaborate on what failures you are facing related to
> > > > relocated
> > > > > > > protobuf classes.. ?
> > > > > > >
> > > > > > > IFAIK, if the issue with location of protobuf classes, still
> old
> > > jar
> > > > > > > protobuf-2.5.0.jar will be available in classpath. So
> downstream
> > > > > > depending
> > > > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > > > >
> > > > > > > -vinay
> > > > > > >
> > > > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> > > weichiu@cloudera.com
> > > > >
> > > > > > > wrote:
> > > > > > >
> > > > > > >> I'm sorry for coming to this late. I missed this message. It
> > > should
> > > > > have
> > > > > > >> been a DISCUSS thread rather than NOTICE.
> > > > > > >>
> > > > > > >> Looks like this is inevitable. But we should make the
> downstream
> > > > > > >> developers aware & make the update easier. As long as it is
> > stated
> > > > > > clearly
> > > > > > >> how to update the code to support Hadoop 3.3, I am okay with
> > that.
> > > > > > >>
> > > > > > >> Here's what I suggest:
> > > > > > >> (1) label the jira incompatible (just updated the jira) and
> > > updated
> > > > > the
> > > > > > >> release note to tell app developer how to update.
> > > > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > > > >>
> > > > > > >> Tez doesn't use the removed Token API, but there's code that
> > > breaks
> > > > > with
> > > > > > >> the relocated protobuf class. The ProtobufHelper API will make
> > > this
> > > > > > >> transition much easier.
> > > > > > >>
> > > > > > >> Other downstreamers that break with the relocated protobuf
> > > include:
> > > > > > Ozone
> > > > > > >> and HBase. but neither of them use the removed Token API.
> > > > > > >>
> > > > > > >>
> > > > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > > > vinayakumarb@apache.org
> > > > > >
> > > > > > >> wrote:
> > > > > > >>
> > > > > > >>> Hi All,
> > > > > > >>>
> > > > > > >>>    This mail is to notify about the Removal of following
> public
> > > > APIs
> > > > > > from
> > > > > > >>> Hadoop Common.
> > > > > > >>>
> > > > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > > > >>>      APIs:
> > > > > > >>>          public Token(TokenProto tokenPB);
> > > > > > >>>          public TokenProto toTokenProto();
> > > > > > >>>
> > > > > > >>>    Reason: These APIs are having Generated protobuf classes
> in
> > > the
> > > > > > >>> signature. Right now due to protobuf upgrade in trunk (soon
> to
> > be
> > > > > 3.3.0
> > > > > > >>> release) these APIs are breaking the downstream builds, even
> > > though
> > > > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > > > Downstreams
> > > > > > >>> are
> > > > > > >>> still referencing having older version (2.5.0) of protobuf,
> > hence
> > > > > build
> > > > > > >>> is
> > > > > > >>> being broken.
> > > > > > >>>
> > > > > > >>>     These APIs were added for the internal
> > purpose(HADOOP-12563),
> > > > to
> > > > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > > > >>> Same purpose can be achieved using the Helper classes without
> > > > > > introducing
> > > > > > >>> protobuf classes in API signatures.
> > > > > > >>>
> > > > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> > > changed
> > > > > > >>> whenever
> > > > > > >>> absolute necessary.
> > > > > > >>>
> > > > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621
> > has
> > > > been
> > > > > > >>> reported to solve downstream build failure.
> > > > > > >>>
> > > > > > >>> So since this API was added for internal purpose easy
> approach
> > to
> > > > > solve
> > > > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > > > mentioned
> > > > > > in
> > > > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > > > maintained.
> > > > > > >>>
> > > > > > >>> If anyone using these APIs outside hadoop project
> accidentally,
> > > > > please
> > > > > > >>> reply to this mail immediately.
> > > > > > >>>
> > > > > > >>> If no objection by next week, will go ahead with removal of
> > above
> > > > > said
> > > > > > >>> APIs
> > > > > > >>> in HADOOP-16621.
> > > > > > >>>
> > > > > > >>> -Vinay
> > > > > > >>>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>
>
> --
>
>
>
> --Brahma Reddy Battula
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Brahma Reddy Battula <br...@apache.org>.
Yes, it's blocker for 3.3.0..Just I hold release for issue.

On Tue, Jun 2, 2020 at 7:08 AM Akira Ajisaka <aa...@apache.org> wrote:

> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>
> Thank you, Vinay. I checked the PR and it mostly looks good.
> How do we proceed with?
>
> I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?
>
> Thanks,
> Akira
>
> On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:
>
> > ProtobufHelper should not be a public API.  Hadoop uses protobuf
> > serialization to expertise RPC performance with many drawbacks.  The
> > generalized object usually require another indirection to map to usable
> > Java object, this is making Hadoop code messy, and that is topic for
> > another day.  The main challenges for UGI class is making the system
> > difficult to secure.
> >
> > In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> > protocol, and secured by JWT token with Google.  This means before
> > deserializing a protobuf object on the wire, the call must deserialize a
> > JSON token to determine if the call is authenticated before deserializing
> > application objects.  Hence, using protobuf for RPC is no longer a good
> > reason for performance gain over JSON because JWT token deserialization
> > happens on every gRPC call to ensure the request is secured properly.
> >
> > In Hadoop world, we are not using JWT token for authentication, we have
> > pluggable token implementation either SPNEGO, delegation token or some
> kind
> > of SASL.  UGI class should not allow protobuf token to be exposed as
> public
> > interface, otherwise down stream application can forge the protobuf token
> > and it will become a privilege escalation issue.  In my opinion, UGI
> class
> > must be as private as possible to prevent forgery.  Down stream
> application
> > are discouraged from using UGI.doAs for impersonation to reduce
> privileges
> > escalation.  Instead, the downstream application should running like Unix
> > daemon instead of root.  This will ensure that vulnerability for one
> > application does not spill over security problems to another application.
> > Some people will disagree with the statement because existing application
> > is already written to take advantage of UGI.doAs, such as Hive loading
> > external table.  Fortunately, Hive provides an option to run without
> doAs.
> >
> > Protobuf is not suitable candidate for security token transport because
> it
> > is a strong type transport.  If multiple tokens are transported with UGI
> > protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> > conversion ambiguity that might create security holes or headache on type
> > casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop
> Token
> > as byte array, and default to JSON serializer is probably simpler
> solution
> > to keep the system robust without repeating the past mistakes.
> >
> > regards,
> > Eric
> >
> > On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > Hi Wei-chu and steve,
> > >
> > > Thanks for sharing insights.
> > >
> > > I have also tried to compile and execute ozone pointing to
> > > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> > >
> > > Other than just the usage of internal protobuf APIs, because of which
> > > compilation would break, I found another major problem was, the
> > Hadoop-rpc
> > > implementations in downstreams which is based on non-shaded Protobuf
> > > classes.
> > >
> > > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > > 'Message', which its expecting to be of 3.7 version and shaded package
> > > (i.e. o.a.h.thirdparty.*).
> > >
> > > So,unless downstreams upgrade their protobuf classes to
> > 'hadoop-thirdparty'
> > > this issue will continue to occur, even after solving compilation
> issues
> > > due to internal usage of private APIs with protobuf signatures.
> > >
> > > I found a possible workaround for this problem.
> > > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > > shading and with protobuf-2.5.0 implementation) to support downstream
> > > implementations.
> > >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within
> Hadoop
> > > and later projects who wish to upgrade their protobufs to 3.x.
> > >
> > > For Ozone compilation:
> > >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > > upgrade. These PRs will remove dependency on Hadoop for those internal
> > APIs
> > > and implemented their own copy in ozone with non-shaded protobuf.
> > >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > > <https://github.com/apache/hadoop-ozone/pull/933>2
> > >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> > >
> > > Also, I had run some tests on Ozone after applying these PRs and
> > > HADOOP-17046 with 3.4.0, tests seems to pass.
> > >
> > > Please help review these PRs.
> > >
> > > Thanks,
> > > -Vinay
> > >
> > >
> > > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> > <stevel@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Okay.
> > > >
> > > > I am not going to be a purist and say "what were they doing -using
> our
> > > > private APIs?" because as we all know, with things like UGI tagged
> > > @private
> > > > there's been no way to get something is done without getting into the
> > > > private stuff.
> > > >
> > > > But why did we do the protobuf changes? So that we could update our
> > > private
> > > > copy of protobuf with out breaking every single downstream
> application.
> > > The
> > > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> > When
> > > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > > weeks
> > > > were absolutely nothing would build whatsoever, not until every
> > > downstream
> > > > project had upgraded to the same version of the library.
> > > >
> > > > If you ever want to see an upgrade which makes a guava update seem a
> > > minor
> > > > detail, protobuf upgrades are it. Hence the shading
> > > >
> > > > HBase
> > > > =====
> > > >
> > > > it looks like HBase has been using deep internal stuff. That is,
> > > > "unfortunate". I think in that world we have to look and say is there
> > > > something specific we can do here to help HBase in a way we could
> also
> > > > backport. They shouldn't need those IPC internals.
> > > >
> > > > Tez & Tokens
> > > > ============
> > > >
> > > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > > "unfortunate".
> > > >
> > > > What is key is this: without us moving those methods things like
> Spark
> > > > wouldn't work. And they weren't even using the methods, just trying
> to
> > > work
> > > > with Token for job submission.
> > > >
> > > > All Tez should need is a byte array serialization of a token. Given
> > Token
> > > > is also Writable, that could be done via WritableUtils in a way which
> > > will
> > > > also work with older releases.
> > > >
> > > > Ozone
> > > > =====
> > > >
> > > > When these were part of/in-sync with the hadoop build there wouldn't
> > have
> > > > been problems. Now there are. Again, they're going in deep, but here
> > > > clearly to simulate some behaviour. Any way to do that differently?
> > > >
> > > > Ratis
> > > > =====
> > > >
> > > > No idea.
> > > >
> > > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > > <weichiu@cloudera.com.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Most of the problems are downstream applications using Hadoop's
> > private
> > > > > APIs.
> > > > >
> > > > > Tez:
> > > > >
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > > incompatible types: com.google.protobuf.ByteString cannot be
> > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > > >
> > > > >
> > > > > Tez keeps track of job tokens internally.
> > > > > The change would look like this:
> > > > >
> > > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > > >   if (stateDb != null) {
> > > > >     TokenProto tokenProto =
> ProtobufHelper.protoFromToken(jobToken);
> > > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > > >
>  .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > > >         .setKind(jobToken.getKind().toString())
> > > > >         .setService(jobToken.getService().toString())
> > > > >         .build();*/
> > > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > > >         .setUser(user).setJobToken(tokenProto).build();
> > > > >     try {
> > > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > > >     } catch (DBException e) {
> > > > >       throw new IOException("Error storing " + jobId, e);
> > > > >     }
> > > > >   }
> > > > >   addJobToken(jobId, user, jobToken);
> > > > > }
> > > > >
> > > > >
> > > > > HBase:
> > > > >
> > > > >    1. HBASE-23833 <
> https://issues.apache.org/jira/browse/HBASE-23833
> > >
> > > > > (this
> > > > >    is recently fixed in the master branch)
> > > > >    2.
> > > > >
> > > > >       [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > > (default-compile) on project hbase-server: Compilation failure:
> > > > > Compilation failure:
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > > incompatible types:
> > > > >
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > > invalid method reference
> > > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > > static context
> > > > >
> > > > >
> > > > > Ozone:
> > > > >
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > > >
> > > > >
> > > > > There's another error where Ozone uses the Hadoop RPC framework
> which
> > > > uses
> > > > > the hadoop.thirdparty protobuf.
> > > > >
> > > > > [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > > Compilation
> > > > > failure
> > > > > [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > > >
> > > > > BlockingService scmDatanodeService =
> > > > >     StorageContainerDatanodeProtocolService.
> > > > >         newReflectiveBlockingService(
> > > > >             new
> > StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > > >                 server,
> Mockito.mock(ProtocolMessageMetrics.class)));
> > > > >
> > > > >
> > > > >
> > > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > > framework
> > > > > too.
> > > > >
> > > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > > vinayakumarb@apache.org>
> > > > > wrote:
> > > > >
> > > > > > hi Wei-Chiu,
> > > > > >
> > > > > > Can you elaborate on what failures you are facing related to
> > > relocated
> > > > > > protobuf classes.. ?
> > > > > >
> > > > > > IFAIK, if the issue with location of protobuf classes, still old
> > jar
> > > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > > depending
> > > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > > >
> > > > > > -vinay
> > > > > >
> > > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> > weichiu@cloudera.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> I'm sorry for coming to this late. I missed this message. It
> > should
> > > > have
> > > > > >> been a DISCUSS thread rather than NOTICE.
> > > > > >>
> > > > > >> Looks like this is inevitable. But we should make the downstream
> > > > > >> developers aware & make the update easier. As long as it is
> stated
> > > > > clearly
> > > > > >> how to update the code to support Hadoop 3.3, I am okay with
> that.
> > > > > >>
> > > > > >> Here's what I suggest:
> > > > > >> (1) label the jira incompatible (just updated the jira) and
> > updated
> > > > the
> > > > > >> release note to tell app developer how to update.
> > > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > > >>
> > > > > >> Tez doesn't use the removed Token API, but there's code that
> > breaks
> > > > with
> > > > > >> the relocated protobuf class. The ProtobufHelper API will make
> > this
> > > > > >> transition much easier.
> > > > > >>
> > > > > >> Other downstreamers that break with the relocated protobuf
> > include:
> > > > > Ozone
> > > > > >> and HBase. but neither of them use the removed Token API.
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > > vinayakumarb@apache.org
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >>> Hi All,
> > > > > >>>
> > > > > >>>    This mail is to notify about the Removal of following public
> > > APIs
> > > > > from
> > > > > >>> Hadoop Common.
> > > > > >>>
> > > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > > >>>      APIs:
> > > > > >>>          public Token(TokenProto tokenPB);
> > > > > >>>          public TokenProto toTokenProto();
> > > > > >>>
> > > > > >>>    Reason: These APIs are having Generated protobuf classes in
> > the
> > > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to
> be
> > > > 3.3.0
> > > > > >>> release) these APIs are breaking the downstream builds, even
> > though
> > > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > > Downstreams
> > > > > >>> are
> > > > > >>> still referencing having older version (2.5.0) of protobuf,
> hence
> > > > build
> > > > > >>> is
> > > > > >>> being broken.
> > > > > >>>
> > > > > >>>     These APIs were added for the internal
> purpose(HADOOP-12563),
> > > to
> > > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > > >>> Same purpose can be achieved using the Helper classes without
> > > > > introducing
> > > > > >>> protobuf classes in API signatures.
> > > > > >>>
> > > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> > changed
> > > > > >>> whenever
> > > > > >>> absolute necessary.
> > > > > >>>
> > > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621
> has
> > > been
> > > > > >>> reported to solve downstream build failure.
> > > > > >>>
> > > > > >>> So since this API was added for internal purpose easy approach
> to
> > > > solve
> > > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > > mentioned
> > > > > in
> > > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > > maintained.
> > > > > >>>
> > > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > > please
> > > > > >>> reply to this mail immediately.
> > > > > >>>
> > > > > >>> If no objection by next week, will go ahead with removal of
> above
> > > > said
> > > > > >>> APIs
> > > > > >>> in HADOOP-16621.
> > > > > >>>
> > > > > >>> -Vinay
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
>


-- 



--Brahma Reddy Battula

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Brahma Reddy Battula <br...@apache.org>.
Yes, it's blocker for 3.3.0..Just I hold release for issue.

On Tue, Jun 2, 2020 at 7:08 AM Akira Ajisaka <aa...@apache.org> wrote:

> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>
> Thank you, Vinay. I checked the PR and it mostly looks good.
> How do we proceed with?
>
> I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?
>
> Thanks,
> Akira
>
> On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:
>
> > ProtobufHelper should not be a public API.  Hadoop uses protobuf
> > serialization to expertise RPC performance with many drawbacks.  The
> > generalized object usually require another indirection to map to usable
> > Java object, this is making Hadoop code messy, and that is topic for
> > another day.  The main challenges for UGI class is making the system
> > difficult to secure.
> >
> > In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> > protocol, and secured by JWT token with Google.  This means before
> > deserializing a protobuf object on the wire, the call must deserialize a
> > JSON token to determine if the call is authenticated before deserializing
> > application objects.  Hence, using protobuf for RPC is no longer a good
> > reason for performance gain over JSON because JWT token deserialization
> > happens on every gRPC call to ensure the request is secured properly.
> >
> > In Hadoop world, we are not using JWT token for authentication, we have
> > pluggable token implementation either SPNEGO, delegation token or some
> kind
> > of SASL.  UGI class should not allow protobuf token to be exposed as
> public
> > interface, otherwise down stream application can forge the protobuf token
> > and it will become a privilege escalation issue.  In my opinion, UGI
> class
> > must be as private as possible to prevent forgery.  Down stream
> application
> > are discouraged from using UGI.doAs for impersonation to reduce
> privileges
> > escalation.  Instead, the downstream application should running like Unix
> > daemon instead of root.  This will ensure that vulnerability for one
> > application does not spill over security problems to another application.
> > Some people will disagree with the statement because existing application
> > is already written to take advantage of UGI.doAs, such as Hive loading
> > external table.  Fortunately, Hive provides an option to run without
> doAs.
> >
> > Protobuf is not suitable candidate for security token transport because
> it
> > is a strong type transport.  If multiple tokens are transported with UGI
> > protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> > conversion ambiguity that might create security holes or headache on type
> > casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop
> Token
> > as byte array, and default to JSON serializer is probably simpler
> solution
> > to keep the system robust without repeating the past mistakes.
> >
> > regards,
> > Eric
> >
> > On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > Hi Wei-chu and steve,
> > >
> > > Thanks for sharing insights.
> > >
> > > I have also tried to compile and execute ozone pointing to
> > > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> > >
> > > Other than just the usage of internal protobuf APIs, because of which
> > > compilation would break, I found another major problem was, the
> > Hadoop-rpc
> > > implementations in downstreams which is based on non-shaded Protobuf
> > > classes.
> > >
> > > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > > 'Message', which its expecting to be of 3.7 version and shaded package
> > > (i.e. o.a.h.thirdparty.*).
> > >
> > > So,unless downstreams upgrade their protobuf classes to
> > 'hadoop-thirdparty'
> > > this issue will continue to occur, even after solving compilation
> issues
> > > due to internal usage of private APIs with protobuf signatures.
> > >
> > > I found a possible workaround for this problem.
> > > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > > shading and with protobuf-2.5.0 implementation) to support downstream
> > > implementations.
> > >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within
> Hadoop
> > > and later projects who wish to upgrade their protobufs to 3.x.
> > >
> > > For Ozone compilation:
> > >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > > upgrade. These PRs will remove dependency on Hadoop for those internal
> > APIs
> > > and implemented their own copy in ozone with non-shaded protobuf.
> > >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > > <https://github.com/apache/hadoop-ozone/pull/933>2
> > >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> > >
> > > Also, I had run some tests on Ozone after applying these PRs and
> > > HADOOP-17046 with 3.4.0, tests seems to pass.
> > >
> > > Please help review these PRs.
> > >
> > > Thanks,
> > > -Vinay
> > >
> > >
> > > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> > <stevel@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Okay.
> > > >
> > > > I am not going to be a purist and say "what were they doing -using
> our
> > > > private APIs?" because as we all know, with things like UGI tagged
> > > @private
> > > > there's been no way to get something is done without getting into the
> > > > private stuff.
> > > >
> > > > But why did we do the protobuf changes? So that we could update our
> > > private
> > > > copy of protobuf with out breaking every single downstream
> application.
> > > The
> > > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> > When
> > > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > > weeks
> > > > were absolutely nothing would build whatsoever, not until every
> > > downstream
> > > > project had upgraded to the same version of the library.
> > > >
> > > > If you ever want to see an upgrade which makes a guava update seem a
> > > minor
> > > > detail, protobuf upgrades are it. Hence the shading
> > > >
> > > > HBase
> > > > =====
> > > >
> > > > it looks like HBase has been using deep internal stuff. That is,
> > > > "unfortunate". I think in that world we have to look and say is there
> > > > something specific we can do here to help HBase in a way we could
> also
> > > > backport. They shouldn't need those IPC internals.
> > > >
> > > > Tez & Tokens
> > > > ============
> > > >
> > > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > > "unfortunate".
> > > >
> > > > What is key is this: without us moving those methods things like
> Spark
> > > > wouldn't work. And they weren't even using the methods, just trying
> to
> > > work
> > > > with Token for job submission.
> > > >
> > > > All Tez should need is a byte array serialization of a token. Given
> > Token
> > > > is also Writable, that could be done via WritableUtils in a way which
> > > will
> > > > also work with older releases.
> > > >
> > > > Ozone
> > > > =====
> > > >
> > > > When these were part of/in-sync with the hadoop build there wouldn't
> > have
> > > > been problems. Now there are. Again, they're going in deep, but here
> > > > clearly to simulate some behaviour. Any way to do that differently?
> > > >
> > > > Ratis
> > > > =====
> > > >
> > > > No idea.
> > > >
> > > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > > <weichiu@cloudera.com.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Most of the problems are downstream applications using Hadoop's
> > private
> > > > > APIs.
> > > > >
> > > > > Tez:
> > > > >
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > > incompatible types: com.google.protobuf.ByteString cannot be
> > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > > >
> > > > >
> > > > > Tez keeps track of job tokens internally.
> > > > > The change would look like this:
> > > > >
> > > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > > >   if (stateDb != null) {
> > > > >     TokenProto tokenProto =
> ProtobufHelper.protoFromToken(jobToken);
> > > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > > >
>  .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > > >         .setKind(jobToken.getKind().toString())
> > > > >         .setService(jobToken.getService().toString())
> > > > >         .build();*/
> > > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > > >         .setUser(user).setJobToken(tokenProto).build();
> > > > >     try {
> > > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > > >     } catch (DBException e) {
> > > > >       throw new IOException("Error storing " + jobId, e);
> > > > >     }
> > > > >   }
> > > > >   addJobToken(jobId, user, jobToken);
> > > > > }
> > > > >
> > > > >
> > > > > HBase:
> > > > >
> > > > >    1. HBASE-23833 <
> https://issues.apache.org/jira/browse/HBASE-23833
> > >
> > > > > (this
> > > > >    is recently fixed in the master branch)
> > > > >    2.
> > > > >
> > > > >       [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > > (default-compile) on project hbase-server: Compilation failure:
> > > > > Compilation failure:
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > > incompatible types:
> > > > >
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > > invalid method reference
> > > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > > static context
> > > > >
> > > > >
> > > > > Ozone:
> > > > >
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > > >
> > > > >
> > > > > There's another error where Ozone uses the Hadoop RPC framework
> which
> > > > uses
> > > > > the hadoop.thirdparty protobuf.
> > > > >
> > > > > [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > > Compilation
> > > > > failure
> > > > > [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > > >
> > > > > BlockingService scmDatanodeService =
> > > > >     StorageContainerDatanodeProtocolService.
> > > > >         newReflectiveBlockingService(
> > > > >             new
> > StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > > >                 server,
> Mockito.mock(ProtocolMessageMetrics.class)));
> > > > >
> > > > >
> > > > >
> > > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > > framework
> > > > > too.
> > > > >
> > > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > > vinayakumarb@apache.org>
> > > > > wrote:
> > > > >
> > > > > > hi Wei-Chiu,
> > > > > >
> > > > > > Can you elaborate on what failures you are facing related to
> > > relocated
> > > > > > protobuf classes.. ?
> > > > > >
> > > > > > IFAIK, if the issue with location of protobuf classes, still old
> > jar
> > > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > > depending
> > > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > > >
> > > > > > -vinay
> > > > > >
> > > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> > weichiu@cloudera.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> I'm sorry for coming to this late. I missed this message. It
> > should
> > > > have
> > > > > >> been a DISCUSS thread rather than NOTICE.
> > > > > >>
> > > > > >> Looks like this is inevitable. But we should make the downstream
> > > > > >> developers aware & make the update easier. As long as it is
> stated
> > > > > clearly
> > > > > >> how to update the code to support Hadoop 3.3, I am okay with
> that.
> > > > > >>
> > > > > >> Here's what I suggest:
> > > > > >> (1) label the jira incompatible (just updated the jira) and
> > updated
> > > > the
> > > > > >> release note to tell app developer how to update.
> > > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > > >>
> > > > > >> Tez doesn't use the removed Token API, but there's code that
> > breaks
> > > > with
> > > > > >> the relocated protobuf class. The ProtobufHelper API will make
> > this
> > > > > >> transition much easier.
> > > > > >>
> > > > > >> Other downstreamers that break with the relocated protobuf
> > include:
> > > > > Ozone
> > > > > >> and HBase. but neither of them use the removed Token API.
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > > vinayakumarb@apache.org
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >>> Hi All,
> > > > > >>>
> > > > > >>>    This mail is to notify about the Removal of following public
> > > APIs
> > > > > from
> > > > > >>> Hadoop Common.
> > > > > >>>
> > > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > > >>>      APIs:
> > > > > >>>          public Token(TokenProto tokenPB);
> > > > > >>>          public TokenProto toTokenProto();
> > > > > >>>
> > > > > >>>    Reason: These APIs are having Generated protobuf classes in
> > the
> > > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to
> be
> > > > 3.3.0
> > > > > >>> release) these APIs are breaking the downstream builds, even
> > though
> > > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > > Downstreams
> > > > > >>> are
> > > > > >>> still referencing having older version (2.5.0) of protobuf,
> hence
> > > > build
> > > > > >>> is
> > > > > >>> being broken.
> > > > > >>>
> > > > > >>>     These APIs were added for the internal
> purpose(HADOOP-12563),
> > > to
> > > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > > >>> Same purpose can be achieved using the Helper classes without
> > > > > introducing
> > > > > >>> protobuf classes in API signatures.
> > > > > >>>
> > > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> > changed
> > > > > >>> whenever
> > > > > >>> absolute necessary.
> > > > > >>>
> > > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621
> has
> > > been
> > > > > >>> reported to solve downstream build failure.
> > > > > >>>
> > > > > >>> So since this API was added for internal purpose easy approach
> to
> > > > solve
> > > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > > mentioned
> > > > > in
> > > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > > maintained.
> > > > > >>>
> > > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > > please
> > > > > >>> reply to this mail immediately.
> > > > > >>>
> > > > > >>> If no objection by next week, will go ahead with removal of
> above
> > > > said
> > > > > >>> APIs
> > > > > >>> in HADOOP-16621.
> > > > > >>>
> > > > > >>> -Vinay
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
>


-- 



--Brahma Reddy Battula

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Brahma Reddy Battula <br...@apache.org>.
Yes, it's blocker for 3.3.0..Just I hold release for issue.

On Tue, Jun 2, 2020 at 7:08 AM Akira Ajisaka <aa...@apache.org> wrote:

> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>
> Thank you, Vinay. I checked the PR and it mostly looks good.
> How do we proceed with?
>
> I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?
>
> Thanks,
> Akira
>
> On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:
>
> > ProtobufHelper should not be a public API.  Hadoop uses protobuf
> > serialization to expertise RPC performance with many drawbacks.  The
> > generalized object usually require another indirection to map to usable
> > Java object, this is making Hadoop code messy, and that is topic for
> > another day.  The main challenges for UGI class is making the system
> > difficult to secure.
> >
> > In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> > protocol, and secured by JWT token with Google.  This means before
> > deserializing a protobuf object on the wire, the call must deserialize a
> > JSON token to determine if the call is authenticated before deserializing
> > application objects.  Hence, using protobuf for RPC is no longer a good
> > reason for performance gain over JSON because JWT token deserialization
> > happens on every gRPC call to ensure the request is secured properly.
> >
> > In Hadoop world, we are not using JWT token for authentication, we have
> > pluggable token implementation either SPNEGO, delegation token or some
> kind
> > of SASL.  UGI class should not allow protobuf token to be exposed as
> public
> > interface, otherwise down stream application can forge the protobuf token
> > and it will become a privilege escalation issue.  In my opinion, UGI
> class
> > must be as private as possible to prevent forgery.  Down stream
> application
> > are discouraged from using UGI.doAs for impersonation to reduce
> privileges
> > escalation.  Instead, the downstream application should running like Unix
> > daemon instead of root.  This will ensure that vulnerability for one
> > application does not spill over security problems to another application.
> > Some people will disagree with the statement because existing application
> > is already written to take advantage of UGI.doAs, such as Hive loading
> > external table.  Fortunately, Hive provides an option to run without
> doAs.
> >
> > Protobuf is not suitable candidate for security token transport because
> it
> > is a strong type transport.  If multiple tokens are transported with UGI
> > protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> > conversion ambiguity that might create security holes or headache on type
> > casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop
> Token
> > as byte array, and default to JSON serializer is probably simpler
> solution
> > to keep the system robust without repeating the past mistakes.
> >
> > regards,
> > Eric
> >
> > On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > Hi Wei-chu and steve,
> > >
> > > Thanks for sharing insights.
> > >
> > > I have also tried to compile and execute ozone pointing to
> > > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> > >
> > > Other than just the usage of internal protobuf APIs, because of which
> > > compilation would break, I found another major problem was, the
> > Hadoop-rpc
> > > implementations in downstreams which is based on non-shaded Protobuf
> > > classes.
> > >
> > > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > > 'Message', which its expecting to be of 3.7 version and shaded package
> > > (i.e. o.a.h.thirdparty.*).
> > >
> > > So,unless downstreams upgrade their protobuf classes to
> > 'hadoop-thirdparty'
> > > this issue will continue to occur, even after solving compilation
> issues
> > > due to internal usage of private APIs with protobuf signatures.
> > >
> > > I found a possible workaround for this problem.
> > > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> > >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > > shading and with protobuf-2.5.0 implementation) to support downstream
> > > implementations.
> > >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within
> Hadoop
> > > and later projects who wish to upgrade their protobufs to 3.x.
> > >
> > > For Ozone compilation:
> > >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > > upgrade. These PRs will remove dependency on Hadoop for those internal
> > APIs
> > > and implemented their own copy in ozone with non-shaded protobuf.
> > >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > > <https://github.com/apache/hadoop-ozone/pull/933>2
> > >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> > >
> > > Also, I had run some tests on Ozone after applying these PRs and
> > > HADOOP-17046 with 3.4.0, tests seems to pass.
> > >
> > > Please help review these PRs.
> > >
> > > Thanks,
> > > -Vinay
> > >
> > >
> > > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> > <stevel@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Okay.
> > > >
> > > > I am not going to be a purist and say "what were they doing -using
> our
> > > > private APIs?" because as we all know, with things like UGI tagged
> > > @private
> > > > there's been no way to get something is done without getting into the
> > > > private stuff.
> > > >
> > > > But why did we do the protobuf changes? So that we could update our
> > > private
> > > > copy of protobuf with out breaking every single downstream
> application.
> > > The
> > > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> > When
> > > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > > weeks
> > > > were absolutely nothing would build whatsoever, not until every
> > > downstream
> > > > project had upgraded to the same version of the library.
> > > >
> > > > If you ever want to see an upgrade which makes a guava update seem a
> > > minor
> > > > detail, protobuf upgrades are it. Hence the shading
> > > >
> > > > HBase
> > > > =====
> > > >
> > > > it looks like HBase has been using deep internal stuff. That is,
> > > > "unfortunate". I think in that world we have to look and say is there
> > > > something specific we can do here to help HBase in a way we could
> also
> > > > backport. They shouldn't need those IPC internals.
> > > >
> > > > Tez & Tokens
> > > > ============
> > > >
> > > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > > "unfortunate".
> > > >
> > > > What is key is this: without us moving those methods things like
> Spark
> > > > wouldn't work. And they weren't even using the methods, just trying
> to
> > > work
> > > > with Token for job submission.
> > > >
> > > > All Tez should need is a byte array serialization of a token. Given
> > Token
> > > > is also Writable, that could be done via WritableUtils in a way which
> > > will
> > > > also work with older releases.
> > > >
> > > > Ozone
> > > > =====
> > > >
> > > > When these were part of/in-sync with the hadoop build there wouldn't
> > have
> > > > been problems. Now there are. Again, they're going in deep, but here
> > > > clearly to simulate some behaviour. Any way to do that differently?
> > > >
> > > > Ratis
> > > > =====
> > > >
> > > > No idea.
> > > >
> > > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > > <weichiu@cloudera.com.invalid
> > > > >
> > > > wrote:
> > > >
> > > > > Most of the problems are downstream applications using Hadoop's
> > private
> > > > > APIs.
> > > > >
> > > > > Tez:
> > > > >
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > > incompatible types: com.google.protobuf.ByteString cannot be
> > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > > >
> > > > >
> > > > > Tez keeps track of job tokens internally.
> > > > > The change would look like this:
> > > > >
> > > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > > >   if (stateDb != null) {
> > > > >     TokenProto tokenProto =
> ProtobufHelper.protoFromToken(jobToken);
> > > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > > >
>  .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > > >         .setKind(jobToken.getKind().toString())
> > > > >         .setService(jobToken.getService().toString())
> > > > >         .build();*/
> > > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > > >         .setUser(user).setJobToken(tokenProto).build();
> > > > >     try {
> > > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > > >     } catch (DBException e) {
> > > > >       throw new IOException("Error storing " + jobId, e);
> > > > >     }
> > > > >   }
> > > > >   addJobToken(jobId, user, jobToken);
> > > > > }
> > > > >
> > > > >
> > > > > HBase:
> > > > >
> > > > >    1. HBASE-23833 <
> https://issues.apache.org/jira/browse/HBASE-23833
> > >
> > > > > (this
> > > > >    is recently fixed in the master branch)
> > > > >    2.
> > > > >
> > > > >       [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > > (default-compile) on project hbase-server: Compilation failure:
> > > > > Compilation failure:
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > > incompatible types:
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > > cannot access
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > > cannot find symbol
> > > > >       [ERROR]   symbol:   method
> > > > >
> > > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > > >       [ERROR]   location: variable proto of type
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > > >       [ERROR]   class file for
> > > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > > incompatible types:
> > > > >
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > > cannot be converted to com.google.protobuf.MessageLite
> > > > >       [ERROR]
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > > invalid method reference
> > > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > > static context
> > > > >
> > > > >
> > > > > Ozone:
> > > > >
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > > -------------------------------------------------------------
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > > >
> > > > >
> > > > > There's another error where Ozone uses the Hadoop RPC framework
> which
> > > > uses
> > > > > the hadoop.thirdparty protobuf.
> > > > >
> > > > > [ERROR] Failed to execute goal
> > > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > > Compilation
> > > > > failure
> > > > > [ERROR]
> > > > >
> > > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > > converted
> > > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > > >
> > > > > BlockingService scmDatanodeService =
> > > > >     StorageContainerDatanodeProtocolService.
> > > > >         newReflectiveBlockingService(
> > > > >             new
> > StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > > >                 server,
> Mockito.mock(ProtocolMessageMetrics.class)));
> > > > >
> > > > >
> > > > >
> > > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > > framework
> > > > > too.
> > > > >
> > > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > > vinayakumarb@apache.org>
> > > > > wrote:
> > > > >
> > > > > > hi Wei-Chiu,
> > > > > >
> > > > > > Can you elaborate on what failures you are facing related to
> > > relocated
> > > > > > protobuf classes.. ?
> > > > > >
> > > > > > IFAIK, if the issue with location of protobuf classes, still old
> > jar
> > > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > > depending
> > > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > > >
> > > > > > -vinay
> > > > > >
> > > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> > weichiu@cloudera.com
> > > >
> > > > > > wrote:
> > > > > >
> > > > > >> I'm sorry for coming to this late. I missed this message. It
> > should
> > > > have
> > > > > >> been a DISCUSS thread rather than NOTICE.
> > > > > >>
> > > > > >> Looks like this is inevitable. But we should make the downstream
> > > > > >> developers aware & make the update easier. As long as it is
> stated
> > > > > clearly
> > > > > >> how to update the code to support Hadoop 3.3, I am okay with
> that.
> > > > > >>
> > > > > >> Here's what I suggest:
> > > > > >> (1) label the jira incompatible (just updated the jira) and
> > updated
> > > > the
> > > > > >> release note to tell app developer how to update.
> > > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > > >>
> > > > > >> Tez doesn't use the removed Token API, but there's code that
> > breaks
> > > > with
> > > > > >> the relocated protobuf class. The ProtobufHelper API will make
> > this
> > > > > >> transition much easier.
> > > > > >>
> > > > > >> Other downstreamers that break with the relocated protobuf
> > include:
> > > > > Ozone
> > > > > >> and HBase. but neither of them use the removed Token API.
> > > > > >>
> > > > > >>
> > > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > > vinayakumarb@apache.org
> > > > >
> > > > > >> wrote:
> > > > > >>
> > > > > >>> Hi All,
> > > > > >>>
> > > > > >>>    This mail is to notify about the Removal of following public
> > > APIs
> > > > > from
> > > > > >>> Hadoop Common.
> > > > > >>>
> > > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > > >>>      APIs:
> > > > > >>>          public Token(TokenProto tokenPB);
> > > > > >>>          public TokenProto toTokenProto();
> > > > > >>>
> > > > > >>>    Reason: These APIs are having Generated protobuf classes in
> > the
> > > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to
> be
> > > > 3.3.0
> > > > > >>> release) these APIs are breaking the downstream builds, even
> > though
> > > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > > Downstreams
> > > > > >>> are
> > > > > >>> still referencing having older version (2.5.0) of protobuf,
> hence
> > > > build
> > > > > >>> is
> > > > > >>> being broken.
> > > > > >>>
> > > > > >>>     These APIs were added for the internal
> purpose(HADOOP-12563),
> > > to
> > > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > > >>> Same purpose can be achieved using the Helper classes without
> > > > > introducing
> > > > > >>> protobuf classes in API signatures.
> > > > > >>>
> > > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> > changed
> > > > > >>> whenever
> > > > > >>> absolute necessary.
> > > > > >>>
> > > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621
> has
> > > been
> > > > > >>> reported to solve downstream build failure.
> > > > > >>>
> > > > > >>> So since this API was added for internal purpose easy approach
> to
> > > > solve
> > > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > > mentioned
> > > > > in
> > > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > > maintained.
> > > > > >>>
> > > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > > please
> > > > > >>> reply to this mail immediately.
> > > > > >>>
> > > > > >>> If no objection by next week, will go ahead with removal of
> above
> > > > said
> > > > > >>> APIs
> > > > > >>> in HADOOP-16621.
> > > > > >>>
> > > > > >>> -Vinay
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
>


-- 



--Brahma Reddy Battula

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Akira Ajisaka <aa...@apache.org>.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
> This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.

Thank you, Vinay. I checked the PR and it mostly looks good.
How do we proceed with?

I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?

Thanks,
Akira

On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:

> ProtobufHelper should not be a public API.  Hadoop uses protobuf
> serialization to expertise RPC performance with many drawbacks.  The
> generalized object usually require another indirection to map to usable
> Java object, this is making Hadoop code messy, and that is topic for
> another day.  The main challenges for UGI class is making the system
> difficult to secure.
>
> In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> protocol, and secured by JWT token with Google.  This means before
> deserializing a protobuf object on the wire, the call must deserialize a
> JSON token to determine if the call is authenticated before deserializing
> application objects.  Hence, using protobuf for RPC is no longer a good
> reason for performance gain over JSON because JWT token deserialization
> happens on every gRPC call to ensure the request is secured properly.
>
> In Hadoop world, we are not using JWT token for authentication, we have
> pluggable token implementation either SPNEGO, delegation token or some kind
> of SASL.  UGI class should not allow protobuf token to be exposed as public
> interface, otherwise down stream application can forge the protobuf token
> and it will become a privilege escalation issue.  In my opinion, UGI class
> must be as private as possible to prevent forgery.  Down stream application
> are discouraged from using UGI.doAs for impersonation to reduce privileges
> escalation.  Instead, the downstream application should running like Unix
> daemon instead of root.  This will ensure that vulnerability for one
> application does not spill over security problems to another application.
> Some people will disagree with the statement because existing application
> is already written to take advantage of UGI.doAs, such as Hive loading
> external table.  Fortunately, Hive provides an option to run without doAs.
>
> Protobuf is not suitable candidate for security token transport because it
> is a strong type transport.  If multiple tokens are transported with UGI
> protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> conversion ambiguity that might create security holes or headache on type
> casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
> as byte array, and default to JSON serializer is probably simpler solution
> to keep the system robust without repeating the past mistakes.
>
> regards,
> Eric
>
> On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > Hi Wei-chu and steve,
> >
> > Thanks for sharing insights.
> >
> > I have also tried to compile and execute ozone pointing to
> > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> >
> > Other than just the usage of internal protobuf APIs, because of which
> > compilation would break, I found another major problem was, the
> Hadoop-rpc
> > implementations in downstreams which is based on non-shaded Protobuf
> > classes.
> >
> > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > 'Message', which its expecting to be of 3.7 version and shaded package
> > (i.e. o.a.h.thirdparty.*).
> >
> > So,unless downstreams upgrade their protobuf classes to
> 'hadoop-thirdparty'
> > this issue will continue to occur, even after solving compilation issues
> > due to internal usage of private APIs with protobuf signatures.
> >
> > I found a possible workaround for this problem.
> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > shading and with protobuf-2.5.0 implementation) to support downstream
> > implementations.
> >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> > and later projects who wish to upgrade their protobufs to 3.x.
> >
> > For Ozone compilation:
> >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > upgrade. These PRs will remove dependency on Hadoop for those internal
> APIs
> > and implemented their own copy in ozone with non-shaded protobuf.
> >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > <https://github.com/apache/hadoop-ozone/pull/933>2
> >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> >
> > Also, I had run some tests on Ozone after applying these PRs and
> > HADOOP-17046 with 3.4.0, tests seems to pass.
> >
> > Please help review these PRs.
> >
> > Thanks,
> > -Vinay
> >
> >
> > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> <stevel@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Okay.
> > >
> > > I am not going to be a purist and say "what were they doing -using our
> > > private APIs?" because as we all know, with things like UGI tagged
> > @private
> > > there's been no way to get something is done without getting into the
> > > private stuff.
> > >
> > > But why did we do the protobuf changes? So that we could update our
> > private
> > > copy of protobuf with out breaking every single downstream application.
> > The
> > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> When
> > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > weeks
> > > were absolutely nothing would build whatsoever, not until every
> > downstream
> > > project had upgraded to the same version of the library.
> > >
> > > If you ever want to see an upgrade which makes a guava update seem a
> > minor
> > > detail, protobuf upgrades are it. Hence the shading
> > >
> > > HBase
> > > =====
> > >
> > > it looks like HBase has been using deep internal stuff. That is,
> > > "unfortunate". I think in that world we have to look and say is there
> > > something specific we can do here to help HBase in a way we could also
> > > backport. They shouldn't need those IPC internals.
> > >
> > > Tez & Tokens
> > > ============
> > >
> > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > "unfortunate".
> > >
> > > What is key is this: without us moving those methods things like Spark
> > > wouldn't work. And they weren't even using the methods, just trying to
> > work
> > > with Token for job submission.
> > >
> > > All Tez should need is a byte array serialization of a token. Given
> Token
> > > is also Writable, that could be done via WritableUtils in a way which
> > will
> > > also work with older releases.
> > >
> > > Ozone
> > > =====
> > >
> > > When these were part of/in-sync with the hadoop build there wouldn't
> have
> > > been problems. Now there are. Again, they're going in deep, but here
> > > clearly to simulate some behaviour. Any way to do that differently?
> > >
> > > Ratis
> > > =====
> > >
> > > No idea.
> > >
> > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > <weichiu@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Most of the problems are downstream applications using Hadoop's
> private
> > > > APIs.
> > > >
> > > > Tez:
> > > >
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > incompatible types: com.google.protobuf.ByteString cannot be
> converted
> > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > >
> > > >
> > > > Tez keeps track of job tokens internally.
> > > > The change would look like this:
> > > >
> > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > >   if (stateDb != null) {
> > > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > >         .setKind(jobToken.getKind().toString())
> > > >         .setService(jobToken.getService().toString())
> > > >         .build();*/
> > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > >         .setUser(user).setJobToken(tokenProto).build();
> > > >     try {
> > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > >     } catch (DBException e) {
> > > >       throw new IOException("Error storing " + jobId, e);
> > > >     }
> > > >   }
> > > >   addJobToken(jobId, user, jobToken);
> > > > }
> > > >
> > > >
> > > > HBase:
> > > >
> > > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833
> >
> > > > (this
> > > >    is recently fixed in the master branch)
> > > >    2.
> > > >
> > > >       [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > (default-compile) on project hbase-server: Compilation failure:
> > > > Compilation failure:
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > incompatible types:
> > > >
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > invalid method reference
> > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > static context
> > > >
> > > >
> > > > Ozone:
> > > >
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > >
> > > >
> > > > There's another error where Ozone uses the Hadoop RPC framework which
> > > uses
> > > > the hadoop.thirdparty protobuf.
> > > >
> > > > [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > Compilation
> > > > failure
> > > > [ERROR]
> > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > converted
> > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > >
> > > > BlockingService scmDatanodeService =
> > > >     StorageContainerDatanodeProtocolService.
> > > >         newReflectiveBlockingService(
> > > >             new
> StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > > >
> > > >
> > > >
> > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > framework
> > > > too.
> > > >
> > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > vinayakumarb@apache.org>
> > > > wrote:
> > > >
> > > > > hi Wei-Chiu,
> > > > >
> > > > > Can you elaborate on what failures you are facing related to
> > relocated
> > > > > protobuf classes.. ?
> > > > >
> > > > > IFAIK, if the issue with location of protobuf classes, still old
> jar
> > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > depending
> > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > >
> > > > > -vinay
> > > > >
> > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> weichiu@cloudera.com
> > >
> > > > > wrote:
> > > > >
> > > > >> I'm sorry for coming to this late. I missed this message. It
> should
> > > have
> > > > >> been a DISCUSS thread rather than NOTICE.
> > > > >>
> > > > >> Looks like this is inevitable. But we should make the downstream
> > > > >> developers aware & make the update easier. As long as it is stated
> > > > clearly
> > > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > > >>
> > > > >> Here's what I suggest:
> > > > >> (1) label the jira incompatible (just updated the jira) and
> updated
> > > the
> > > > >> release note to tell app developer how to update.
> > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > >>
> > > > >> Tez doesn't use the removed Token API, but there's code that
> breaks
> > > with
> > > > >> the relocated protobuf class. The ProtobufHelper API will make
> this
> > > > >> transition much easier.
> > > > >>
> > > > >> Other downstreamers that break with the relocated protobuf
> include:
> > > > Ozone
> > > > >> and HBase. but neither of them use the removed Token API.
> > > > >>
> > > > >>
> > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > vinayakumarb@apache.org
> > > >
> > > > >> wrote:
> > > > >>
> > > > >>> Hi All,
> > > > >>>
> > > > >>>    This mail is to notify about the Removal of following public
> > APIs
> > > > from
> > > > >>> Hadoop Common.
> > > > >>>
> > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > >>>      APIs:
> > > > >>>          public Token(TokenProto tokenPB);
> > > > >>>          public TokenProto toTokenProto();
> > > > >>>
> > > > >>>    Reason: These APIs are having Generated protobuf classes in
> the
> > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > > 3.3.0
> > > > >>> release) these APIs are breaking the downstream builds, even
> though
> > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > Downstreams
> > > > >>> are
> > > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > > build
> > > > >>> is
> > > > >>> being broken.
> > > > >>>
> > > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> > to
> > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > >>> Same purpose can be achieved using the Helper classes without
> > > > introducing
> > > > >>> protobuf classes in API signatures.
> > > > >>>
> > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> changed
> > > > >>> whenever
> > > > >>> absolute necessary.
> > > > >>>
> > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> > been
> > > > >>> reported to solve downstream build failure.
> > > > >>>
> > > > >>> So since this API was added for internal purpose easy approach to
> > > solve
> > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > mentioned
> > > > in
> > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > maintained.
> > > > >>>
> > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > please
> > > > >>> reply to this mail immediately.
> > > > >>>
> > > > >>> If no objection by next week, will go ahead with removal of above
> > > said
> > > > >>> APIs
> > > > >>> in HADOOP-16621.
> > > > >>>
> > > > >>> -Vinay
> > > > >>>
> > > > >>
> > > >
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Akira Ajisaka <aa...@apache.org>.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
> This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.

Thank you, Vinay. I checked the PR and it mostly looks good.
How do we proceed with?

I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?

Thanks,
Akira

On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:

> ProtobufHelper should not be a public API.  Hadoop uses protobuf
> serialization to expertise RPC performance with many drawbacks.  The
> generalized object usually require another indirection to map to usable
> Java object, this is making Hadoop code messy, and that is topic for
> another day.  The main challenges for UGI class is making the system
> difficult to secure.
>
> In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> protocol, and secured by JWT token with Google.  This means before
> deserializing a protobuf object on the wire, the call must deserialize a
> JSON token to determine if the call is authenticated before deserializing
> application objects.  Hence, using protobuf for RPC is no longer a good
> reason for performance gain over JSON because JWT token deserialization
> happens on every gRPC call to ensure the request is secured properly.
>
> In Hadoop world, we are not using JWT token for authentication, we have
> pluggable token implementation either SPNEGO, delegation token or some kind
> of SASL.  UGI class should not allow protobuf token to be exposed as public
> interface, otherwise down stream application can forge the protobuf token
> and it will become a privilege escalation issue.  In my opinion, UGI class
> must be as private as possible to prevent forgery.  Down stream application
> are discouraged from using UGI.doAs for impersonation to reduce privileges
> escalation.  Instead, the downstream application should running like Unix
> daemon instead of root.  This will ensure that vulnerability for one
> application does not spill over security problems to another application.
> Some people will disagree with the statement because existing application
> is already written to take advantage of UGI.doAs, such as Hive loading
> external table.  Fortunately, Hive provides an option to run without doAs.
>
> Protobuf is not suitable candidate for security token transport because it
> is a strong type transport.  If multiple tokens are transported with UGI
> protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> conversion ambiguity that might create security holes or headache on type
> casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
> as byte array, and default to JSON serializer is probably simpler solution
> to keep the system robust without repeating the past mistakes.
>
> regards,
> Eric
>
> On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > Hi Wei-chu and steve,
> >
> > Thanks for sharing insights.
> >
> > I have also tried to compile and execute ozone pointing to
> > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> >
> > Other than just the usage of internal protobuf APIs, because of which
> > compilation would break, I found another major problem was, the
> Hadoop-rpc
> > implementations in downstreams which is based on non-shaded Protobuf
> > classes.
> >
> > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > 'Message', which its expecting to be of 3.7 version and shaded package
> > (i.e. o.a.h.thirdparty.*).
> >
> > So,unless downstreams upgrade their protobuf classes to
> 'hadoop-thirdparty'
> > this issue will continue to occur, even after solving compilation issues
> > due to internal usage of private APIs with protobuf signatures.
> >
> > I found a possible workaround for this problem.
> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > shading and with protobuf-2.5.0 implementation) to support downstream
> > implementations.
> >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> > and later projects who wish to upgrade their protobufs to 3.x.
> >
> > For Ozone compilation:
> >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > upgrade. These PRs will remove dependency on Hadoop for those internal
> APIs
> > and implemented their own copy in ozone with non-shaded protobuf.
> >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > <https://github.com/apache/hadoop-ozone/pull/933>2
> >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> >
> > Also, I had run some tests on Ozone after applying these PRs and
> > HADOOP-17046 with 3.4.0, tests seems to pass.
> >
> > Please help review these PRs.
> >
> > Thanks,
> > -Vinay
> >
> >
> > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> <stevel@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Okay.
> > >
> > > I am not going to be a purist and say "what were they doing -using our
> > > private APIs?" because as we all know, with things like UGI tagged
> > @private
> > > there's been no way to get something is done without getting into the
> > > private stuff.
> > >
> > > But why did we do the protobuf changes? So that we could update our
> > private
> > > copy of protobuf with out breaking every single downstream application.
> > The
> > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> When
> > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > weeks
> > > were absolutely nothing would build whatsoever, not until every
> > downstream
> > > project had upgraded to the same version of the library.
> > >
> > > If you ever want to see an upgrade which makes a guava update seem a
> > minor
> > > detail, protobuf upgrades are it. Hence the shading
> > >
> > > HBase
> > > =====
> > >
> > > it looks like HBase has been using deep internal stuff. That is,
> > > "unfortunate". I think in that world we have to look and say is there
> > > something specific we can do here to help HBase in a way we could also
> > > backport. They shouldn't need those IPC internals.
> > >
> > > Tez & Tokens
> > > ============
> > >
> > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > "unfortunate".
> > >
> > > What is key is this: without us moving those methods things like Spark
> > > wouldn't work. And they weren't even using the methods, just trying to
> > work
> > > with Token for job submission.
> > >
> > > All Tez should need is a byte array serialization of a token. Given
> Token
> > > is also Writable, that could be done via WritableUtils in a way which
> > will
> > > also work with older releases.
> > >
> > > Ozone
> > > =====
> > >
> > > When these were part of/in-sync with the hadoop build there wouldn't
> have
> > > been problems. Now there are. Again, they're going in deep, but here
> > > clearly to simulate some behaviour. Any way to do that differently?
> > >
> > > Ratis
> > > =====
> > >
> > > No idea.
> > >
> > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > <weichiu@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Most of the problems are downstream applications using Hadoop's
> private
> > > > APIs.
> > > >
> > > > Tez:
> > > >
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > incompatible types: com.google.protobuf.ByteString cannot be
> converted
> > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > >
> > > >
> > > > Tez keeps track of job tokens internally.
> > > > The change would look like this:
> > > >
> > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > >   if (stateDb != null) {
> > > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > >         .setKind(jobToken.getKind().toString())
> > > >         .setService(jobToken.getService().toString())
> > > >         .build();*/
> > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > >         .setUser(user).setJobToken(tokenProto).build();
> > > >     try {
> > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > >     } catch (DBException e) {
> > > >       throw new IOException("Error storing " + jobId, e);
> > > >     }
> > > >   }
> > > >   addJobToken(jobId, user, jobToken);
> > > > }
> > > >
> > > >
> > > > HBase:
> > > >
> > > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833
> >
> > > > (this
> > > >    is recently fixed in the master branch)
> > > >    2.
> > > >
> > > >       [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > (default-compile) on project hbase-server: Compilation failure:
> > > > Compilation failure:
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > incompatible types:
> > > >
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > invalid method reference
> > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > static context
> > > >
> > > >
> > > > Ozone:
> > > >
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > >
> > > >
> > > > There's another error where Ozone uses the Hadoop RPC framework which
> > > uses
> > > > the hadoop.thirdparty protobuf.
> > > >
> > > > [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > Compilation
> > > > failure
> > > > [ERROR]
> > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > converted
> > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > >
> > > > BlockingService scmDatanodeService =
> > > >     StorageContainerDatanodeProtocolService.
> > > >         newReflectiveBlockingService(
> > > >             new
> StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > > >
> > > >
> > > >
> > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > framework
> > > > too.
> > > >
> > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > vinayakumarb@apache.org>
> > > > wrote:
> > > >
> > > > > hi Wei-Chiu,
> > > > >
> > > > > Can you elaborate on what failures you are facing related to
> > relocated
> > > > > protobuf classes.. ?
> > > > >
> > > > > IFAIK, if the issue with location of protobuf classes, still old
> jar
> > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > depending
> > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > >
> > > > > -vinay
> > > > >
> > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> weichiu@cloudera.com
> > >
> > > > > wrote:
> > > > >
> > > > >> I'm sorry for coming to this late. I missed this message. It
> should
> > > have
> > > > >> been a DISCUSS thread rather than NOTICE.
> > > > >>
> > > > >> Looks like this is inevitable. But we should make the downstream
> > > > >> developers aware & make the update easier. As long as it is stated
> > > > clearly
> > > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > > >>
> > > > >> Here's what I suggest:
> > > > >> (1) label the jira incompatible (just updated the jira) and
> updated
> > > the
> > > > >> release note to tell app developer how to update.
> > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > >>
> > > > >> Tez doesn't use the removed Token API, but there's code that
> breaks
> > > with
> > > > >> the relocated protobuf class. The ProtobufHelper API will make
> this
> > > > >> transition much easier.
> > > > >>
> > > > >> Other downstreamers that break with the relocated protobuf
> include:
> > > > Ozone
> > > > >> and HBase. but neither of them use the removed Token API.
> > > > >>
> > > > >>
> > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > vinayakumarb@apache.org
> > > >
> > > > >> wrote:
> > > > >>
> > > > >>> Hi All,
> > > > >>>
> > > > >>>    This mail is to notify about the Removal of following public
> > APIs
> > > > from
> > > > >>> Hadoop Common.
> > > > >>>
> > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > >>>      APIs:
> > > > >>>          public Token(TokenProto tokenPB);
> > > > >>>          public TokenProto toTokenProto();
> > > > >>>
> > > > >>>    Reason: These APIs are having Generated protobuf classes in
> the
> > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > > 3.3.0
> > > > >>> release) these APIs are breaking the downstream builds, even
> though
> > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > Downstreams
> > > > >>> are
> > > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > > build
> > > > >>> is
> > > > >>> being broken.
> > > > >>>
> > > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> > to
> > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > >>> Same purpose can be achieved using the Helper classes without
> > > > introducing
> > > > >>> protobuf classes in API signatures.
> > > > >>>
> > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> changed
> > > > >>> whenever
> > > > >>> absolute necessary.
> > > > >>>
> > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> > been
> > > > >>> reported to solve downstream build failure.
> > > > >>>
> > > > >>> So since this API was added for internal purpose easy approach to
> > > solve
> > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > mentioned
> > > > in
> > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > maintained.
> > > > >>>
> > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > please
> > > > >>> reply to this mail immediately.
> > > > >>>
> > > > >>> If no objection by next week, will go ahead with removal of above
> > > said
> > > > >>> APIs
> > > > >>> in HADOOP-16621.
> > > > >>>
> > > > >>> -Vinay
> > > > >>>
> > > > >>
> > > >
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Akira Ajisaka <aa...@apache.org>.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
> This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.

Thank you, Vinay. I checked the PR and it mostly looks good.
How do we proceed with?

I suppose Hadoop 3.3.0 is blocked by this issue. Is it true or not?

Thanks,
Akira

On Tue, May 19, 2020 at 2:06 AM Eric Yang <ey...@apache.org> wrote:

> ProtobufHelper should not be a public API.  Hadoop uses protobuf
> serialization to expertise RPC performance with many drawbacks.  The
> generalized object usually require another indirection to map to usable
> Java object, this is making Hadoop code messy, and that is topic for
> another day.  The main challenges for UGI class is making the system
> difficult to secure.
>
> In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
> protocol, and secured by JWT token with Google.  This means before
> deserializing a protobuf object on the wire, the call must deserialize a
> JSON token to determine if the call is authenticated before deserializing
> application objects.  Hence, using protobuf for RPC is no longer a good
> reason for performance gain over JSON because JWT token deserialization
> happens on every gRPC call to ensure the request is secured properly.
>
> In Hadoop world, we are not using JWT token for authentication, we have
> pluggable token implementation either SPNEGO, delegation token or some kind
> of SASL.  UGI class should not allow protobuf token to be exposed as public
> interface, otherwise down stream application can forge the protobuf token
> and it will become a privilege escalation issue.  In my opinion, UGI class
> must be as private as possible to prevent forgery.  Down stream application
> are discouraged from using UGI.doAs for impersonation to reduce privileges
> escalation.  Instead, the downstream application should running like Unix
> daemon instead of root.  This will ensure that vulnerability for one
> application does not spill over security problems to another application.
> Some people will disagree with the statement because existing application
> is already written to take advantage of UGI.doAs, such as Hive loading
> external table.  Fortunately, Hive provides an option to run without doAs.
>
> Protobuf is not suitable candidate for security token transport because it
> is a strong type transport.  If multiple tokens are transported with UGI
> protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
> conversion ambiguity that might create security holes or headache on type
> casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
> as byte array, and default to JSON serializer is probably simpler solution
> to keep the system robust without repeating the past mistakes.
>
> regards,
> Eric
>
> On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > Hi Wei-chu and steve,
> >
> > Thanks for sharing insights.
> >
> > I have also tried to compile and execute ozone pointing to
> > trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
> >
> > Other than just the usage of internal protobuf APIs, because of which
> > compilation would break, I found another major problem was, the
> Hadoop-rpc
> > implementations in downstreams which is based on non-shaded Protobuf
> > classes.
> >
> > 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> > 'Message', which its expecting to be of 3.7 version and shaded package
> > (i.e. o.a.h.thirdparty.*).
> >
> > So,unless downstreams upgrade their protobuf classes to
> 'hadoop-thirdparty'
> > this issue will continue to occur, even after solving compilation issues
> > due to internal usage of private APIs with protobuf signatures.
> >
> > I found a possible workaround for this problem.
> > Please check https://issues.apache.org/jira/browse/HADOOP-17046
> >   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> > shading and with protobuf-2.5.0 implementation) to support downstream
> > implementations.
> >   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> > and later projects who wish to upgrade their protobufs to 3.x.
> >
> > For Ozone compilation:
> >   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> > upgrade. These PRs will remove dependency on Hadoop for those internal
> APIs
> > and implemented their own copy in ozone with non-shaded protobuf.
> >     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> > <https://github.com/apache/hadoop-ozone/pull/933>2
> >     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
> >
> > Also, I had run some tests on Ozone after applying these PRs and
> > HADOOP-17046 with 3.4.0, tests seems to pass.
> >
> > Please help review these PRs.
> >
> > Thanks,
> > -Vinay
> >
> >
> > On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran
> <stevel@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Okay.
> > >
> > > I am not going to be a purist and say "what were they doing -using our
> > > private APIs?" because as we all know, with things like UGI tagged
> > @private
> > > there's been no way to get something is done without getting into the
> > > private stuff.
> > >
> > > But why did we do the protobuf changes? So that we could update our
> > private
> > > copy of protobuf with out breaking every single downstream application.
> > The
> > > great protobuf upgrade to 2.5 is not something we wanted to repeat.
> When
> > > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> > weeks
> > > were absolutely nothing would build whatsoever, not until every
> > downstream
> > > project had upgraded to the same version of the library.
> > >
> > > If you ever want to see an upgrade which makes a guava update seem a
> > minor
> > > detail, protobuf upgrades are it. Hence the shading
> > >
> > > HBase
> > > =====
> > >
> > > it looks like HBase has been using deep internal stuff. That is,
> > > "unfortunate". I think in that world we have to look and say is there
> > > something specific we can do here to help HBase in a way we could also
> > > backport. They shouldn't need those IPC internals.
> > >
> > > Tez & Tokens
> > > ============
> > >
> > > I didn't know Tez was using those protobuf APIs internally. That is,
> > > "unfortunate".
> > >
> > > What is key is this: without us moving those methods things like Spark
> > > wouldn't work. And they weren't even using the methods, just trying to
> > work
> > > with Token for job submission.
> > >
> > > All Tez should need is a byte array serialization of a token. Given
> Token
> > > is also Writable, that could be done via WritableUtils in a way which
> > will
> > > also work with older releases.
> > >
> > > Ozone
> > > =====
> > >
> > > When these were part of/in-sync with the hadoop build there wouldn't
> have
> > > been problems. Now there are. Again, they're going in deep, but here
> > > clearly to simulate some behaviour. Any way to do that differently?
> > >
> > > Ratis
> > > =====
> > >
> > > No idea.
> > >
> > > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> > <weichiu@cloudera.com.invalid
> > > >
> > > wrote:
> > >
> > > > Most of the problems are downstream applications using Hadoop's
> private
> > > > APIs.
> > > >
> > > > Tez:
> > > >
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > > incompatible types: com.google.protobuf.ByteString cannot be
> converted
> > > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > > >
> > > >
> > > > Tez keeps track of job tokens internally.
> > > > The change would look like this:
> > > >
> > > > private void recordJobShuffleInfo(JobID jobId, String user,
> > > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > > >   if (stateDb != null) {
> > > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > > >         .setKind(jobToken.getKind().toString())
> > > >         .setService(jobToken.getService().toString())
> > > >         .build();*/
> > > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > > >         .setUser(user).setJobToken(tokenProto).build();
> > > >     try {
> > > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > > >     } catch (DBException e) {
> > > >       throw new IOException("Error storing " + jobId, e);
> > > >     }
> > > >   }
> > > >   addJobToken(jobId, user, jobToken);
> > > > }
> > > >
> > > >
> > > > HBase:
> > > >
> > > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833
> >
> > > > (this
> > > >    is recently fixed in the master branch)
> > > >    2.
> > > >
> > > >       [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > > (default-compile) on project hbase-server: Compilation failure:
> > > > Compilation failure:
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > > incompatible types:
> > > >
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > > cannot access
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > > cannot find symbol
> > > >       [ERROR]   symbol:   method
> > > >
> > > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > > >       [ERROR]   location: variable proto of type
> > > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > > >       [ERROR]   class file for
> > > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > > incompatible types:
> > > >
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > > cannot be converted to com.google.protobuf.MessageLite
> > > >       [ERROR]
> > > >
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > > invalid method reference
> > > >       [ERROR]   non-static method get() cannot be referenced from a
> > > > static context
> > > >
> > > >
> > > > Ozone:
> > > >
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > > -------------------------------------------------------------
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > > >
> > > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > > >
> > > >
> > > > There's another error where Ozone uses the Hadoop RPC framework which
> > > uses
> > > > the hadoop.thirdparty protobuf.
> > > >
> > > > [ERROR] Failed to execute goal
> > > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > > (default-testCompile) on project hadoop-hdds-container-service:
> > > Compilation
> > > > failure
> > > > [ERROR]
> > > >
> > > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > > incompatible types: com.google.protobuf.BlockingService cannot be
> > > converted
> > > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > > >
> > > > BlockingService scmDatanodeService =
> > > >     StorageContainerDatanodeProtocolService.
> > > >         newReflectiveBlockingService(
> > > >             new
> StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > > >
> > > >
> > > >
> > > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > > framework
> > > > too.
> > > >
> > > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> > vinayakumarb@apache.org>
> > > > wrote:
> > > >
> > > > > hi Wei-Chiu,
> > > > >
> > > > > Can you elaborate on what failures you are facing related to
> > relocated
> > > > > protobuf classes.. ?
> > > > >
> > > > > IFAIK, if the issue with location of protobuf classes, still old
> jar
> > > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > > depending
> > > > > on 2.5.0 version of protobuf still be able to access them.
> > > > >
> > > > > -vinay
> > > > >
> > > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <
> weichiu@cloudera.com
> > >
> > > > > wrote:
> > > > >
> > > > >> I'm sorry for coming to this late. I missed this message. It
> should
> > > have
> > > > >> been a DISCUSS thread rather than NOTICE.
> > > > >>
> > > > >> Looks like this is inevitable. But we should make the downstream
> > > > >> developers aware & make the update easier. As long as it is stated
> > > > clearly
> > > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > > >>
> > > > >> Here's what I suggest:
> > > > >> (1) label the jira incompatible (just updated the jira) and
> updated
> > > the
> > > > >> release note to tell app developer how to update.
> > > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > > >>
> > > > >> Tez doesn't use the removed Token API, but there's code that
> breaks
> > > with
> > > > >> the relocated protobuf class. The ProtobufHelper API will make
> this
> > > > >> transition much easier.
> > > > >>
> > > > >> Other downstreamers that break with the relocated protobuf
> include:
> > > > Ozone
> > > > >> and HBase. but neither of them use the removed Token API.
> > > > >>
> > > > >>
> > > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> > vinayakumarb@apache.org
> > > >
> > > > >> wrote:
> > > > >>
> > > > >>> Hi All,
> > > > >>>
> > > > >>>    This mail is to notify about the Removal of following public
> > APIs
> > > > from
> > > > >>> Hadoop Common.
> > > > >>>
> > > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > > >>>      APIs:
> > > > >>>          public Token(TokenProto tokenPB);
> > > > >>>          public TokenProto toTokenProto();
> > > > >>>
> > > > >>>    Reason: These APIs are having Generated protobuf classes in
> the
> > > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > > 3.3.0
> > > > >>> release) these APIs are breaking the downstream builds, even
> though
> > > > >>> downstreams dont use these APIs (just Loading Token class).
> > > Downstreams
> > > > >>> are
> > > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > > build
> > > > >>> is
> > > > >>> being broken.
> > > > >>>
> > > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> > to
> > > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > > >>> Same purpose can be achieved using the Helper classes without
> > > > introducing
> > > > >>> protobuf classes in API signatures.
> > > > >>>
> > > > >>> Token.java is marked as Evolving, so I believe APIs can be
> changed
> > > > >>> whenever
> > > > >>> absolute necessary.
> > > > >>>
> > > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> > been
> > > > >>> reported to solve downstream build failure.
> > > > >>>
> > > > >>> So since this API was added for internal purpose easy approach to
> > > solve
> > > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > > mentioned
> > > > in
> > > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> > maintained.
> > > > >>>
> > > > >>> If anyone using these APIs outside hadoop project accidentally,
> > > please
> > > > >>> reply to this mail immediately.
> > > > >>>
> > > > >>> If no objection by next week, will go ahead with removal of above
> > > said
> > > > >>> APIs
> > > > >>> in HADOOP-16621.
> > > > >>>
> > > > >>> -Vinay
> > > > >>>
> > > > >>
> > > >
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Eric Yang <ey...@apache.org>.
ProtobufHelper should not be a public API.  Hadoop uses protobuf
serialization to expertise RPC performance with many drawbacks.  The
generalized object usually require another indirection to map to usable
Java object, this is making Hadoop code messy, and that is topic for
another day.  The main challenges for UGI class is making the system
difficult to secure.

In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
protocol, and secured by JWT token with Google.  This means before
deserializing a protobuf object on the wire, the call must deserialize a
JSON token to determine if the call is authenticated before deserializing
application objects.  Hence, using protobuf for RPC is no longer a good
reason for performance gain over JSON because JWT token deserialization
happens on every gRPC call to ensure the request is secured properly.

In Hadoop world, we are not using JWT token for authentication, we have
pluggable token implementation either SPNEGO, delegation token or some kind
of SASL.  UGI class should not allow protobuf token to be exposed as public
interface, otherwise down stream application can forge the protobuf token
and it will become a privilege escalation issue.  In my opinion, UGI class
must be as private as possible to prevent forgery.  Down stream application
are discouraged from using UGI.doAs for impersonation to reduce privileges
escalation.  Instead, the downstream application should running like Unix
daemon instead of root.  This will ensure that vulnerability for one
application does not spill over security problems to another application.
Some people will disagree with the statement because existing application
is already written to take advantage of UGI.doAs, such as Hive loading
external table.  Fortunately, Hive provides an option to run without doAs.

Protobuf is not suitable candidate for security token transport because it
is a strong type transport.  If multiple tokens are transported with UGI
protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
conversion ambiguity that might create security holes or headache on type
casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
as byte array, and default to JSON serializer is probably simpler solution
to keep the system robust without repeating the past mistakes.

regards,
Eric

On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
wrote:

> Hi Wei-chu and steve,
>
> Thanks for sharing insights.
>
> I have also tried to compile and execute ozone pointing to
> trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
>
> Other than just the usage of internal protobuf APIs, because of which
> compilation would break, I found another major problem was, the Hadoop-rpc
> implementations in downstreams which is based on non-shaded Protobuf
> classes.
>
> 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> 'Message', which its expecting to be of 3.7 version and shaded package
> (i.e. o.a.h.thirdparty.*).
>
> So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
> this issue will continue to occur, even after solving compilation issues
> due to internal usage of private APIs with protobuf signatures.
>
> I found a possible workaround for this problem.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
>   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> and later projects who wish to upgrade their protobufs to 3.x.
>
> For Ozone compilation:
>   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> upgrade. These PRs will remove dependency on Hadoop for those internal APIs
> and implemented their own copy in ozone with non-shaded protobuf.
>     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> <https://github.com/apache/hadoop-ozone/pull/933>2
>     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
>
> Also, I had run some tests on Ozone after applying these PRs and
> HADOOP-17046 with 3.4.0, tests seems to pass.
>
> Please help review these PRs.
>
> Thanks,
> -Vinay
>
>
> On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <stevel@cloudera.com.invalid
> >
> wrote:
>
> > Okay.
> >
> > I am not going to be a purist and say "what were they doing -using our
> > private APIs?" because as we all know, with things like UGI tagged
> @private
> > there's been no way to get something is done without getting into the
> > private stuff.
> >
> > But why did we do the protobuf changes? So that we could update our
> private
> > copy of protobuf with out breaking every single downstream application.
> The
> > great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> weeks
> > were absolutely nothing would build whatsoever, not until every
> downstream
> > project had upgraded to the same version of the library.
> >
> > If you ever want to see an upgrade which makes a guava update seem a
> minor
> > detail, protobuf upgrades are it. Hence the shading
> >
> > HBase
> > =====
> >
> > it looks like HBase has been using deep internal stuff. That is,
> > "unfortunate". I think in that world we have to look and say is there
> > something specific we can do here to help HBase in a way we could also
> > backport. They shouldn't need those IPC internals.
> >
> > Tez & Tokens
> > ============
> >
> > I didn't know Tez was using those protobuf APIs internally. That is,
> > "unfortunate".
> >
> > What is key is this: without us moving those methods things like Spark
> > wouldn't work. And they weren't even using the methods, just trying to
> work
> > with Token for job submission.
> >
> > All Tez should need is a byte array serialization of a token. Given Token
> > is also Writable, that could be done via WritableUtils in a way which
> will
> > also work with older releases.
> >
> > Ozone
> > =====
> >
> > When these were part of/in-sync with the hadoop build there wouldn't have
> > been problems. Now there are. Again, they're going in deep, but here
> > clearly to simulate some behaviour. Any way to do that differently?
> >
> > Ratis
> > =====
> >
> > No idea.
> >
> > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> <weichiu@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Most of the problems are downstream applications using Hadoop's private
> > > APIs.
> > >
> > > Tez:
> > >
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > incompatible types: com.google.protobuf.ByteString cannot be converted
> > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > >
> > >
> > > Tez keeps track of job tokens internally.
> > > The change would look like this:
> > >
> > > private void recordJobShuffleInfo(JobID jobId, String user,
> > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > >   if (stateDb != null) {
> > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > >         .setKind(jobToken.getKind().toString())
> > >         .setService(jobToken.getService().toString())
> > >         .build();*/
> > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > >         .setUser(user).setJobToken(tokenProto).build();
> > >     try {
> > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > >     } catch (DBException e) {
> > >       throw new IOException("Error storing " + jobId, e);
> > >     }
> > >   }
> > >   addJobToken(jobId, user, jobToken);
> > > }
> > >
> > >
> > > HBase:
> > >
> > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > > (this
> > >    is recently fixed in the master branch)
> > >    2.
> > >
> > >       [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > (default-compile) on project hbase-server: Compilation failure:
> > > Compilation failure:
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > incompatible types:
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > invalid method reference
> > >       [ERROR]   non-static method get() cannot be referenced from a
> > > static context
> > >
> > >
> > > Ozone:
> > >
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > >
> > >
> > > There's another error where Ozone uses the Hadoop RPC framework which
> > uses
> > > the hadoop.thirdparty protobuf.
> > >
> > > [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > (default-testCompile) on project hadoop-hdds-container-service:
> > Compilation
> > > failure
> > > [ERROR]
> > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > incompatible types: com.google.protobuf.BlockingService cannot be
> > converted
> > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > >
> > > BlockingService scmDatanodeService =
> > >     StorageContainerDatanodeProtocolService.
> > >         newReflectiveBlockingService(
> > >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > >
> > >
> > >
> > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > framework
> > > too.
> > >
> > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> vinayakumarb@apache.org>
> > > wrote:
> > >
> > > > hi Wei-Chiu,
> > > >
> > > > Can you elaborate on what failures you are facing related to
> relocated
> > > > protobuf classes.. ?
> > > >
> > > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > depending
> > > > on 2.5.0 version of protobuf still be able to access them.
> > > >
> > > > -vinay
> > > >
> > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <weichiu@cloudera.com
> >
> > > > wrote:
> > > >
> > > >> I'm sorry for coming to this late. I missed this message. It should
> > have
> > > >> been a DISCUSS thread rather than NOTICE.
> > > >>
> > > >> Looks like this is inevitable. But we should make the downstream
> > > >> developers aware & make the update easier. As long as it is stated
> > > clearly
> > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > >>
> > > >> Here's what I suggest:
> > > >> (1) label the jira incompatible (just updated the jira) and updated
> > the
> > > >> release note to tell app developer how to update.
> > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > >>
> > > >> Tez doesn't use the removed Token API, but there's code that breaks
> > with
> > > >> the relocated protobuf class. The ProtobufHelper API will make this
> > > >> transition much easier.
> > > >>
> > > >> Other downstreamers that break with the relocated protobuf include:
> > > Ozone
> > > >> and HBase. but neither of them use the removed Token API.
> > > >>
> > > >>
> > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> vinayakumarb@apache.org
> > >
> > > >> wrote:
> > > >>
> > > >>> Hi All,
> > > >>>
> > > >>>    This mail is to notify about the Removal of following public
> APIs
> > > from
> > > >>> Hadoop Common.
> > > >>>
> > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > >>>      APIs:
> > > >>>          public Token(TokenProto tokenPB);
> > > >>>          public TokenProto toTokenProto();
> > > >>>
> > > >>>    Reason: These APIs are having Generated protobuf classes in the
> > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > 3.3.0
> > > >>> release) these APIs are breaking the downstream builds, even though
> > > >>> downstreams dont use these APIs (just Loading Token class).
> > Downstreams
> > > >>> are
> > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > build
> > > >>> is
> > > >>> being broken.
> > > >>>
> > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> to
> > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > >>> Same purpose can be achieved using the Helper classes without
> > > introducing
> > > >>> protobuf classes in API signatures.
> > > >>>
> > > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > > >>> whenever
> > > >>> absolute necessary.
> > > >>>
> > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> been
> > > >>> reported to solve downstream build failure.
> > > >>>
> > > >>> So since this API was added for internal purpose easy approach to
> > solve
> > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > mentioned
> > > in
> > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> maintained.
> > > >>>
> > > >>> If anyone using these APIs outside hadoop project accidentally,
> > please
> > > >>> reply to this mail immediately.
> > > >>>
> > > >>> If no objection by next week, will go ahead with removal of above
> > said
> > > >>> APIs
> > > >>> in HADOOP-16621.
> > > >>>
> > > >>> -Vinay
> > > >>>
> > > >>
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Eric Yang <ey...@apache.org>.
ProtobufHelper should not be a public API.  Hadoop uses protobuf
serialization to expertise RPC performance with many drawbacks.  The
generalized object usually require another indirection to map to usable
Java object, this is making Hadoop code messy, and that is topic for
another day.  The main challenges for UGI class is making the system
difficult to secure.

In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
protocol, and secured by JWT token with Google.  This means before
deserializing a protobuf object on the wire, the call must deserialize a
JSON token to determine if the call is authenticated before deserializing
application objects.  Hence, using protobuf for RPC is no longer a good
reason for performance gain over JSON because JWT token deserialization
happens on every gRPC call to ensure the request is secured properly.

In Hadoop world, we are not using JWT token for authentication, we have
pluggable token implementation either SPNEGO, delegation token or some kind
of SASL.  UGI class should not allow protobuf token to be exposed as public
interface, otherwise down stream application can forge the protobuf token
and it will become a privilege escalation issue.  In my opinion, UGI class
must be as private as possible to prevent forgery.  Down stream application
are discouraged from using UGI.doAs for impersonation to reduce privileges
escalation.  Instead, the downstream application should running like Unix
daemon instead of root.  This will ensure that vulnerability for one
application does not spill over security problems to another application.
Some people will disagree with the statement because existing application
is already written to take advantage of UGI.doAs, such as Hive loading
external table.  Fortunately, Hive provides an option to run without doAs.

Protobuf is not suitable candidate for security token transport because it
is a strong type transport.  If multiple tokens are transported with UGI
protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
conversion ambiguity that might create security holes or headache on type
casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
as byte array, and default to JSON serializer is probably simpler solution
to keep the system robust without repeating the past mistakes.

regards,
Eric

On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
wrote:

> Hi Wei-chu and steve,
>
> Thanks for sharing insights.
>
> I have also tried to compile and execute ozone pointing to
> trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
>
> Other than just the usage of internal protobuf APIs, because of which
> compilation would break, I found another major problem was, the Hadoop-rpc
> implementations in downstreams which is based on non-shaded Protobuf
> classes.
>
> 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> 'Message', which its expecting to be of 3.7 version and shaded package
> (i.e. o.a.h.thirdparty.*).
>
> So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
> this issue will continue to occur, even after solving compilation issues
> due to internal usage of private APIs with protobuf signatures.
>
> I found a possible workaround for this problem.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
>   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> and later projects who wish to upgrade their protobufs to 3.x.
>
> For Ozone compilation:
>   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> upgrade. These PRs will remove dependency on Hadoop for those internal APIs
> and implemented their own copy in ozone with non-shaded protobuf.
>     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> <https://github.com/apache/hadoop-ozone/pull/933>2
>     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
>
> Also, I had run some tests on Ozone after applying these PRs and
> HADOOP-17046 with 3.4.0, tests seems to pass.
>
> Please help review these PRs.
>
> Thanks,
> -Vinay
>
>
> On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <stevel@cloudera.com.invalid
> >
> wrote:
>
> > Okay.
> >
> > I am not going to be a purist and say "what were they doing -using our
> > private APIs?" because as we all know, with things like UGI tagged
> @private
> > there's been no way to get something is done without getting into the
> > private stuff.
> >
> > But why did we do the protobuf changes? So that we could update our
> private
> > copy of protobuf with out breaking every single downstream application.
> The
> > great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> weeks
> > were absolutely nothing would build whatsoever, not until every
> downstream
> > project had upgraded to the same version of the library.
> >
> > If you ever want to see an upgrade which makes a guava update seem a
> minor
> > detail, protobuf upgrades are it. Hence the shading
> >
> > HBase
> > =====
> >
> > it looks like HBase has been using deep internal stuff. That is,
> > "unfortunate". I think in that world we have to look and say is there
> > something specific we can do here to help HBase in a way we could also
> > backport. They shouldn't need those IPC internals.
> >
> > Tez & Tokens
> > ============
> >
> > I didn't know Tez was using those protobuf APIs internally. That is,
> > "unfortunate".
> >
> > What is key is this: without us moving those methods things like Spark
> > wouldn't work. And they weren't even using the methods, just trying to
> work
> > with Token for job submission.
> >
> > All Tez should need is a byte array serialization of a token. Given Token
> > is also Writable, that could be done via WritableUtils in a way which
> will
> > also work with older releases.
> >
> > Ozone
> > =====
> >
> > When these were part of/in-sync with the hadoop build there wouldn't have
> > been problems. Now there are. Again, they're going in deep, but here
> > clearly to simulate some behaviour. Any way to do that differently?
> >
> > Ratis
> > =====
> >
> > No idea.
> >
> > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> <weichiu@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Most of the problems are downstream applications using Hadoop's private
> > > APIs.
> > >
> > > Tez:
> > >
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > incompatible types: com.google.protobuf.ByteString cannot be converted
> > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > >
> > >
> > > Tez keeps track of job tokens internally.
> > > The change would look like this:
> > >
> > > private void recordJobShuffleInfo(JobID jobId, String user,
> > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > >   if (stateDb != null) {
> > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > >         .setKind(jobToken.getKind().toString())
> > >         .setService(jobToken.getService().toString())
> > >         .build();*/
> > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > >         .setUser(user).setJobToken(tokenProto).build();
> > >     try {
> > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > >     } catch (DBException e) {
> > >       throw new IOException("Error storing " + jobId, e);
> > >     }
> > >   }
> > >   addJobToken(jobId, user, jobToken);
> > > }
> > >
> > >
> > > HBase:
> > >
> > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > > (this
> > >    is recently fixed in the master branch)
> > >    2.
> > >
> > >       [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > (default-compile) on project hbase-server: Compilation failure:
> > > Compilation failure:
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > incompatible types:
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > invalid method reference
> > >       [ERROR]   non-static method get() cannot be referenced from a
> > > static context
> > >
> > >
> > > Ozone:
> > >
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > >
> > >
> > > There's another error where Ozone uses the Hadoop RPC framework which
> > uses
> > > the hadoop.thirdparty protobuf.
> > >
> > > [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > (default-testCompile) on project hadoop-hdds-container-service:
> > Compilation
> > > failure
> > > [ERROR]
> > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > incompatible types: com.google.protobuf.BlockingService cannot be
> > converted
> > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > >
> > > BlockingService scmDatanodeService =
> > >     StorageContainerDatanodeProtocolService.
> > >         newReflectiveBlockingService(
> > >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > >
> > >
> > >
> > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > framework
> > > too.
> > >
> > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> vinayakumarb@apache.org>
> > > wrote:
> > >
> > > > hi Wei-Chiu,
> > > >
> > > > Can you elaborate on what failures you are facing related to
> relocated
> > > > protobuf classes.. ?
> > > >
> > > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > depending
> > > > on 2.5.0 version of protobuf still be able to access them.
> > > >
> > > > -vinay
> > > >
> > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <weichiu@cloudera.com
> >
> > > > wrote:
> > > >
> > > >> I'm sorry for coming to this late. I missed this message. It should
> > have
> > > >> been a DISCUSS thread rather than NOTICE.
> > > >>
> > > >> Looks like this is inevitable. But we should make the downstream
> > > >> developers aware & make the update easier. As long as it is stated
> > > clearly
> > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > >>
> > > >> Here's what I suggest:
> > > >> (1) label the jira incompatible (just updated the jira) and updated
> > the
> > > >> release note to tell app developer how to update.
> > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > >>
> > > >> Tez doesn't use the removed Token API, but there's code that breaks
> > with
> > > >> the relocated protobuf class. The ProtobufHelper API will make this
> > > >> transition much easier.
> > > >>
> > > >> Other downstreamers that break with the relocated protobuf include:
> > > Ozone
> > > >> and HBase. but neither of them use the removed Token API.
> > > >>
> > > >>
> > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> vinayakumarb@apache.org
> > >
> > > >> wrote:
> > > >>
> > > >>> Hi All,
> > > >>>
> > > >>>    This mail is to notify about the Removal of following public
> APIs
> > > from
> > > >>> Hadoop Common.
> > > >>>
> > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > >>>      APIs:
> > > >>>          public Token(TokenProto tokenPB);
> > > >>>          public TokenProto toTokenProto();
> > > >>>
> > > >>>    Reason: These APIs are having Generated protobuf classes in the
> > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > 3.3.0
> > > >>> release) these APIs are breaking the downstream builds, even though
> > > >>> downstreams dont use these APIs (just Loading Token class).
> > Downstreams
> > > >>> are
> > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > build
> > > >>> is
> > > >>> being broken.
> > > >>>
> > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> to
> > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > >>> Same purpose can be achieved using the Helper classes without
> > > introducing
> > > >>> protobuf classes in API signatures.
> > > >>>
> > > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > > >>> whenever
> > > >>> absolute necessary.
> > > >>>
> > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> been
> > > >>> reported to solve downstream build failure.
> > > >>>
> > > >>> So since this API was added for internal purpose easy approach to
> > solve
> > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > mentioned
> > > in
> > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> maintained.
> > > >>>
> > > >>> If anyone using these APIs outside hadoop project accidentally,
> > please
> > > >>> reply to this mail immediately.
> > > >>>
> > > >>> If no objection by next week, will go ahead with removal of above
> > said
> > > >>> APIs
> > > >>> in HADOOP-16621.
> > > >>>
> > > >>> -Vinay
> > > >>>
> > > >>
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Eric Yang <ey...@apache.org>.
ProtobufHelper should not be a public API.  Hadoop uses protobuf
serialization to expertise RPC performance with many drawbacks.  The
generalized object usually require another indirection to map to usable
Java object, this is making Hadoop code messy, and that is topic for
another day.  The main challenges for UGI class is making the system
difficult to secure.

In Google's world, gRPC is built on top of protobuf + HTTP/2 binary
protocol, and secured by JWT token with Google.  This means before
deserializing a protobuf object on the wire, the call must deserialize a
JSON token to determine if the call is authenticated before deserializing
application objects.  Hence, using protobuf for RPC is no longer a good
reason for performance gain over JSON because JWT token deserialization
happens on every gRPC call to ensure the request is secured properly.

In Hadoop world, we are not using JWT token for authentication, we have
pluggable token implementation either SPNEGO, delegation token or some kind
of SASL.  UGI class should not allow protobuf token to be exposed as public
interface, otherwise down stream application can forge the protobuf token
and it will become a privilege escalation issue.  In my opinion, UGI class
must be as private as possible to prevent forgery.  Down stream application
are discouraged from using UGI.doAs for impersonation to reduce privileges
escalation.  Instead, the downstream application should running like Unix
daemon instead of root.  This will ensure that vulnerability for one
application does not spill over security problems to another application.
Some people will disagree with the statement because existing application
is already written to take advantage of UGI.doAs, such as Hive loading
external table.  Fortunately, Hive provides an option to run without doAs.

Protobuf is not suitable candidate for security token transport because it
is a strong type transport.  If multiple tokens are transported with UGI
protobuf, small difference in ASCII, UTF-8, or UTF-16 can cause
conversion ambiguity that might create security holes or headache on type
casting.  I am +1 on removing protobuf from Hadoop Token API.  Hadoop Token
as byte array, and default to JSON serializer is probably simpler solution
to keep the system robust without repeating the past mistakes.

regards,
Eric

On Sun, May 17, 2020 at 11:56 PM Vinayakumar B <vi...@apache.org>
wrote:

> Hi Wei-chu and steve,
>
> Thanks for sharing insights.
>
> I have also tried to compile and execute ozone pointing to
> trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.
>
> Other than just the usage of internal protobuf APIs, because of which
> compilation would break, I found another major problem was, the Hadoop-rpc
> implementations in downstreams which is based on non-shaded Protobuf
> classes.
>
> 'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
> 'Message', which its expecting to be of 3.7 version and shaded package
> (i.e. o.a.h.thirdparty.*).
>
> So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
> this issue will continue to occur, even after solving compilation issues
> due to internal usage of private APIs with protobuf signatures.
>
> I found a possible workaround for this problem.
> Please check https://issues.apache.org/jira/browse/HADOOP-17046
>   This Jira proposes to keep existing ProtobuRpcEngine as-is (without
> shading and with protobuf-2.5.0 implementation) to support downstream
> implementations.
>   Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
> and later projects who wish to upgrade their protobufs to 3.x.
>
> For Ozone compilation:
>   I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
> upgrade. These PRs will remove dependency on Hadoop for those internal APIs
> and implemented their own copy in ozone with non-shaded protobuf.
>     HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
> <https://github.com/apache/hadoop-ozone/pull/933>2
>     HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933
>
> Also, I had run some tests on Ozone after applying these PRs and
> HADOOP-17046 with 3.4.0, tests seems to pass.
>
> Please help review these PRs.
>
> Thanks,
> -Vinay
>
>
> On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <stevel@cloudera.com.invalid
> >
> wrote:
>
> > Okay.
> >
> > I am not going to be a purist and say "what were they doing -using our
> > private APIs?" because as we all know, with things like UGI tagged
> @private
> > there's been no way to get something is done without getting into the
> > private stuff.
> >
> > But why did we do the protobuf changes? So that we could update our
> private
> > copy of protobuf with out breaking every single downstream application.
> The
> > great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> > was that? before hadoop-2.2 shipped? I certainly remember a couple of
> weeks
> > were absolutely nothing would build whatsoever, not until every
> downstream
> > project had upgraded to the same version of the library.
> >
> > If you ever want to see an upgrade which makes a guava update seem a
> minor
> > detail, protobuf upgrades are it. Hence the shading
> >
> > HBase
> > =====
> >
> > it looks like HBase has been using deep internal stuff. That is,
> > "unfortunate". I think in that world we have to look and say is there
> > something specific we can do here to help HBase in a way we could also
> > backport. They shouldn't need those IPC internals.
> >
> > Tez & Tokens
> > ============
> >
> > I didn't know Tez was using those protobuf APIs internally. That is,
> > "unfortunate".
> >
> > What is key is this: without us moving those methods things like Spark
> > wouldn't work. And they weren't even using the methods, just trying to
> work
> > with Token for job submission.
> >
> > All Tez should need is a byte array serialization of a token. Given Token
> > is also Writable, that could be done via WritableUtils in a way which
> will
> > also work with older releases.
> >
> > Ozone
> > =====
> >
> > When these were part of/in-sync with the hadoop build there wouldn't have
> > been problems. Now there are. Again, they're going in deep, but here
> > clearly to simulate some behaviour. Any way to do that differently?
> >
> > Ratis
> > =====
> >
> > No idea.
> >
> > On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang
> <weichiu@cloudera.com.invalid
> > >
> > wrote:
> >
> > > Most of the problems are downstream applications using Hadoop's private
> > > APIs.
> > >
> > > Tez:
> > >
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > > incompatible types: com.google.protobuf.ByteString cannot be converted
> > > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> > >
> > >
> > > Tez keeps track of job tokens internally.
> > > The change would look like this:
> > >
> > > private void recordJobShuffleInfo(JobID jobId, String user,
> > >     Token<JobTokenIdentifier> jobToken) throws IOException {
> > >   if (stateDb != null) {
> > >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> > >     /*TokenProto tokenProto = TokenProto.newBuilder()
> > >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> > >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> > >         .setKind(jobToken.getKind().toString())
> > >         .setService(jobToken.getService().toString())
> > >         .build();*/
> > >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> > >         .setUser(user).setJobToken(tokenProto).build();
> > >     try {
> > >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> > >     } catch (DBException e) {
> > >       throw new IOException("Error storing " + jobId, e);
> > >     }
> > >   }
> > >   addJobToken(jobId, user, jobToken);
> > > }
> > >
> > >
> > > HBase:
> > >
> > >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > > (this
> > >    is recently fixed in the master branch)
> > >    2.
> > >
> > >       [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > > (default-compile) on project hbase-server: Compilation failure:
> > > Compilation failure:
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > > incompatible types:
> > >
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > > cannot find symbol
> > >       [ERROR]   symbol:   method
> > >
> > >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> > >       [ERROR]   location: variable proto of type
> > >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> > >       [ERROR]   class file for
> > > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > > incompatible types:
> > >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > > cannot be converted to com.google.protobuf.MessageLite
> > >       [ERROR]
> > >
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > > invalid method reference
> > >       [ERROR]   non-static method get() cannot be referenced from a
> > > static context
> > >
> > >
> > > Ozone:
> > >
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > > -------------------------------------------------------------
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> > >
> > >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > > incompatible types: com.google.protobuf.ServiceException cannot be
> > > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> > >
> > >
> > > There's another error where Ozone uses the Hadoop RPC framework which
> > uses
> > > the hadoop.thirdparty protobuf.
> > >
> > > [ERROR] Failed to execute goal
> > > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > > (default-testCompile) on project hadoop-hdds-container-service:
> > Compilation
> > > failure
> > > [ERROR]
> > >
> > >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > > incompatible types: com.google.protobuf.BlockingService cannot be
> > converted
> > > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> > >
> > > BlockingService scmDatanodeService =
> > >     StorageContainerDatanodeProtocolService.
> > >         newReflectiveBlockingService(
> > >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> > >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> > >
> > >
> > >
> > > Ratis probably breaks as well since it depends on the Hadoop RPC
> > framework
> > > too.
> > >
> > > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <
> vinayakumarb@apache.org>
> > > wrote:
> > >
> > > > hi Wei-Chiu,
> > > >
> > > > Can you elaborate on what failures you are facing related to
> relocated
> > > > protobuf classes.. ?
> > > >
> > > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > > depending
> > > > on 2.5.0 version of protobuf still be able to access them.
> > > >
> > > > -vinay
> > > >
> > > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <weichiu@cloudera.com
> >
> > > > wrote:
> > > >
> > > >> I'm sorry for coming to this late. I missed this message. It should
> > have
> > > >> been a DISCUSS thread rather than NOTICE.
> > > >>
> > > >> Looks like this is inevitable. But we should make the downstream
> > > >> developers aware & make the update easier. As long as it is stated
> > > clearly
> > > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > > >>
> > > >> Here's what I suggest:
> > > >> (1) label the jira incompatible (just updated the jira) and updated
> > the
> > > >> release note to tell app developer how to update.
> > > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > > >>
> > > >> Tez doesn't use the removed Token API, but there's code that breaks
> > with
> > > >> the relocated protobuf class. The ProtobufHelper API will make this
> > > >> transition much easier.
> > > >>
> > > >> Other downstreamers that break with the relocated protobuf include:
> > > Ozone
> > > >> and HBase. but neither of them use the removed Token API.
> > > >>
> > > >>
> > > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <
> vinayakumarb@apache.org
> > >
> > > >> wrote:
> > > >>
> > > >>> Hi All,
> > > >>>
> > > >>>    This mail is to notify about the Removal of following public
> APIs
> > > from
> > > >>> Hadoop Common.
> > > >>>
> > > >>>      ClassName: org.apache.hadoop.security.token.Token
> > > >>>      APIs:
> > > >>>          public Token(TokenProto tokenPB);
> > > >>>          public TokenProto toTokenProto();
> > > >>>
> > > >>>    Reason: These APIs are having Generated protobuf classes in the
> > > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> > 3.3.0
> > > >>> release) these APIs are breaking the downstream builds, even though
> > > >>> downstreams dont use these APIs (just Loading Token class).
> > Downstreams
> > > >>> are
> > > >>> still referencing having older version (2.5.0) of protobuf, hence
> > build
> > > >>> is
> > > >>> being broken.
> > > >>>
> > > >>>     These APIs were added for the internal purpose(HADOOP-12563),
> to
> > > >>> support serializing tokens using protobuf in UGI Credentials.
> > > >>> Same purpose can be achieved using the Helper classes without
> > > introducing
> > > >>> protobuf classes in API signatures.
> > > >>>
> > > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > > >>> whenever
> > > >>> absolute necessary.
> > > >>>
> > > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has
> been
> > > >>> reported to solve downstream build failure.
> > > >>>
> > > >>> So since this API was added for internal purpose easy approach to
> > solve
> > > >>> this is to remove APIs and use helper classes. Otherwise, as
> > mentioned
> > > in
> > > >>> HADOOP-16621, workaround will add unnecessary codes to be
> maintained.
> > > >>>
> > > >>> If anyone using these APIs outside hadoop project accidentally,
> > please
> > > >>> reply to this mail immediately.
> > > >>>
> > > >>> If no objection by next week, will go ahead with removal of above
> > said
> > > >>> APIs
> > > >>> in HADOOP-16621.
> > > >>>
> > > >>> -Vinay
> > > >>>
> > > >>
> > >
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
Hi Wei-chu and steve,

Thanks for sharing insights.

I have also tried to compile and execute ozone pointing to
trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.

Other than just the usage of internal protobuf APIs, because of which
compilation would break, I found another major problem was, the Hadoop-rpc
implementations in downstreams which is based on non-shaded Protobuf
classes.

'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
'Message', which its expecting to be of 3.7 version and shaded package
(i.e. o.a.h.thirdparty.*).

So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
this issue will continue to occur, even after solving compilation issues
due to internal usage of private APIs with protobuf signatures.

I found a possible workaround for this problem.
Please check https://issues.apache.org/jira/browse/HADOOP-17046
  This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.
  Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
and later projects who wish to upgrade their protobufs to 3.x.

For Ozone compilation:
  I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
upgrade. These PRs will remove dependency on Hadoop for those internal APIs
and implemented their own copy in ozone with non-shaded protobuf.
    HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
<https://github.com/apache/hadoop-ozone/pull/933>2
    HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933

Also, I had run some tests on Ozone after applying these PRs and
HADOOP-17046 with 3.4.0, tests seems to pass.

Please help review these PRs.

Thanks,
-Vinay


On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <st...@cloudera.com.invalid>
wrote:

> Okay.
>
> I am not going to be a purist and say "what were they doing -using our
> private APIs?" because as we all know, with things like UGI tagged @private
> there's been no way to get something is done without getting into the
> private stuff.
>
> But why did we do the protobuf changes? So that we could update our private
> copy of protobuf with out breaking every single downstream application. The
> great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
> were absolutely nothing would build whatsoever, not until every downstream
> project had upgraded to the same version of the library.
>
> If you ever want to see an upgrade which makes a guava update seem a minor
> detail, protobuf upgrades are it. Hence the shading
>
> HBase
> =====
>
> it looks like HBase has been using deep internal stuff. That is,
> "unfortunate". I think in that world we have to look and say is there
> something specific we can do here to help HBase in a way we could also
> backport. They shouldn't need those IPC internals.
>
> Tez & Tokens
> ============
>
> I didn't know Tez was using those protobuf APIs internally. That is,
> "unfortunate".
>
> What is key is this: without us moving those methods things like Spark
> wouldn't work. And they weren't even using the methods, just trying to work
> with Token for job submission.
>
> All Tez should need is a byte array serialization of a token. Given Token
> is also Writable, that could be done via WritableUtils in a way which will
> also work with older releases.
>
> Ozone
> =====
>
> When these were part of/in-sync with the hadoop build there wouldn't have
> been problems. Now there are. Again, they're going in deep, but here
> clearly to simulate some behaviour. Any way to do that differently?
>
> Ratis
> =====
>
> No idea.
>
> On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <weichiu@cloudera.com.invalid
> >
> wrote:
>
> > Most of the problems are downstream applications using Hadoop's private
> > APIs.
> >
> > Tez:
> >
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > incompatible types: com.google.protobuf.ByteString cannot be converted
> > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> >
> >
> > Tez keeps track of job tokens internally.
> > The change would look like this:
> >
> > private void recordJobShuffleInfo(JobID jobId, String user,
> >     Token<JobTokenIdentifier> jobToken) throws IOException {
> >   if (stateDb != null) {
> >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> >     /*TokenProto tokenProto = TokenProto.newBuilder()
> >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> >         .setKind(jobToken.getKind().toString())
> >         .setService(jobToken.getService().toString())
> >         .build();*/
> >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> >         .setUser(user).setJobToken(tokenProto).build();
> >     try {
> >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> >     } catch (DBException e) {
> >       throw new IOException("Error storing " + jobId, e);
> >     }
> >   }
> >   addJobToken(jobId, user, jobToken);
> > }
> >
> >
> > HBase:
> >
> >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > (this
> >    is recently fixed in the master branch)
> >    2.
> >
> >       [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > (default-compile) on project hbase-server: Compilation failure:
> > Compilation failure:
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > incompatible types:
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > invalid method reference
> >       [ERROR]   non-static method get() cannot be referenced from a
> > static context
> >
> >
> > Ozone:
> >
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> >
> >
> > There's another error where Ozone uses the Hadoop RPC framework which
> uses
> > the hadoop.thirdparty protobuf.
> >
> > [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > (default-testCompile) on project hadoop-hdds-container-service:
> Compilation
> > failure
> > [ERROR]
> >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > incompatible types: com.google.protobuf.BlockingService cannot be
> converted
> > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> >
> > BlockingService scmDatanodeService =
> >     StorageContainerDatanodeProtocolService.
> >         newReflectiveBlockingService(
> >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> >
> >
> >
> > Ratis probably breaks as well since it depends on the Hadoop RPC
> framework
> > too.
> >
> > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > hi Wei-Chiu,
> > >
> > > Can you elaborate on what failures you are facing related to relocated
> > > protobuf classes.. ?
> > >
> > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > depending
> > > on 2.5.0 version of protobuf still be able to access them.
> > >
> > > -vinay
> > >
> > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > > wrote:
> > >
> > >> I'm sorry for coming to this late. I missed this message. It should
> have
> > >> been a DISCUSS thread rather than NOTICE.
> > >>
> > >> Looks like this is inevitable. But we should make the downstream
> > >> developers aware & make the update easier. As long as it is stated
> > clearly
> > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > >>
> > >> Here's what I suggest:
> > >> (1) label the jira incompatible (just updated the jira) and updated
> the
> > >> release note to tell app developer how to update.
> > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > >>
> > >> Tez doesn't use the removed Token API, but there's code that breaks
> with
> > >> the relocated protobuf class. The ProtobufHelper API will make this
> > >> transition much easier.
> > >>
> > >> Other downstreamers that break with the relocated protobuf include:
> > Ozone
> > >> and HBase. but neither of them use the removed Token API.
> > >>
> > >>
> > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vinayakumarb@apache.org
> >
> > >> wrote:
> > >>
> > >>> Hi All,
> > >>>
> > >>>    This mail is to notify about the Removal of following public APIs
> > from
> > >>> Hadoop Common.
> > >>>
> > >>>      ClassName: org.apache.hadoop.security.token.Token
> > >>>      APIs:
> > >>>          public Token(TokenProto tokenPB);
> > >>>          public TokenProto toTokenProto();
> > >>>
> > >>>    Reason: These APIs are having Generated protobuf classes in the
> > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> 3.3.0
> > >>> release) these APIs are breaking the downstream builds, even though
> > >>> downstreams dont use these APIs (just Loading Token class).
> Downstreams
> > >>> are
> > >>> still referencing having older version (2.5.0) of protobuf, hence
> build
> > >>> is
> > >>> being broken.
> > >>>
> > >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> > >>> support serializing tokens using protobuf in UGI Credentials.
> > >>> Same purpose can be achieved using the Helper classes without
> > introducing
> > >>> protobuf classes in API signatures.
> > >>>
> > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > >>> whenever
> > >>> absolute necessary.
> > >>>
> > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> > >>> reported to solve downstream build failure.
> > >>>
> > >>> So since this API was added for internal purpose easy approach to
> solve
> > >>> this is to remove APIs and use helper classes. Otherwise, as
> mentioned
> > in
> > >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> > >>>
> > >>> If anyone using these APIs outside hadoop project accidentally,
> please
> > >>> reply to this mail immediately.
> > >>>
> > >>> If no objection by next week, will go ahead with removal of above
> said
> > >>> APIs
> > >>> in HADOOP-16621.
> > >>>
> > >>> -Vinay
> > >>>
> > >>
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
Hi Wei-chu and steve,

Thanks for sharing insights.

I have also tried to compile and execute ozone pointing to
trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.

Other than just the usage of internal protobuf APIs, because of which
compilation would break, I found another major problem was, the Hadoop-rpc
implementations in downstreams which is based on non-shaded Protobuf
classes.

'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
'Message', which its expecting to be of 3.7 version and shaded package
(i.e. o.a.h.thirdparty.*).

So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
this issue will continue to occur, even after solving compilation issues
due to internal usage of private APIs with protobuf signatures.

I found a possible workaround for this problem.
Please check https://issues.apache.org/jira/browse/HADOOP-17046
  This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.
  Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
and later projects who wish to upgrade their protobufs to 3.x.

For Ozone compilation:
  I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
upgrade. These PRs will remove dependency on Hadoop for those internal APIs
and implemented their own copy in ozone with non-shaded protobuf.
    HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
<https://github.com/apache/hadoop-ozone/pull/933>2
    HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933

Also, I had run some tests on Ozone after applying these PRs and
HADOOP-17046 with 3.4.0, tests seems to pass.

Please help review these PRs.

Thanks,
-Vinay


On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <st...@cloudera.com.invalid>
wrote:

> Okay.
>
> I am not going to be a purist and say "what were they doing -using our
> private APIs?" because as we all know, with things like UGI tagged @private
> there's been no way to get something is done without getting into the
> private stuff.
>
> But why did we do the protobuf changes? So that we could update our private
> copy of protobuf with out breaking every single downstream application. The
> great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
> were absolutely nothing would build whatsoever, not until every downstream
> project had upgraded to the same version of the library.
>
> If you ever want to see an upgrade which makes a guava update seem a minor
> detail, protobuf upgrades are it. Hence the shading
>
> HBase
> =====
>
> it looks like HBase has been using deep internal stuff. That is,
> "unfortunate". I think in that world we have to look and say is there
> something specific we can do here to help HBase in a way we could also
> backport. They shouldn't need those IPC internals.
>
> Tez & Tokens
> ============
>
> I didn't know Tez was using those protobuf APIs internally. That is,
> "unfortunate".
>
> What is key is this: without us moving those methods things like Spark
> wouldn't work. And they weren't even using the methods, just trying to work
> with Token for job submission.
>
> All Tez should need is a byte array serialization of a token. Given Token
> is also Writable, that could be done via WritableUtils in a way which will
> also work with older releases.
>
> Ozone
> =====
>
> When these were part of/in-sync with the hadoop build there wouldn't have
> been problems. Now there are. Again, they're going in deep, but here
> clearly to simulate some behaviour. Any way to do that differently?
>
> Ratis
> =====
>
> No idea.
>
> On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <weichiu@cloudera.com.invalid
> >
> wrote:
>
> > Most of the problems are downstream applications using Hadoop's private
> > APIs.
> >
> > Tez:
> >
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > incompatible types: com.google.protobuf.ByteString cannot be converted
> > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> >
> >
> > Tez keeps track of job tokens internally.
> > The change would look like this:
> >
> > private void recordJobShuffleInfo(JobID jobId, String user,
> >     Token<JobTokenIdentifier> jobToken) throws IOException {
> >   if (stateDb != null) {
> >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> >     /*TokenProto tokenProto = TokenProto.newBuilder()
> >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> >         .setKind(jobToken.getKind().toString())
> >         .setService(jobToken.getService().toString())
> >         .build();*/
> >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> >         .setUser(user).setJobToken(tokenProto).build();
> >     try {
> >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> >     } catch (DBException e) {
> >       throw new IOException("Error storing " + jobId, e);
> >     }
> >   }
> >   addJobToken(jobId, user, jobToken);
> > }
> >
> >
> > HBase:
> >
> >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > (this
> >    is recently fixed in the master branch)
> >    2.
> >
> >       [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > (default-compile) on project hbase-server: Compilation failure:
> > Compilation failure:
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > incompatible types:
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > invalid method reference
> >       [ERROR]   non-static method get() cannot be referenced from a
> > static context
> >
> >
> > Ozone:
> >
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> >
> >
> > There's another error where Ozone uses the Hadoop RPC framework which
> uses
> > the hadoop.thirdparty protobuf.
> >
> > [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > (default-testCompile) on project hadoop-hdds-container-service:
> Compilation
> > failure
> > [ERROR]
> >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > incompatible types: com.google.protobuf.BlockingService cannot be
> converted
> > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> >
> > BlockingService scmDatanodeService =
> >     StorageContainerDatanodeProtocolService.
> >         newReflectiveBlockingService(
> >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> >
> >
> >
> > Ratis probably breaks as well since it depends on the Hadoop RPC
> framework
> > too.
> >
> > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > hi Wei-Chiu,
> > >
> > > Can you elaborate on what failures you are facing related to relocated
> > > protobuf classes.. ?
> > >
> > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > depending
> > > on 2.5.0 version of protobuf still be able to access them.
> > >
> > > -vinay
> > >
> > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > > wrote:
> > >
> > >> I'm sorry for coming to this late. I missed this message. It should
> have
> > >> been a DISCUSS thread rather than NOTICE.
> > >>
> > >> Looks like this is inevitable. But we should make the downstream
> > >> developers aware & make the update easier. As long as it is stated
> > clearly
> > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > >>
> > >> Here's what I suggest:
> > >> (1) label the jira incompatible (just updated the jira) and updated
> the
> > >> release note to tell app developer how to update.
> > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > >>
> > >> Tez doesn't use the removed Token API, but there's code that breaks
> with
> > >> the relocated protobuf class. The ProtobufHelper API will make this
> > >> transition much easier.
> > >>
> > >> Other downstreamers that break with the relocated protobuf include:
> > Ozone
> > >> and HBase. but neither of them use the removed Token API.
> > >>
> > >>
> > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vinayakumarb@apache.org
> >
> > >> wrote:
> > >>
> > >>> Hi All,
> > >>>
> > >>>    This mail is to notify about the Removal of following public APIs
> > from
> > >>> Hadoop Common.
> > >>>
> > >>>      ClassName: org.apache.hadoop.security.token.Token
> > >>>      APIs:
> > >>>          public Token(TokenProto tokenPB);
> > >>>          public TokenProto toTokenProto();
> > >>>
> > >>>    Reason: These APIs are having Generated protobuf classes in the
> > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> 3.3.0
> > >>> release) these APIs are breaking the downstream builds, even though
> > >>> downstreams dont use these APIs (just Loading Token class).
> Downstreams
> > >>> are
> > >>> still referencing having older version (2.5.0) of protobuf, hence
> build
> > >>> is
> > >>> being broken.
> > >>>
> > >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> > >>> support serializing tokens using protobuf in UGI Credentials.
> > >>> Same purpose can be achieved using the Helper classes without
> > introducing
> > >>> protobuf classes in API signatures.
> > >>>
> > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > >>> whenever
> > >>> absolute necessary.
> > >>>
> > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> > >>> reported to solve downstream build failure.
> > >>>
> > >>> So since this API was added for internal purpose easy approach to
> solve
> > >>> this is to remove APIs and use helper classes. Otherwise, as
> mentioned
> > in
> > >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> > >>>
> > >>> If anyone using these APIs outside hadoop project accidentally,
> please
> > >>> reply to this mail immediately.
> > >>>
> > >>> If no objection by next week, will go ahead with removal of above
> said
> > >>> APIs
> > >>> in HADOOP-16621.
> > >>>
> > >>> -Vinay
> > >>>
> > >>
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
Hi Wei-chu and steve,

Thanks for sharing insights.

I have also tried to compile and execute ozone pointing to
trunk(3.4.0-SNAPSHOT) which have shaded and upgraded protobuf.

Other than just the usage of internal protobuf APIs, because of which
compilation would break, I found another major problem was, the Hadoop-rpc
implementations in downstreams which is based on non-shaded Protobuf
classes.

'ProtobufRpcEngine' takes arguments and tries to typecast to Protobuf
'Message', which its expecting to be of 3.7 version and shaded package
(i.e. o.a.h.thirdparty.*).

So,unless downstreams upgrade their protobuf classes to 'hadoop-thirdparty'
this issue will continue to occur, even after solving compilation issues
due to internal usage of private APIs with protobuf signatures.

I found a possible workaround for this problem.
Please check https://issues.apache.org/jira/browse/HADOOP-17046
  This Jira proposes to keep existing ProtobuRpcEngine as-is (without
shading and with protobuf-2.5.0 implementation) to support downstream
implementations.
  Use new ProtobufRpcEngine2 to use shaded protobuf classes within Hadoop
and later projects who wish to upgrade their protobufs to 3.x.

For Ozone compilation:
  I have submitted to PRs to make preparations to adopt to Hadoop 3.3+
upgrade. These PRs will remove dependency on Hadoop for those internal APIs
and implemented their own copy in ozone with non-shaded protobuf.
    HDDS-3603: https://github.com/apache/hadoop-ozone/pull/93
<https://github.com/apache/hadoop-ozone/pull/933>2
    HDDS-3604: https://github.com/apache/hadoop-ozone/pull/933

Also, I had run some tests on Ozone after applying these PRs and
HADOOP-17046 with 3.4.0, tests seems to pass.

Please help review these PRs.

Thanks,
-Vinay


On Wed, Apr 29, 2020 at 5:02 PM Steve Loughran <st...@cloudera.com.invalid>
wrote:

> Okay.
>
> I am not going to be a purist and say "what were they doing -using our
> private APIs?" because as we all know, with things like UGI tagged @private
> there's been no way to get something is done without getting into the
> private stuff.
>
> But why did we do the protobuf changes? So that we could update our private
> copy of protobuf with out breaking every single downstream application. The
> great protobuf upgrade to 2.5 is not something we wanted to repeat. When
> was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
> were absolutely nothing would build whatsoever, not until every downstream
> project had upgraded to the same version of the library.
>
> If you ever want to see an upgrade which makes a guava update seem a minor
> detail, protobuf upgrades are it. Hence the shading
>
> HBase
> =====
>
> it looks like HBase has been using deep internal stuff. That is,
> "unfortunate". I think in that world we have to look and say is there
> something specific we can do here to help HBase in a way we could also
> backport. They shouldn't need those IPC internals.
>
> Tez & Tokens
> ============
>
> I didn't know Tez was using those protobuf APIs internally. That is,
> "unfortunate".
>
> What is key is this: without us moving those methods things like Spark
> wouldn't work. And they weren't even using the methods, just trying to work
> with Token for job submission.
>
> All Tez should need is a byte array serialization of a token. Given Token
> is also Writable, that could be done via WritableUtils in a way which will
> also work with older releases.
>
> Ozone
> =====
>
> When these were part of/in-sync with the hadoop build there wouldn't have
> been problems. Now there are. Again, they're going in deep, but here
> clearly to simulate some behaviour. Any way to do that differently?
>
> Ratis
> =====
>
> No idea.
>
> On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <weichiu@cloudera.com.invalid
> >
> wrote:
>
> > Most of the problems are downstream applications using Hadoop's private
> > APIs.
> >
> > Tez:
> >
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> > incompatible types: com.google.protobuf.ByteString cannot be converted
> > to org.apache.hadoop.thirdparty.protobuf.ByteString
> > 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
> >
> >
> > Tez keeps track of job tokens internally.
> > The change would look like this:
> >
> > private void recordJobShuffleInfo(JobID jobId, String user,
> >     Token<JobTokenIdentifier> jobToken) throws IOException {
> >   if (stateDb != null) {
> >     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
> >     /*TokenProto tokenProto = TokenProto.newBuilder()
> >         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
> >         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
> >         .setKind(jobToken.getKind().toString())
> >         .setService(jobToken.getService().toString())
> >         .build();*/
> >     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
> >         .setUser(user).setJobToken(tokenProto).build();
> >     try {
> >       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
> >     } catch (DBException e) {
> >       throw new IOException("Error storing " + jobId, e);
> >     }
> >   }
> >   addJobToken(jobId, user, jobToken);
> > }
> >
> >
> > HBase:
> >
> >    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> > (this
> >    is recently fixed in the master branch)
> >    2.
> >
> >       [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> > (default-compile) on project hbase-server: Compilation failure:
> > Compilation failure:
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> > cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> > cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ByteString not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> > incompatible types:
> >
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> > cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> > cannot find symbol
> >       [ERROR]   symbol:   method
> >
> >
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
> >       [ERROR]   location: variable proto of type
> >
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> > cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
> >       [ERROR]   class file for
> > org.apache.hadoop.thirdparty.protobuf.Descriptors not found
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> > incompatible types:
> > org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> > cannot be converted to com.google.protobuf.MessageLite
> >       [ERROR]
> >
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> > invalid method reference
> >       [ERROR]   non-static method get() cannot be referenced from a
> > static context
> >
> >
> > Ozone:
> >
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> > -------------------------------------------------------------
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
> >
> >
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> > incompatible types: com.google.protobuf.ServiceException cannot be
> > converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> > 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
> >
> >
> > There's another error where Ozone uses the Hadoop RPC framework which
> uses
> > the hadoop.thirdparty protobuf.
> >
> > [ERROR] Failed to execute goal
> > org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> > (default-testCompile) on project hadoop-hdds-container-service:
> Compilation
> > failure
> > [ERROR]
> >
> >
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> > incompatible types: com.google.protobuf.BlockingService cannot be
> converted
> > to org.apache.hadoop.thirdparty.protobuf.BlockingService
> >
> > BlockingService scmDatanodeService =
> >     StorageContainerDatanodeProtocolService.
> >         newReflectiveBlockingService(
> >             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
> >                 server, Mockito.mock(ProtocolMessageMetrics.class)));
> >
> >
> >
> > Ratis probably breaks as well since it depends on the Hadoop RPC
> framework
> > too.
> >
> > On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> > wrote:
> >
> > > hi Wei-Chiu,
> > >
> > > Can you elaborate on what failures you are facing related to relocated
> > > protobuf classes.. ?
> > >
> > > IFAIK, if the issue with location of protobuf classes, still old jar
> > > protobuf-2.5.0.jar will be available in classpath. So downstream
> > depending
> > > on 2.5.0 version of protobuf still be able to access them.
> > >
> > > -vinay
> > >
> > > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > > wrote:
> > >
> > >> I'm sorry for coming to this late. I missed this message. It should
> have
> > >> been a DISCUSS thread rather than NOTICE.
> > >>
> > >> Looks like this is inevitable. But we should make the downstream
> > >> developers aware & make the update easier. As long as it is stated
> > clearly
> > >> how to update the code to support Hadoop 3.3, I am okay with that.
> > >>
> > >> Here's what I suggest:
> > >> (1) label the jira incompatible (just updated the jira) and updated
> the
> > >> release note to tell app developer how to update.
> > >> (2) declare ProtobufHelper a public API HADOOP-17019
> > >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> > >>
> > >> Tez doesn't use the removed Token API, but there's code that breaks
> with
> > >> the relocated protobuf class. The ProtobufHelper API will make this
> > >> transition much easier.
> > >>
> > >> Other downstreamers that break with the relocated protobuf include:
> > Ozone
> > >> and HBase. but neither of them use the removed Token API.
> > >>
> > >>
> > >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vinayakumarb@apache.org
> >
> > >> wrote:
> > >>
> > >>> Hi All,
> > >>>
> > >>>    This mail is to notify about the Removal of following public APIs
> > from
> > >>> Hadoop Common.
> > >>>
> > >>>      ClassName: org.apache.hadoop.security.token.Token
> > >>>      APIs:
> > >>>          public Token(TokenProto tokenPB);
> > >>>          public TokenProto toTokenProto();
> > >>>
> > >>>    Reason: These APIs are having Generated protobuf classes in the
> > >>> signature. Right now due to protobuf upgrade in trunk (soon to be
> 3.3.0
> > >>> release) these APIs are breaking the downstream builds, even though
> > >>> downstreams dont use these APIs (just Loading Token class).
> Downstreams
> > >>> are
> > >>> still referencing having older version (2.5.0) of protobuf, hence
> build
> > >>> is
> > >>> being broken.
> > >>>
> > >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> > >>> support serializing tokens using protobuf in UGI Credentials.
> > >>> Same purpose can be achieved using the Helper classes without
> > introducing
> > >>> protobuf classes in API signatures.
> > >>>
> > >>> Token.java is marked as Evolving, so I believe APIs can be changed
> > >>> whenever
> > >>> absolute necessary.
> > >>>
> > >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> > >>> reported to solve downstream build failure.
> > >>>
> > >>> So since this API was added for internal purpose easy approach to
> solve
> > >>> this is to remove APIs and use helper classes. Otherwise, as
> mentioned
> > in
> > >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> > >>>
> > >>> If anyone using these APIs outside hadoop project accidentally,
> please
> > >>> reply to this mail immediately.
> > >>>
> > >>> If no objection by next week, will go ahead with removal of above
> said
> > >>> APIs
> > >>> in HADOOP-16621.
> > >>>
> > >>> -Vinay
> > >>>
> > >>
> >
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Steve Loughran <st...@cloudera.com.INVALID>.
Okay.

I am not going to be a purist and say "what were they doing -using our
private APIs?" because as we all know, with things like UGI tagged @private
there's been no way to get something is done without getting into the
private stuff.

But why did we do the protobuf changes? So that we could update our private
copy of protobuf with out breaking every single downstream application. The
great protobuf upgrade to 2.5 is not something we wanted to repeat. When
was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
were absolutely nothing would build whatsoever, not until every downstream
project had upgraded to the same version of the library.

If you ever want to see an upgrade which makes a guava update seem a minor
detail, protobuf upgrades are it. Hence the shading

HBase
=====

it looks like HBase has been using deep internal stuff. That is,
"unfortunate". I think in that world we have to look and say is there
something specific we can do here to help HBase in a way we could also
backport. They shouldn't need those IPC internals.

Tez & Tokens
============

I didn't know Tez was using those protobuf APIs internally. That is,
"unfortunate".

What is key is this: without us moving those methods things like Spark
wouldn't work. And they weren't even using the methods, just trying to work
with Token for job submission.

All Tez should need is a byte array serialization of a token. Given Token
is also Writable, that could be done via WritableUtils in a way which will
also work with older releases.

Ozone
=====

When these were part of/in-sync with the hadoop build there wouldn't have
been problems. Now there are. Again, they're going in deep, but here
clearly to simulate some behaviour. Any way to do that differently?

Ratis
=====

No idea.

On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <we...@cloudera.com.invalid>
wrote:

> Most of the problems are downstream applications using Hadoop's private
> APIs.
>
> Tez:
>
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> -------------------------------------------------------------
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> incompatible types: com.google.protobuf.ByteString cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.ByteString
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
>
>
> Tez keeps track of job tokens internally.
> The change would look like this:
>
> private void recordJobShuffleInfo(JobID jobId, String user,
>     Token<JobTokenIdentifier> jobToken) throws IOException {
>   if (stateDb != null) {
>     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
>     /*TokenProto tokenProto = TokenProto.newBuilder()
>         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
>         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
>         .setKind(jobToken.getKind().toString())
>         .setService(jobToken.getService().toString())
>         .build();*/
>     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
>         .setUser(user).setJobToken(tokenProto).build();
>     try {
>       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
>     } catch (DBException e) {
>       throw new IOException("Error storing " + jobId, e);
>     }
>   }
>   addJobToken(jobId, user, jobToken);
> }
>
>
> HBase:
>
>    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> (this
>    is recently fixed in the master branch)
>    2.
>
>       [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> (default-compile) on project hbase-server: Compilation failure:
> Compilation failure:
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ByteString not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.Descriptors not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> incompatible types:
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> invalid method reference
>       [ERROR]   non-static method get() cannot be referenced from a
> static context
>
>
> Ozone:
>
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> -------------------------------------------------------------
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
>
>
> There's another error where Ozone uses the Hadoop RPC framework which uses
> the hadoop.thirdparty protobuf.
>
> [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> (default-testCompile) on project hadoop-hdds-container-service: Compilation
> failure
> [ERROR]
>
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> incompatible types: com.google.protobuf.BlockingService cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.BlockingService
>
> BlockingService scmDatanodeService =
>     StorageContainerDatanodeProtocolService.
>         newReflectiveBlockingService(
>             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
>                 server, Mockito.mock(ProtocolMessageMetrics.class)));
>
>
>
> Ratis probably breaks as well since it depends on the Hadoop RPC framework
> too.
>
> On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > hi Wei-Chiu,
> >
> > Can you elaborate on what failures you are facing related to relocated
> > protobuf classes.. ?
> >
> > IFAIK, if the issue with location of protobuf classes, still old jar
> > protobuf-2.5.0.jar will be available in classpath. So downstream
> depending
> > on 2.5.0 version of protobuf still be able to access them.
> >
> > -vinay
> >
> > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > wrote:
> >
> >> I'm sorry for coming to this late. I missed this message. It should have
> >> been a DISCUSS thread rather than NOTICE.
> >>
> >> Looks like this is inevitable. But we should make the downstream
> >> developers aware & make the update easier. As long as it is stated
> clearly
> >> how to update the code to support Hadoop 3.3, I am okay with that.
> >>
> >> Here's what I suggest:
> >> (1) label the jira incompatible (just updated the jira) and updated the
> >> release note to tell app developer how to update.
> >> (2) declare ProtobufHelper a public API HADOOP-17019
> >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> >>
> >> Tez doesn't use the removed Token API, but there's code that breaks with
> >> the relocated protobuf class. The ProtobufHelper API will make this
> >> transition much easier.
> >>
> >> Other downstreamers that break with the relocated protobuf include:
> Ozone
> >> and HBase. but neither of them use the removed Token API.
> >>
> >>
> >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> >> wrote:
> >>
> >>> Hi All,
> >>>
> >>>    This mail is to notify about the Removal of following public APIs
> from
> >>> Hadoop Common.
> >>>
> >>>      ClassName: org.apache.hadoop.security.token.Token
> >>>      APIs:
> >>>          public Token(TokenProto tokenPB);
> >>>          public TokenProto toTokenProto();
> >>>
> >>>    Reason: These APIs are having Generated protobuf classes in the
> >>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
> >>> release) these APIs are breaking the downstream builds, even though
> >>> downstreams dont use these APIs (just Loading Token class). Downstreams
> >>> are
> >>> still referencing having older version (2.5.0) of protobuf, hence build
> >>> is
> >>> being broken.
> >>>
> >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> >>> support serializing tokens using protobuf in UGI Credentials.
> >>> Same purpose can be achieved using the Helper classes without
> introducing
> >>> protobuf classes in API signatures.
> >>>
> >>> Token.java is marked as Evolving, so I believe APIs can be changed
> >>> whenever
> >>> absolute necessary.
> >>>
> >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> >>> reported to solve downstream build failure.
> >>>
> >>> So since this API was added for internal purpose easy approach to solve
> >>> this is to remove APIs and use helper classes. Otherwise, as mentioned
> in
> >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> >>>
> >>> If anyone using these APIs outside hadoop project accidentally, please
> >>> reply to this mail immediately.
> >>>
> >>> If no objection by next week, will go ahead with removal of above said
> >>> APIs
> >>> in HADOOP-16621.
> >>>
> >>> -Vinay
> >>>
> >>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Steve Loughran <st...@cloudera.com.INVALID>.
Okay.

I am not going to be a purist and say "what were they doing -using our
private APIs?" because as we all know, with things like UGI tagged @private
there's been no way to get something is done without getting into the
private stuff.

But why did we do the protobuf changes? So that we could update our private
copy of protobuf with out breaking every single downstream application. The
great protobuf upgrade to 2.5 is not something we wanted to repeat. When
was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
were absolutely nothing would build whatsoever, not until every downstream
project had upgraded to the same version of the library.

If you ever want to see an upgrade which makes a guava update seem a minor
detail, protobuf upgrades are it. Hence the shading

HBase
=====

it looks like HBase has been using deep internal stuff. That is,
"unfortunate". I think in that world we have to look and say is there
something specific we can do here to help HBase in a way we could also
backport. They shouldn't need those IPC internals.

Tez & Tokens
============

I didn't know Tez was using those protobuf APIs internally. That is,
"unfortunate".

What is key is this: without us moving those methods things like Spark
wouldn't work. And they weren't even using the methods, just trying to work
with Token for job submission.

All Tez should need is a byte array serialization of a token. Given Token
is also Writable, that could be done via WritableUtils in a way which will
also work with older releases.

Ozone
=====

When these were part of/in-sync with the hadoop build there wouldn't have
been problems. Now there are. Again, they're going in deep, but here
clearly to simulate some behaviour. Any way to do that differently?

Ratis
=====

No idea.

On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <we...@cloudera.com.invalid>
wrote:

> Most of the problems are downstream applications using Hadoop's private
> APIs.
>
> Tez:
>
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> -------------------------------------------------------------
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> incompatible types: com.google.protobuf.ByteString cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.ByteString
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
>
>
> Tez keeps track of job tokens internally.
> The change would look like this:
>
> private void recordJobShuffleInfo(JobID jobId, String user,
>     Token<JobTokenIdentifier> jobToken) throws IOException {
>   if (stateDb != null) {
>     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
>     /*TokenProto tokenProto = TokenProto.newBuilder()
>         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
>         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
>         .setKind(jobToken.getKind().toString())
>         .setService(jobToken.getService().toString())
>         .build();*/
>     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
>         .setUser(user).setJobToken(tokenProto).build();
>     try {
>       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
>     } catch (DBException e) {
>       throw new IOException("Error storing " + jobId, e);
>     }
>   }
>   addJobToken(jobId, user, jobToken);
> }
>
>
> HBase:
>
>    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> (this
>    is recently fixed in the master branch)
>    2.
>
>       [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> (default-compile) on project hbase-server: Compilation failure:
> Compilation failure:
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ByteString not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.Descriptors not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> incompatible types:
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> invalid method reference
>       [ERROR]   non-static method get() cannot be referenced from a
> static context
>
>
> Ozone:
>
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> -------------------------------------------------------------
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
>
>
> There's another error where Ozone uses the Hadoop RPC framework which uses
> the hadoop.thirdparty protobuf.
>
> [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> (default-testCompile) on project hadoop-hdds-container-service: Compilation
> failure
> [ERROR]
>
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> incompatible types: com.google.protobuf.BlockingService cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.BlockingService
>
> BlockingService scmDatanodeService =
>     StorageContainerDatanodeProtocolService.
>         newReflectiveBlockingService(
>             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
>                 server, Mockito.mock(ProtocolMessageMetrics.class)));
>
>
>
> Ratis probably breaks as well since it depends on the Hadoop RPC framework
> too.
>
> On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > hi Wei-Chiu,
> >
> > Can you elaborate on what failures you are facing related to relocated
> > protobuf classes.. ?
> >
> > IFAIK, if the issue with location of protobuf classes, still old jar
> > protobuf-2.5.0.jar will be available in classpath. So downstream
> depending
> > on 2.5.0 version of protobuf still be able to access them.
> >
> > -vinay
> >
> > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > wrote:
> >
> >> I'm sorry for coming to this late. I missed this message. It should have
> >> been a DISCUSS thread rather than NOTICE.
> >>
> >> Looks like this is inevitable. But we should make the downstream
> >> developers aware & make the update easier. As long as it is stated
> clearly
> >> how to update the code to support Hadoop 3.3, I am okay with that.
> >>
> >> Here's what I suggest:
> >> (1) label the jira incompatible (just updated the jira) and updated the
> >> release note to tell app developer how to update.
> >> (2) declare ProtobufHelper a public API HADOOP-17019
> >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> >>
> >> Tez doesn't use the removed Token API, but there's code that breaks with
> >> the relocated protobuf class. The ProtobufHelper API will make this
> >> transition much easier.
> >>
> >> Other downstreamers that break with the relocated protobuf include:
> Ozone
> >> and HBase. but neither of them use the removed Token API.
> >>
> >>
> >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> >> wrote:
> >>
> >>> Hi All,
> >>>
> >>>    This mail is to notify about the Removal of following public APIs
> from
> >>> Hadoop Common.
> >>>
> >>>      ClassName: org.apache.hadoop.security.token.Token
> >>>      APIs:
> >>>          public Token(TokenProto tokenPB);
> >>>          public TokenProto toTokenProto();
> >>>
> >>>    Reason: These APIs are having Generated protobuf classes in the
> >>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
> >>> release) these APIs are breaking the downstream builds, even though
> >>> downstreams dont use these APIs (just Loading Token class). Downstreams
> >>> are
> >>> still referencing having older version (2.5.0) of protobuf, hence build
> >>> is
> >>> being broken.
> >>>
> >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> >>> support serializing tokens using protobuf in UGI Credentials.
> >>> Same purpose can be achieved using the Helper classes without
> introducing
> >>> protobuf classes in API signatures.
> >>>
> >>> Token.java is marked as Evolving, so I believe APIs can be changed
> >>> whenever
> >>> absolute necessary.
> >>>
> >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> >>> reported to solve downstream build failure.
> >>>
> >>> So since this API was added for internal purpose easy approach to solve
> >>> this is to remove APIs and use helper classes. Otherwise, as mentioned
> in
> >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> >>>
> >>> If anyone using these APIs outside hadoop project accidentally, please
> >>> reply to this mail immediately.
> >>>
> >>> If no objection by next week, will go ahead with removal of above said
> >>> APIs
> >>> in HADOOP-16621.
> >>>
> >>> -Vinay
> >>>
> >>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Steve Loughran <st...@cloudera.com.INVALID>.
Okay.

I am not going to be a purist and say "what were they doing -using our
private APIs?" because as we all know, with things like UGI tagged @private
there's been no way to get something is done without getting into the
private stuff.

But why did we do the protobuf changes? So that we could update our private
copy of protobuf with out breaking every single downstream application. The
great protobuf upgrade to 2.5 is not something we wanted to repeat. When
was that? before hadoop-2.2 shipped? I certainly remember a couple of weeks
were absolutely nothing would build whatsoever, not until every downstream
project had upgraded to the same version of the library.

If you ever want to see an upgrade which makes a guava update seem a minor
detail, protobuf upgrades are it. Hence the shading

HBase
=====

it looks like HBase has been using deep internal stuff. That is,
"unfortunate". I think in that world we have to look and say is there
something specific we can do here to help HBase in a way we could also
backport. They shouldn't need those IPC internals.

Tez & Tokens
============

I didn't know Tez was using those protobuf APIs internally. That is,
"unfortunate".

What is key is this: without us moving those methods things like Spark
wouldn't work. And they weren't even using the methods, just trying to work
with Token for job submission.

All Tez should need is a byte array serialization of a token. Given Token
is also Writable, that could be done via WritableUtils in a way which will
also work with older releases.

Ozone
=====

When these were part of/in-sync with the hadoop build there wouldn't have
been problems. Now there are. Again, they're going in deep, but here
clearly to simulate some behaviour. Any way to do that differently?

Ratis
=====

No idea.

On Wed, 29 Apr 2020 at 07:12, Wei-Chiu Chuang <we...@cloudera.com.invalid>
wrote:

> Most of the problems are downstream applications using Hadoop's private
> APIs.
>
> Tez:
>
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
> -------------------------------------------------------------
> 17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
> incompatible types: com.google.protobuf.ByteString cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.ByteString
> 17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error
>
>
> Tez keeps track of job tokens internally.
> The change would look like this:
>
> private void recordJobShuffleInfo(JobID jobId, String user,
>     Token<JobTokenIdentifier> jobToken) throws IOException {
>   if (stateDb != null) {
>     TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
>     /*TokenProto tokenProto = TokenProto.newBuilder()
>         .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
>         .setPassword(ByteString.copyFrom(jobToken.getPassword()))
>         .setKind(jobToken.getKind().toString())
>         .setService(jobToken.getService().toString())
>         .build();*/
>     JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
>         .setUser(user).setJobToken(tokenProto).build();
>     try {
>       stateDb.put(bytes(jobId.toString()), proto.toByteArray());
>     } catch (DBException e) {
>       throw new IOException("Error storing " + jobId, e);
>     }
>   }
>   addJobToken(jobId, user, jobToken);
> }
>
>
> HBase:
>
>    1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833>
> (this
>    is recently fixed in the master branch)
>    2.
>
>       [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
> (default-compile) on project hbase-server: Compilation failure:
> Compilation failure:
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
> cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
> cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
> cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ByteString not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
> incompatible types:
>
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
> cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
> cannot find symbol
>       [ERROR]   symbol:   method
>
> writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
>       [ERROR]   location: variable proto of type
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
> cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
>       [ERROR]   class file for
> org.apache.hadoop.thirdparty.protobuf.Descriptors not found
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
> incompatible types:
> org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
> cannot be converted to com.google.protobuf.MessageLite
>       [ERROR]
> /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
> invalid method reference
>       [ERROR]   non-static method get() cannot be referenced from a
> static context
>
>
> Ozone:
>
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
> -------------------------------------------------------------
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
>
> /grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
> incompatible types: com.google.protobuf.ServiceException cannot be
> converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
> 17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors
>
>
> There's another error where Ozone uses the Hadoop RPC framework which uses
> the hadoop.thirdparty protobuf.
>
> [ERROR] Failed to execute goal
> org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
> (default-testCompile) on project hadoop-hdds-container-service: Compilation
> failure
> [ERROR]
>
> /Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
> incompatible types: com.google.protobuf.BlockingService cannot be converted
> to org.apache.hadoop.thirdparty.protobuf.BlockingService
>
> BlockingService scmDatanodeService =
>     StorageContainerDatanodeProtocolService.
>         newReflectiveBlockingService(
>             new StorageContainerDatanodeProtocolServerSideTranslatorPB(
>                 server, Mockito.mock(ProtocolMessageMetrics.class)));
>
>
>
> Ratis probably breaks as well since it depends on the Hadoop RPC framework
> too.
>
> On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
> wrote:
>
> > hi Wei-Chiu,
> >
> > Can you elaborate on what failures you are facing related to relocated
> > protobuf classes.. ?
> >
> > IFAIK, if the issue with location of protobuf classes, still old jar
> > protobuf-2.5.0.jar will be available in classpath. So downstream
> depending
> > on 2.5.0 version of protobuf still be able to access them.
> >
> > -vinay
> >
> > On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> > wrote:
> >
> >> I'm sorry for coming to this late. I missed this message. It should have
> >> been a DISCUSS thread rather than NOTICE.
> >>
> >> Looks like this is inevitable. But we should make the downstream
> >> developers aware & make the update easier. As long as it is stated
> clearly
> >> how to update the code to support Hadoop 3.3, I am okay with that.
> >>
> >> Here's what I suggest:
> >> (1) label the jira incompatible (just updated the jira) and updated the
> >> release note to tell app developer how to update.
> >> (2) declare ProtobufHelper a public API HADOOP-17019
> >> <https://issues.apache.org/jira/browse/HADOOP-17019>
> >>
> >> Tez doesn't use the removed Token API, but there's code that breaks with
> >> the relocated protobuf class. The ProtobufHelper API will make this
> >> transition much easier.
> >>
> >> Other downstreamers that break with the relocated protobuf include:
> Ozone
> >> and HBase. but neither of them use the removed Token API.
> >>
> >>
> >> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> >> wrote:
> >>
> >>> Hi All,
> >>>
> >>>    This mail is to notify about the Removal of following public APIs
> from
> >>> Hadoop Common.
> >>>
> >>>      ClassName: org.apache.hadoop.security.token.Token
> >>>      APIs:
> >>>          public Token(TokenProto tokenPB);
> >>>          public TokenProto toTokenProto();
> >>>
> >>>    Reason: These APIs are having Generated protobuf classes in the
> >>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
> >>> release) these APIs are breaking the downstream builds, even though
> >>> downstreams dont use these APIs (just Loading Token class). Downstreams
> >>> are
> >>> still referencing having older version (2.5.0) of protobuf, hence build
> >>> is
> >>> being broken.
> >>>
> >>>     These APIs were added for the internal purpose(HADOOP-12563), to
> >>> support serializing tokens using protobuf in UGI Credentials.
> >>> Same purpose can be achieved using the Helper classes without
> introducing
> >>> protobuf classes in API signatures.
> >>>
> >>> Token.java is marked as Evolving, so I believe APIs can be changed
> >>> whenever
> >>> absolute necessary.
> >>>
> >>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
> >>> reported to solve downstream build failure.
> >>>
> >>> So since this API was added for internal purpose easy approach to solve
> >>> this is to remove APIs and use helper classes. Otherwise, as mentioned
> in
> >>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
> >>>
> >>> If anyone using these APIs outside hadoop project accidentally, please
> >>> reply to this mail immediately.
> >>>
> >>> If no objection by next week, will go ahead with removal of above said
> >>> APIs
> >>> in HADOOP-16621.
> >>>
> >>> -Vinay
> >>>
> >>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Wei-Chiu Chuang <we...@cloudera.com.INVALID>.
Most of the problems are downstream applications using Hadoop's private
APIs.

Tez:

17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
-------------------------------------------------------------
17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
incompatible types: com.google.protobuf.ByteString cannot be converted
to org.apache.hadoop.thirdparty.protobuf.ByteString
17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error


Tez keeps track of job tokens internally.
The change would look like this:

private void recordJobShuffleInfo(JobID jobId, String user,
    Token<JobTokenIdentifier> jobToken) throws IOException {
  if (stateDb != null) {
    TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
    /*TokenProto tokenProto = TokenProto.newBuilder()
        .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
        .setPassword(ByteString.copyFrom(jobToken.getPassword()))
        .setKind(jobToken.getKind().toString())
        .setService(jobToken.getService().toString())
        .build();*/
    JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
        .setUser(user).setJobToken(tokenProto).build();
    try {
      stateDb.put(bytes(jobId.toString()), proto.toByteArray());
    } catch (DBException e) {
      throw new IOException("Error storing " + jobId, e);
    }
  }
  addJobToken(jobId, user, jobToken);
}


HBase:

   1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833> (this
   is recently fixed in the master branch)
   2.

      [ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
(default-compile) on project hbase-server: Compilation failure:
Compilation failure:
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ByteString not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.Descriptors not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
invalid method reference
      [ERROR]   non-static method get() cannot be referenced from a
static context


Ozone:

17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
-------------------------------------------------------------
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors


There's another error where Ozone uses the Hadoop RPC framework which uses
the hadoop.thirdparty protobuf.

[ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
(default-testCompile) on project hadoop-hdds-container-service: Compilation
failure
[ERROR]
/Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
incompatible types: com.google.protobuf.BlockingService cannot be converted
to org.apache.hadoop.thirdparty.protobuf.BlockingService

BlockingService scmDatanodeService =
    StorageContainerDatanodeProtocolService.
        newReflectiveBlockingService(
            new StorageContainerDatanodeProtocolServerSideTranslatorPB(
                server, Mockito.mock(ProtocolMessageMetrics.class)));



Ratis probably breaks as well since it depends on the Hadoop RPC framework
too.

On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
wrote:

> hi Wei-Chiu,
>
> Can you elaborate on what failures you are facing related to relocated
> protobuf classes.. ?
>
> IFAIK, if the issue with location of protobuf classes, still old jar
> protobuf-2.5.0.jar will be available in classpath. So downstream depending
> on 2.5.0 version of protobuf still be able to access them.
>
> -vinay
>
> On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> wrote:
>
>> I'm sorry for coming to this late. I missed this message. It should have
>> been a DISCUSS thread rather than NOTICE.
>>
>> Looks like this is inevitable. But we should make the downstream
>> developers aware & make the update easier. As long as it is stated clearly
>> how to update the code to support Hadoop 3.3, I am okay with that.
>>
>> Here's what I suggest:
>> (1) label the jira incompatible (just updated the jira) and updated the
>> release note to tell app developer how to update.
>> (2) declare ProtobufHelper a public API HADOOP-17019
>> <https://issues.apache.org/jira/browse/HADOOP-17019>
>>
>> Tez doesn't use the removed Token API, but there's code that breaks with
>> the relocated protobuf class. The ProtobufHelper API will make this
>> transition much easier.
>>
>> Other downstreamers that break with the relocated protobuf include: Ozone
>> and HBase. but neither of them use the removed Token API.
>>
>>
>> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
>> wrote:
>>
>>> Hi All,
>>>
>>>    This mail is to notify about the Removal of following public APIs from
>>> Hadoop Common.
>>>
>>>      ClassName: org.apache.hadoop.security.token.Token
>>>      APIs:
>>>          public Token(TokenProto tokenPB);
>>>          public TokenProto toTokenProto();
>>>
>>>    Reason: These APIs are having Generated protobuf classes in the
>>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>>> release) these APIs are breaking the downstream builds, even though
>>> downstreams dont use these APIs (just Loading Token class). Downstreams
>>> are
>>> still referencing having older version (2.5.0) of protobuf, hence build
>>> is
>>> being broken.
>>>
>>>     These APIs were added for the internal purpose(HADOOP-12563), to
>>> support serializing tokens using protobuf in UGI Credentials.
>>> Same purpose can be achieved using the Helper classes without introducing
>>> protobuf classes in API signatures.
>>>
>>> Token.java is marked as Evolving, so I believe APIs can be changed
>>> whenever
>>> absolute necessary.
>>>
>>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>>> reported to solve downstream build failure.
>>>
>>> So since this API was added for internal purpose easy approach to solve
>>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>>
>>> If anyone using these APIs outside hadoop project accidentally, please
>>> reply to this mail immediately.
>>>
>>> If no objection by next week, will go ahead with removal of above said
>>> APIs
>>> in HADOOP-16621.
>>>
>>> -Vinay
>>>
>>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Wei-Chiu Chuang <we...@cloudera.com.INVALID>.
Most of the problems are downstream applications using Hadoop's private
APIs.

Tez:

17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
-------------------------------------------------------------
17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
incompatible types: com.google.protobuf.ByteString cannot be converted
to org.apache.hadoop.thirdparty.protobuf.ByteString
17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error


Tez keeps track of job tokens internally.
The change would look like this:

private void recordJobShuffleInfo(JobID jobId, String user,
    Token<JobTokenIdentifier> jobToken) throws IOException {
  if (stateDb != null) {
    TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
    /*TokenProto tokenProto = TokenProto.newBuilder()
        .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
        .setPassword(ByteString.copyFrom(jobToken.getPassword()))
        .setKind(jobToken.getKind().toString())
        .setService(jobToken.getService().toString())
        .build();*/
    JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
        .setUser(user).setJobToken(tokenProto).build();
    try {
      stateDb.put(bytes(jobId.toString()), proto.toByteArray());
    } catch (DBException e) {
      throw new IOException("Error storing " + jobId, e);
    }
  }
  addJobToken(jobId, user, jobToken);
}


HBase:

   1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833> (this
   is recently fixed in the master branch)
   2.

      [ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
(default-compile) on project hbase-server: Compilation failure:
Compilation failure:
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ByteString not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.Descriptors not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
invalid method reference
      [ERROR]   non-static method get() cannot be referenced from a
static context


Ozone:

17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
-------------------------------------------------------------
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors


There's another error where Ozone uses the Hadoop RPC framework which uses
the hadoop.thirdparty protobuf.

[ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
(default-testCompile) on project hadoop-hdds-container-service: Compilation
failure
[ERROR]
/Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
incompatible types: com.google.protobuf.BlockingService cannot be converted
to org.apache.hadoop.thirdparty.protobuf.BlockingService

BlockingService scmDatanodeService =
    StorageContainerDatanodeProtocolService.
        newReflectiveBlockingService(
            new StorageContainerDatanodeProtocolServerSideTranslatorPB(
                server, Mockito.mock(ProtocolMessageMetrics.class)));



Ratis probably breaks as well since it depends on the Hadoop RPC framework
too.

On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
wrote:

> hi Wei-Chiu,
>
> Can you elaborate on what failures you are facing related to relocated
> protobuf classes.. ?
>
> IFAIK, if the issue with location of protobuf classes, still old jar
> protobuf-2.5.0.jar will be available in classpath. So downstream depending
> on 2.5.0 version of protobuf still be able to access them.
>
> -vinay
>
> On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> wrote:
>
>> I'm sorry for coming to this late. I missed this message. It should have
>> been a DISCUSS thread rather than NOTICE.
>>
>> Looks like this is inevitable. But we should make the downstream
>> developers aware & make the update easier. As long as it is stated clearly
>> how to update the code to support Hadoop 3.3, I am okay with that.
>>
>> Here's what I suggest:
>> (1) label the jira incompatible (just updated the jira) and updated the
>> release note to tell app developer how to update.
>> (2) declare ProtobufHelper a public API HADOOP-17019
>> <https://issues.apache.org/jira/browse/HADOOP-17019>
>>
>> Tez doesn't use the removed Token API, but there's code that breaks with
>> the relocated protobuf class. The ProtobufHelper API will make this
>> transition much easier.
>>
>> Other downstreamers that break with the relocated protobuf include: Ozone
>> and HBase. but neither of them use the removed Token API.
>>
>>
>> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
>> wrote:
>>
>>> Hi All,
>>>
>>>    This mail is to notify about the Removal of following public APIs from
>>> Hadoop Common.
>>>
>>>      ClassName: org.apache.hadoop.security.token.Token
>>>      APIs:
>>>          public Token(TokenProto tokenPB);
>>>          public TokenProto toTokenProto();
>>>
>>>    Reason: These APIs are having Generated protobuf classes in the
>>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>>> release) these APIs are breaking the downstream builds, even though
>>> downstreams dont use these APIs (just Loading Token class). Downstreams
>>> are
>>> still referencing having older version (2.5.0) of protobuf, hence build
>>> is
>>> being broken.
>>>
>>>     These APIs were added for the internal purpose(HADOOP-12563), to
>>> support serializing tokens using protobuf in UGI Credentials.
>>> Same purpose can be achieved using the Helper classes without introducing
>>> protobuf classes in API signatures.
>>>
>>> Token.java is marked as Evolving, so I believe APIs can be changed
>>> whenever
>>> absolute necessary.
>>>
>>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>>> reported to solve downstream build failure.
>>>
>>> So since this API was added for internal purpose easy approach to solve
>>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>>
>>> If anyone using these APIs outside hadoop project accidentally, please
>>> reply to this mail immediately.
>>>
>>> If no objection by next week, will go ahead with removal of above said
>>> APIs
>>> in HADOOP-16621.
>>>
>>> -Vinay
>>>
>>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Wei-Chiu Chuang <we...@cloudera.com.INVALID>.
Most of the problems are downstream applications using Hadoop's private
APIs.

Tez:

17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
-------------------------------------------------------------
17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
incompatible types: com.google.protobuf.ByteString cannot be converted
to org.apache.hadoop.thirdparty.protobuf.ByteString
17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error


Tez keeps track of job tokens internally.
The change would look like this:

private void recordJobShuffleInfo(JobID jobId, String user,
    Token<JobTokenIdentifier> jobToken) throws IOException {
  if (stateDb != null) {
    TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
    /*TokenProto tokenProto = TokenProto.newBuilder()
        .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
        .setPassword(ByteString.copyFrom(jobToken.getPassword()))
        .setKind(jobToken.getKind().toString())
        .setService(jobToken.getService().toString())
        .build();*/
    JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
        .setUser(user).setJobToken(tokenProto).build();
    try {
      stateDb.put(bytes(jobId.toString()), proto.toByteArray());
    } catch (DBException e) {
      throw new IOException("Error storing " + jobId, e);
    }
  }
  addJobToken(jobId, user, jobToken);
}


HBase:

   1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833> (this
   is recently fixed in the master branch)
   2.

      [ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
(default-compile) on project hbase-server: Compilation failure:
Compilation failure:
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ByteString not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.Descriptors not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
invalid method reference
      [ERROR]   non-static method get() cannot be referenced from a
static context


Ozone:

17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
-------------------------------------------------------------
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors


There's another error where Ozone uses the Hadoop RPC framework which uses
the hadoop.thirdparty protobuf.

[ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
(default-testCompile) on project hadoop-hdds-container-service: Compilation
failure
[ERROR]
/Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
incompatible types: com.google.protobuf.BlockingService cannot be converted
to org.apache.hadoop.thirdparty.protobuf.BlockingService

BlockingService scmDatanodeService =
    StorageContainerDatanodeProtocolService.
        newReflectiveBlockingService(
            new StorageContainerDatanodeProtocolServerSideTranslatorPB(
                server, Mockito.mock(ProtocolMessageMetrics.class)));



Ratis probably breaks as well since it depends on the Hadoop RPC framework
too.

On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
wrote:

> hi Wei-Chiu,
>
> Can you elaborate on what failures you are facing related to relocated
> protobuf classes.. ?
>
> IFAIK, if the issue with location of protobuf classes, still old jar
> protobuf-2.5.0.jar will be available in classpath. So downstream depending
> on 2.5.0 version of protobuf still be able to access them.
>
> -vinay
>
> On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> wrote:
>
>> I'm sorry for coming to this late. I missed this message. It should have
>> been a DISCUSS thread rather than NOTICE.
>>
>> Looks like this is inevitable. But we should make the downstream
>> developers aware & make the update easier. As long as it is stated clearly
>> how to update the code to support Hadoop 3.3, I am okay with that.
>>
>> Here's what I suggest:
>> (1) label the jira incompatible (just updated the jira) and updated the
>> release note to tell app developer how to update.
>> (2) declare ProtobufHelper a public API HADOOP-17019
>> <https://issues.apache.org/jira/browse/HADOOP-17019>
>>
>> Tez doesn't use the removed Token API, but there's code that breaks with
>> the relocated protobuf class. The ProtobufHelper API will make this
>> transition much easier.
>>
>> Other downstreamers that break with the relocated protobuf include: Ozone
>> and HBase. but neither of them use the removed Token API.
>>
>>
>> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
>> wrote:
>>
>>> Hi All,
>>>
>>>    This mail is to notify about the Removal of following public APIs from
>>> Hadoop Common.
>>>
>>>      ClassName: org.apache.hadoop.security.token.Token
>>>      APIs:
>>>          public Token(TokenProto tokenPB);
>>>          public TokenProto toTokenProto();
>>>
>>>    Reason: These APIs are having Generated protobuf classes in the
>>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>>> release) these APIs are breaking the downstream builds, even though
>>> downstreams dont use these APIs (just Loading Token class). Downstreams
>>> are
>>> still referencing having older version (2.5.0) of protobuf, hence build
>>> is
>>> being broken.
>>>
>>>     These APIs were added for the internal purpose(HADOOP-12563), to
>>> support serializing tokens using protobuf in UGI Credentials.
>>> Same purpose can be achieved using the Helper classes without introducing
>>> protobuf classes in API signatures.
>>>
>>> Token.java is marked as Evolving, so I believe APIs can be changed
>>> whenever
>>> absolute necessary.
>>>
>>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>>> reported to solve downstream build failure.
>>>
>>> So since this API was added for internal purpose easy approach to solve
>>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>>
>>> If anyone using these APIs outside hadoop project accidentally, please
>>> reply to this mail immediately.
>>>
>>> If no objection by next week, will go ahead with removal of above said
>>> APIs
>>> in HADOOP-16621.
>>>
>>> -Vinay
>>>
>>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Wei-Chiu Chuang <we...@cloudera.com.INVALID>.
Most of the problems are downstream applications using Hadoop's private
APIs.

Tez:

17:08:38 2020/04/16 00:08:38 INFO    : [ERROR] COMPILATION ERROR :
17:08:38 2020/04/16 00:08:38 INFO    : [INFO]
-------------------------------------------------------------
17:08:38 2020/04/16 00:08:38 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/tez/tez-plugins/tez-aux-services/src/main/java/org/apache/tez/auxservices/ShuffleHandler.java:[757,45]
incompatible types: com.google.protobuf.ByteString cannot be converted
to org.apache.hadoop.thirdparty.protobuf.ByteString
17:08:38 2020/04/16 00:08:38 INFO    : [INFO] 1 error


Tez keeps track of job tokens internally.
The change would look like this:

private void recordJobShuffleInfo(JobID jobId, String user,
    Token<JobTokenIdentifier> jobToken) throws IOException {
  if (stateDb != null) {
    TokenProto tokenProto = ProtobufHelper.protoFromToken(jobToken);
    /*TokenProto tokenProto = TokenProto.newBuilder()
        .setIdentifier(ByteString.copyFrom(jobToken.getIdentifier()))
        .setPassword(ByteString.copyFrom(jobToken.getPassword()))
        .setKind(jobToken.getKind().toString())
        .setService(jobToken.getService().toString())
        .build();*/
    JobShuffleInfoProto proto = JobShuffleInfoProto.newBuilder()
        .setUser(user).setJobToken(tokenProto).build();
    try {
      stateDb.put(bytes(jobId.toString()), proto.toByteArray());
    } catch (DBException e) {
      throw new IOException("Error storing " + jobId, e);
    }
  }
  addJobToken(jobId, user, jobToken);
}


HBase:

   1. HBASE-23833 <https://issues.apache.org/jira/browse/HBASE-23833> (this
   is recently fixed in the master branch)
   2.

      [ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.8.1:compile
(default-compile) on project hbase-server: Compilation failure:
Compilation failure:
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[361,44]
cannot access org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.MessageOrBuilder not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[362,14]
cannot access org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.GeneratedMessageV3 not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[366,16]
cannot access org.apache.hadoop.thirdparty.protobuf.ByteString
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ByteString not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[375,12]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputSaslHelper.java:[702,81]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferEncryptorMessageProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[314,66]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[330,81]
cannot access org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.ProtocolMessageEnum not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[380,10]
cannot find symbol
      [ERROR]   symbol:   method
writeDelimitedTo(org.apache.hbase.thirdparty.io.netty.buffer.ByteBufOutputStream)
      [ERROR]   location: variable proto of type
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpWriteBlockProto
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutputHelper.java:[422,77]
cannot access org.apache.hadoop.thirdparty.protobuf.Descriptors
      [ERROR]   class file for
org.apache.hadoop.thirdparty.protobuf.Descriptors not found
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/io/asyncfs/FanOutOneBlockAsyncDFSOutput.java:[323,64]
incompatible types:
org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.PipelineAckProto
cannot be converted to com.google.protobuf.MessageLite
      [ERROR] /Users/weichiu/sandbox/hbase/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/SyncReplicationWALProvider.java:[209,68]
invalid method reference
      [ERROR]   non-static method get() cannot be referenced from a
static context


Ozone:

17:01:19 2020/04/16 00:01:19 INFO    : [ERROR] COMPILATION ERROR :
17:01:19 2020/04/16 00:01:19 INFO    : [INFO]
-------------------------------------------------------------
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java:[110,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [ERROR]
/grid/0/jenkins/workspace/workspace/CDH-CANARY-parallel-centos7/SOURCES/ozone/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java:[116,47]
incompatible types: com.google.protobuf.ServiceException cannot be
converted to org.apache.hadoop.thirdparty.protobuf.ServiceException
17:01:19 2020/04/16 00:01:19 INFO    : [INFO] 2 errors


There's another error where Ozone uses the Hadoop RPC framework which uses
the hadoop.thirdparty protobuf.

[ERROR] Failed to execute goal
org.apache.maven.plugins:maven-compiler-plugin:3.1:testCompile
(default-testCompile) on project hadoop-hdds-container-service: Compilation
failure
[ERROR]
/Users/weichiu/sandbox/ozone/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/SCMTestUtils.java:[103,41]
incompatible types: com.google.protobuf.BlockingService cannot be converted
to org.apache.hadoop.thirdparty.protobuf.BlockingService

BlockingService scmDatanodeService =
    StorageContainerDatanodeProtocolService.
        newReflectiveBlockingService(
            new StorageContainerDatanodeProtocolServerSideTranslatorPB(
                server, Mockito.mock(ProtocolMessageMetrics.class)));



Ratis probably breaks as well since it depends on the Hadoop RPC framework
too.

On Tue, Apr 28, 2020 at 10:58 PM Vinayakumar B <vi...@apache.org>
wrote:

> hi Wei-Chiu,
>
> Can you elaborate on what failures you are facing related to relocated
> protobuf classes.. ?
>
> IFAIK, if the issue with location of protobuf classes, still old jar
> protobuf-2.5.0.jar will be available in classpath. So downstream depending
> on 2.5.0 version of protobuf still be able to access them.
>
> -vinay
>
> On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com>
> wrote:
>
>> I'm sorry for coming to this late. I missed this message. It should have
>> been a DISCUSS thread rather than NOTICE.
>>
>> Looks like this is inevitable. But we should make the downstream
>> developers aware & make the update easier. As long as it is stated clearly
>> how to update the code to support Hadoop 3.3, I am okay with that.
>>
>> Here's what I suggest:
>> (1) label the jira incompatible (just updated the jira) and updated the
>> release note to tell app developer how to update.
>> (2) declare ProtobufHelper a public API HADOOP-17019
>> <https://issues.apache.org/jira/browse/HADOOP-17019>
>>
>> Tez doesn't use the removed Token API, but there's code that breaks with
>> the relocated protobuf class. The ProtobufHelper API will make this
>> transition much easier.
>>
>> Other downstreamers that break with the relocated protobuf include: Ozone
>> and HBase. but neither of them use the removed Token API.
>>
>>
>> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
>> wrote:
>>
>>> Hi All,
>>>
>>>    This mail is to notify about the Removal of following public APIs from
>>> Hadoop Common.
>>>
>>>      ClassName: org.apache.hadoop.security.token.Token
>>>      APIs:
>>>          public Token(TokenProto tokenPB);
>>>          public TokenProto toTokenProto();
>>>
>>>    Reason: These APIs are having Generated protobuf classes in the
>>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>>> release) these APIs are breaking the downstream builds, even though
>>> downstreams dont use these APIs (just Loading Token class). Downstreams
>>> are
>>> still referencing having older version (2.5.0) of protobuf, hence build
>>> is
>>> being broken.
>>>
>>>     These APIs were added for the internal purpose(HADOOP-12563), to
>>> support serializing tokens using protobuf in UGI Credentials.
>>> Same purpose can be achieved using the Helper classes without introducing
>>> protobuf classes in API signatures.
>>>
>>> Token.java is marked as Evolving, so I believe APIs can be changed
>>> whenever
>>> absolute necessary.
>>>
>>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>>> reported to solve downstream build failure.
>>>
>>> So since this API was added for internal purpose easy approach to solve
>>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>>
>>> If anyone using these APIs outside hadoop project accidentally, please
>>> reply to this mail immediately.
>>>
>>> If no objection by next week, will go ahead with removal of above said
>>> APIs
>>> in HADOOP-16621.
>>>
>>> -Vinay
>>>
>>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
hi Wei-Chiu,

Can you elaborate on what failures you are facing related to relocated
protobuf classes.. ?

IFAIK, if the issue with location of protobuf classes, still old jar
protobuf-2.5.0.jar will be available in classpath. So downstream depending
on 2.5.0 version of protobuf still be able to access them.

-vinay

On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com> wrote:

> I'm sorry for coming to this late. I missed this message. It should have
> been a DISCUSS thread rather than NOTICE.
>
> Looks like this is inevitable. But we should make the downstream
> developers aware & make the update easier. As long as it is stated clearly
> how to update the code to support Hadoop 3.3, I am okay with that.
>
> Here's what I suggest:
> (1) label the jira incompatible (just updated the jira) and updated the
> release note to tell app developer how to update.
> (2) declare ProtobufHelper a public API HADOOP-17019
> <https://issues.apache.org/jira/browse/HADOOP-17019>
>
> Tez doesn't use the removed Token API, but there's code that breaks with
> the relocated protobuf class. The ProtobufHelper API will make this
> transition much easier.
>
> Other downstreamers that break with the relocated protobuf include: Ozone
> and HBase. but neither of them use the removed Token API.
>
>
> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> wrote:
>
>> Hi All,
>>
>>    This mail is to notify about the Removal of following public APIs from
>> Hadoop Common.
>>
>>      ClassName: org.apache.hadoop.security.token.Token
>>      APIs:
>>          public Token(TokenProto tokenPB);
>>          public TokenProto toTokenProto();
>>
>>    Reason: These APIs are having Generated protobuf classes in the
>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>> release) these APIs are breaking the downstream builds, even though
>> downstreams dont use these APIs (just Loading Token class). Downstreams
>> are
>> still referencing having older version (2.5.0) of protobuf, hence build is
>> being broken.
>>
>>     These APIs were added for the internal purpose(HADOOP-12563), to
>> support serializing tokens using protobuf in UGI Credentials.
>> Same purpose can be achieved using the Helper classes without introducing
>> protobuf classes in API signatures.
>>
>> Token.java is marked as Evolving, so I believe APIs can be changed
>> whenever
>> absolute necessary.
>>
>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>> reported to solve downstream build failure.
>>
>> So since this API was added for internal purpose easy approach to solve
>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>
>> If anyone using these APIs outside hadoop project accidentally, please
>> reply to this mail immediately.
>>
>> If no objection by next week, will go ahead with removal of above said
>> APIs
>> in HADOOP-16621.
>>
>> -Vinay
>>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
hi Wei-Chiu,

Can you elaborate on what failures you are facing related to relocated
protobuf classes.. ?

IFAIK, if the issue with location of protobuf classes, still old jar
protobuf-2.5.0.jar will be available in classpath. So downstream depending
on 2.5.0 version of protobuf still be able to access them.

-vinay

On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com> wrote:

> I'm sorry for coming to this late. I missed this message. It should have
> been a DISCUSS thread rather than NOTICE.
>
> Looks like this is inevitable. But we should make the downstream
> developers aware & make the update easier. As long as it is stated clearly
> how to update the code to support Hadoop 3.3, I am okay with that.
>
> Here's what I suggest:
> (1) label the jira incompatible (just updated the jira) and updated the
> release note to tell app developer how to update.
> (2) declare ProtobufHelper a public API HADOOP-17019
> <https://issues.apache.org/jira/browse/HADOOP-17019>
>
> Tez doesn't use the removed Token API, but there's code that breaks with
> the relocated protobuf class. The ProtobufHelper API will make this
> transition much easier.
>
> Other downstreamers that break with the relocated protobuf include: Ozone
> and HBase. but neither of them use the removed Token API.
>
>
> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> wrote:
>
>> Hi All,
>>
>>    This mail is to notify about the Removal of following public APIs from
>> Hadoop Common.
>>
>>      ClassName: org.apache.hadoop.security.token.Token
>>      APIs:
>>          public Token(TokenProto tokenPB);
>>          public TokenProto toTokenProto();
>>
>>    Reason: These APIs are having Generated protobuf classes in the
>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>> release) these APIs are breaking the downstream builds, even though
>> downstreams dont use these APIs (just Loading Token class). Downstreams
>> are
>> still referencing having older version (2.5.0) of protobuf, hence build is
>> being broken.
>>
>>     These APIs were added for the internal purpose(HADOOP-12563), to
>> support serializing tokens using protobuf in UGI Credentials.
>> Same purpose can be achieved using the Helper classes without introducing
>> protobuf classes in API signatures.
>>
>> Token.java is marked as Evolving, so I believe APIs can be changed
>> whenever
>> absolute necessary.
>>
>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>> reported to solve downstream build failure.
>>
>> So since this API was added for internal purpose easy approach to solve
>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>
>> If anyone using these APIs outside hadoop project accidentally, please
>> reply to this mail immediately.
>>
>> If no objection by next week, will go ahead with removal of above said
>> APIs
>> in HADOOP-16621.
>>
>> -Vinay
>>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
hi Wei-Chiu,

Can you elaborate on what failures you are facing related to relocated
protobuf classes.. ?

IFAIK, if the issue with location of protobuf classes, still old jar
protobuf-2.5.0.jar will be available in classpath. So downstream depending
on 2.5.0 version of protobuf still be able to access them.

-vinay

On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com> wrote:

> I'm sorry for coming to this late. I missed this message. It should have
> been a DISCUSS thread rather than NOTICE.
>
> Looks like this is inevitable. But we should make the downstream
> developers aware & make the update easier. As long as it is stated clearly
> how to update the code to support Hadoop 3.3, I am okay with that.
>
> Here's what I suggest:
> (1) label the jira incompatible (just updated the jira) and updated the
> release note to tell app developer how to update.
> (2) declare ProtobufHelper a public API HADOOP-17019
> <https://issues.apache.org/jira/browse/HADOOP-17019>
>
> Tez doesn't use the removed Token API, but there's code that breaks with
> the relocated protobuf class. The ProtobufHelper API will make this
> transition much easier.
>
> Other downstreamers that break with the relocated protobuf include: Ozone
> and HBase. but neither of them use the removed Token API.
>
>
> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> wrote:
>
>> Hi All,
>>
>>    This mail is to notify about the Removal of following public APIs from
>> Hadoop Common.
>>
>>      ClassName: org.apache.hadoop.security.token.Token
>>      APIs:
>>          public Token(TokenProto tokenPB);
>>          public TokenProto toTokenProto();
>>
>>    Reason: These APIs are having Generated protobuf classes in the
>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>> release) these APIs are breaking the downstream builds, even though
>> downstreams dont use these APIs (just Loading Token class). Downstreams
>> are
>> still referencing having older version (2.5.0) of protobuf, hence build is
>> being broken.
>>
>>     These APIs were added for the internal purpose(HADOOP-12563), to
>> support serializing tokens using protobuf in UGI Credentials.
>> Same purpose can be achieved using the Helper classes without introducing
>> protobuf classes in API signatures.
>>
>> Token.java is marked as Evolving, so I believe APIs can be changed
>> whenever
>> absolute necessary.
>>
>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>> reported to solve downstream build failure.
>>
>> So since this API was added for internal purpose easy approach to solve
>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>
>> If anyone using these APIs outside hadoop project accidentally, please
>> reply to this mail immediately.
>>
>> If no objection by next week, will go ahead with removal of above said
>> APIs
>> in HADOOP-16621.
>>
>> -Vinay
>>
>

Re: [NOTICE] Removal of protobuf classes from Hadoop Token's public APIs' signature

Posted by Vinayakumar B <vi...@apache.org>.
hi Wei-Chiu,

Can you elaborate on what failures you are facing related to relocated
protobuf classes.. ?

IFAIK, if the issue with location of protobuf classes, still old jar
protobuf-2.5.0.jar will be available in classpath. So downstream depending
on 2.5.0 version of protobuf still be able to access them.

-vinay

On Wed, 29 Apr 2020, 11:17 am Wei-Chiu Chuang, <we...@cloudera.com> wrote:

> I'm sorry for coming to this late. I missed this message. It should have
> been a DISCUSS thread rather than NOTICE.
>
> Looks like this is inevitable. But we should make the downstream
> developers aware & make the update easier. As long as it is stated clearly
> how to update the code to support Hadoop 3.3, I am okay with that.
>
> Here's what I suggest:
> (1) label the jira incompatible (just updated the jira) and updated the
> release note to tell app developer how to update.
> (2) declare ProtobufHelper a public API HADOOP-17019
> <https://issues.apache.org/jira/browse/HADOOP-17019>
>
> Tez doesn't use the removed Token API, but there's code that breaks with
> the relocated protobuf class. The ProtobufHelper API will make this
> transition much easier.
>
> Other downstreamers that break with the relocated protobuf include: Ozone
> and HBase. but neither of them use the removed Token API.
>
>
> On Wed, Jan 8, 2020 at 4:40 AM Vinayakumar B <vi...@apache.org>
> wrote:
>
>> Hi All,
>>
>>    This mail is to notify about the Removal of following public APIs from
>> Hadoop Common.
>>
>>      ClassName: org.apache.hadoop.security.token.Token
>>      APIs:
>>          public Token(TokenProto tokenPB);
>>          public TokenProto toTokenProto();
>>
>>    Reason: These APIs are having Generated protobuf classes in the
>> signature. Right now due to protobuf upgrade in trunk (soon to be 3.3.0
>> release) these APIs are breaking the downstream builds, even though
>> downstreams dont use these APIs (just Loading Token class). Downstreams
>> are
>> still referencing having older version (2.5.0) of protobuf, hence build is
>> being broken.
>>
>>     These APIs were added for the internal purpose(HADOOP-12563), to
>> support serializing tokens using protobuf in UGI Credentials.
>> Same purpose can be achieved using the Helper classes without introducing
>> protobuf classes in API signatures.
>>
>> Token.java is marked as Evolving, so I believe APIs can be changed
>> whenever
>> absolute necessary.
>>
>>     Jira https://issues.apache.org/jira/browse/HADOOP-16621 has been
>> reported to solve downstream build failure.
>>
>> So since this API was added for internal purpose easy approach to solve
>> this is to remove APIs and use helper classes. Otherwise, as mentioned in
>> HADOOP-16621, workaround will add unnecessary codes to be maintained.
>>
>> If anyone using these APIs outside hadoop project accidentally, please
>> reply to this mail immediately.
>>
>> If no objection by next week, will go ahead with removal of above said
>> APIs
>> in HADOOP-16621.
>>
>> -Vinay
>>
>