You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Dean Chen <de...@ocirs.com> on 2015/04/06 23:38:14 UTC

Re: Experience using binary packages on various Hadoop distros

This would be great for those of us running on HDP. At eBay we recently ran
in to few problems using the generic Hadoop lib. Two off of the top of my
head:

* Needed to included our custom Hadoop client due to custom keberos
integration
* Minor difference in HDFS protocol causing the following error. Was fixed
by removing the HDFS jar from the Spark assembly.

Exception in thread "Driver" java.lang.IllegalStateException
        at
org.spark-project.guava.common.base.Preconditions.checkState(Preconditions.java:133)
        at
org.apache.hadoop.hdfs.protocolPB.PBHelper.convert(PBHelper.java:673)
        at
org.apache.hadoop.hdfs.protocolPB.PBHelper.convertLocatedBlock(PBHelper.java:1100)
        at
org.apache.hadoop.hdfs.protocolPB.PBHelper.convert(PBHelper.java:1118)
        at
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:221)
        at
org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1142)
        at
org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1132)
        at
org.apache.hadoop.hdfs.DFSClient.getBlockLocations(DFSClient.java:1182)
        at
org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:218)
        at
org.apache.hadoop.hdfs.DistributedFileSystem$1.doCall(DistributedFileSystem.java:214)
        at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:214)
        at
org.apache.hadoop.hdfs.DistributedFileSystem.getFileBlockLocations(DistributedFileSystem.java:206)
        at
org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:332)
        at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:201)
        at
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
        at
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:203)
        at scala.Option.getOrElse(Option.scala:120)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:203)
        at org.apache.spark.rdd.MappedRDD.getPartitions(MappedRDD.scala:28)
        at
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
        at
org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:203)
        at scala.Option.getOrElse(Option.scala:120)
        at org.apache.spark.rdd.RDD.partitions(RDD.scala:203)
        at org.apache.spark.rdd.RDD.take(RDD.scala:1060)
        at org.apache.spark.rdd.RDD.first(RDD.scala:1093)
        at
com.ebay.ss.niffler.miner.acronym.AcronymMinerSparkLaunchOld$.test10(AcronymMiner.scala:664)
        at
com.ebay.ss.niffler.miner.acronym.AcronymMinerSparkLaunchOld$.main(AcronymMiner.scala:611)
        at
com.ebay.ss.niffler.miner.acronym.AcronymMinerSparkLaunchOld.main(AcronymMiner.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at
org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:427)


--
Dean Chen

On Wed, Mar 25, 2015 at 11:51 AM, Marcelo Vanzin <va...@cloudera.com>
wrote:

> Hey Patrick,
>
> The only issue I've seen so far has been the YARN container ID issue.
> That can be technically be described as a breakage in forwards
> compatibility in YARN. The APIs didn't break, but the data transferred
> through YARN's protocol has, and the old library cannot understand the
> data sent by a new service (the new container ID).
>
> The main issue with publishing BYOH is what Matei already mentioned.
> It would be worth it to take a look at what projects that depend on
> Hadoop do, though.
>
> Speaking with the Cloudera hat on, Spark in CDH is already "BYOH",
> except Hadoop is already there with the rest of CDH.
>
>
> On Tue, Mar 24, 2015 at 12:05 PM, Patrick Wendell <pw...@gmail.com>
> wrote:
> > Hey All,
> >
> > For a while we've published binary packages with different Hadoop
> > client's pre-bundled. We currently have three interfaces to a Hadoop
> > cluster (a) the HDFS client (b) the YARN client (c) the Hive client.
> >
> > Because (a) and (b) are supposed to be backwards compatible
> > interfaces. My working assumption was that for the most part (modulo
> > Hive) our packages work with *newer* Hadoop versions. For instance,
> > our Hadoop 2.4 package should work with HDFS 2.6 and YARN 2.6.
> > However, I have heard murmurings that these are not compatible in
> > practice.
> >
> > So I have three questions I'd like to put out to the community:
> >
> > 1. Have people had difficulty using 2.4 packages with newer Hadoop
> > versions? If so, what specific incompatibilities have you hit?
> > 2. Have people had issues using our binary Hadoop packages in general
> > with commercial or Apache Hadoop distro's, such that you have to build
> > from source?
> > 3. How would people feel about publishing a "bring your own Hadoop"
> > binary, where you are required to point us to a local Hadoop
> > distribution by setting HADOOP_HOME? This might be better for ensuring
> > full compatibility:
> > https://issues.apache.org/jira/browse/SPARK-6511
> >
> > - Patrick
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
> > For additional commands, e-mail: dev-help@spark.apache.org
> >
>
>
>
> --
> Marcelo
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: dev-unsubscribe@spark.apache.org
> For additional commands, e-mail: dev-help@spark.apache.org
>
>