Glad to hear that it has worked. Yeah, I guess right now, if there is any
dependency conflicts, your solution would be the way to resolve it.

Could you open a JIRA to enhance the hello-samza tutorial docs to address
this issue?

Thanks!

-Yi

On Wed, Jan 6, 2016 at 12:47 PM, Stanislav Los <stanis...@magnetic.com>
wrote:

> So, for now, I made it work by replacing any generic hadoop-* jar in
> hello-samza distribution with related jar from CDH:
>
> [stanis...@hcm001.us-east-1.mgnt.cc samza]$ ls -l deploy/samza-cdh/lib/ |
> grep hadoop
>
> -rwxr-xr-x 1 stanislav stanislav    21533 Jan  6 15:19
> hadoop-annotations-2.6.0-cdh5.4.0.jar
>
> -rwxr-xr-x 1 stanislav stanislav    73113 Jan  6 15:22
> hadoop-auth-2.6.0-cdh5.4.0.jar
>
> -rwxr-xr-x 1 stanislav stanislav  3339429 Jan  6 15:22
> hadoop-common-2.6.0-cdh5.4.0.jar
>
> -rw-r--r-- 1 stanislav stanislav  7970062 Jan  6 15:22
> hadoop-hdfs-2.6.0-cdh5.4.0.jar
>
> -rw-r--r-- 1 stanislav stanislav  1875075 Jan  6 15:23
> hadoop-yarn-api-2.6.0-cdh5.4.0.jar
>
> -rw-r--r-- 1 stanislav stanislav   119623 Jan  6 15:23
> hadoop-yarn-client-2.6.0-cdh5.4.0.jar
>
> -rw-r--r-- 1 stanislav stanislav  1540461 Jan  6 15:23
> hadoop-yarn-common-2.6.0-cdh5.4.0.jar
>
> I guess, I can update hello-samza pom.xml by explicitly excluding generic
> hadoop dependencies and replacing them with related stuff from CDH repo to
> achieve the same on a build level.
>
> On Wed, Jan 6, 2016 at 3:18 PM, Yi Pan <nickpa...@gmail.com> wrote:
>
> > Token expiration is still a problem w/o Kerberos in YARN 2.6.0 client.
> But
> > the exception stack trace from that bug is also different from yours. I
> am
> > suspecting there is some incompatible dependency libraries between CDH5.4
> > distro and Samza that caused your serialization problems here.
> >
> > On Wed, Jan 6, 2016 at 11:50 AM, Stanislav Los <stanis...@magnetic.com>
> > wrote:
> >
> > > Hi Yi,
> > >
> > > Our cluster doesn't have Kerberos security in place, so I don't know
> how
> > it
> > > could be related.
> > > CDH5.4 runs on YARN 2.6.0.
> > > I compiled Samza with YARN 2.6.0 and YARN 2.6.1.
> > > In both cases I get the same error.
> > > Should token expiration be a problem for the cluster without Kerberos?
> > >
> > > Regards.
> > >
> > > On Wed, Jan 6, 2016 at 2:39 PM, Yi Pan <nickpa...@gmail.com> wrote:
> > >
> > > > Hi, Stanislav,
> > > >
> > > > Could it be related to SAMZA-727? Your exception stack trace is
> > different
> > > > though. Besides that, Samza 0.10 requires minimum YARN 2.6.1 now, due
> > to
> > > a
> > > > token expiration issue in YARN 2.6.0 client lib.
> > > >
> > > > -Yi
> > > >
> > > > On Wed, Jan 6, 2016 at 7:52 AM, Stanislav Los <
> stanis...@magnetic.com>
> > > > wrote:
> > > >
> > > > > Hi,
> > > > >
> > > > > Any ideas why Samza can't start a container while running
> hello-samza
> > > on
> > > > > CDH5.4?
> > > > >
> > > > > It runs fine on local grid. But when running on cluster I can see
> it
> > > > talks
> > > > > fine to YARN's  ResourceManager and Kafka, get's resources, but
> fails
> > > to
> > > > > start container. At first, I thought it's because CDH5.4 uses YARN
> > > 2.6.0,
> > > > > while Samza uses 2.6.1, so I recompiled Samza with YARN 2.6.0 and
> > > > > re-assambled hello-samza. No luck so far. Also, I thought it's
> > because
> > > of
> > > > > different protobuf version, but I checked, it's the same in CDH and
> > > > Samza.
> > > > >
> > > > > Followed
> > > > >
> > http://samza.apache.org/learn/tutorials/0.10/deploy-samza-to-CDH.html
> > > to
> > > > > deploy.
> > > > >
> > > > > See full logs:
> > > > > Samza Deploy Script output http://pastebin.com/0K8HVMzt
> > > > > YARN logs http://pastebin.com/UvNp6240
> > > > >
> > > > > An error is:
> > > > >
> > > > > java version "1.7.0_80"
> > > > >
> > > > > Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
> > > > >
> > > > > Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
> > > > >
> > > > > com.google.protobuf.InvalidProtocolBufferException: Protocol
> message
> > > > > contained an invalid tag (zero).
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> com.google.protobuf.InvalidProtocolBufferException.invalidTag(InvalidProtocolBufferException.java:89)
> > > > >
> > > > > at
> > > >
> com.google.protobuf.CodedInputStream.readTag(CodedInputStream.java:108)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$AMRMTokenIdentifierProto.<init>(YarnSecurityTokenProtos.java:1078)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$AMRMTokenIdentifierProto.<init>(YarnSecurityTokenProtos.java:1042)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$AMRMTokenIdentifierProto$1.parsePartialFrom(YarnSecurityTokenProtos.java:1138)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$AMRMTokenIdentifierProto$1.parsePartialFrom(YarnSecurityTokenProtos.java:1133)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> com.google.protobuf.AbstractParser.parsePartialFrom(AbstractParser.java:200)
> > > > >
> > > > > at
> > > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:217)
> > > > >
> > > > > at
> > > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:223)
> > > > >
> > > > > at
> > com.google.protobuf.AbstractParser.parseFrom(AbstractParser.java:49)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$AMRMTokenIdentifierProto.parseFrom(YarnSecurityTokenProtos.java:1307)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.yarn.security.AMRMTokenIdentifier.readFields(AMRMTokenIdentifier.java:81)
> > > > >
> > > > > at
> > > >
> org.apache.hadoop.security.token.Token.decodeIdentifier(Token.java:142)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerUtil.startContainer(ContainerUtil.java:181)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerUtil.runContainer(ContainerUtil.java:119)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerAllocator.run(ContainerAllocator.java:65)
> > > > >
> > > > > at java.lang.Thread.run(Thread.java:745)
> > > > >
> > > > > Exception in thread "Container Allocator Thread"
> > > > > org.apache.samza.SamzaException: IO Exception when writing
> > credentials
> > > to
> > > > > output buffer
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerUtil.startContainer(ContainerUtil.java:190)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerUtil.runContainer(ContainerUtil.java:119)
> > > > >
> > > > > at
> > > > >
> > > > >
> > > >
> > >
> >
> org.apache.samza.job.yarn.ContainerAllocator.run(ContainerAllocator.java:65)
> > > > >
> > > > > at java.lang.Thread.run(Thread.java:745)
> > > > >
> > > > >
> > > > > Regards,
> > > > > Stan
> > > > >
> > > >
> > >
> >
>

Reply via email to