[ https://issues.apache.org/jira/browse/SPARK-12279?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15126026#comment-15126026 ]
Pierre Beauvois commented on SPARK-12279: ----------------------------------------- Hi [~yabodnar], I recently had the opportunity to do more tests on this feature. I installed Spark 1.6.0 on our cluster so I to I took advantage of this opportunity to reconfigure entirely Spark/HBase. Here is my new configuration: - configuration of the HBase security {code} <property> <name>hbase.security.exec.permission.checks</name> <value>true</value> </property> <property> <name>hbase.security.visibility.mutations.checkauth</name> <value>false</value> </property> <property> <name>hbase.security.authentication</name> <value>kerberos</value> </property> <property> <name>hbase.security.authorization</name> <value>true</value> </property> {code} - creation of a symbolic link to hbase-site.xml in the dir /opt/application/Spark/current/conf/ - creation of symbolic links to HBase libraries in /opt/application/Spark/current/lib_managed/jars/ - addition of these HBase libraries in the spark-defaults.conf {code} # HBase path to the hbase main configuration ( spark.driver.extraClassPath = /opt/application/Spark/current/lib_managed/jars/hive-hbase-handler.jar:/opt/application/Spark/current/lib_managed/jars/hbase-server.jar:/opt/application/Spark/current/lib_managed/jars/hbase-protocol.jar:/opt/application/Spark/current/lib_managed/jars/hbase-hadoop2-compat.jar:/opt/application/Spark/current/lib_managed/jars/hbase-client.jar:/opt/application/Spark/current/lib_managed/jars/hbase-common.jar:/opt/application/Spark/current/lib_managed/jars/hbase-annotations.jar:/opt/application/Spark/current/lib_managed/jars/guava.jar:/opt/application/Spark/current/lib_managed/jars/htrace-core-incubating.jar {code} The good news is that I had been able to make it work but only in local mode. In local mode I did the following: {code} $ spark-shell -v --master local ... scala> import org.apache.spark._ import org.apache.spark._ scala> import org.apache.spark.rdd.NewHadoopRDD import org.apache.spark.rdd.NewHadoopRDD scala> import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.Path scala> import org.apache.hadoop.hbase.util.Bytes import org.apache.hadoop.hbase.util.Bytes scala> import org.apache.hadoop.hbase.HColumnDescriptor import org.apache.hadoop.hbase.HColumnDescriptor scala> import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} scala> import org.apache.hadoop.hbase.client.{HBaseAdmin, Put, HTable, Result} import org.apache.hadoop.hbase.client.{HBaseAdmin, Put, HTable, Result} scala> import org.apache.hadoop.hbase.mapreduce.TableInputFormat import org.apache.hadoop.hbase.mapreduce.TableInputFormat scala> import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.io.ImmutableBytesWritable scala> val conf = HBaseConfiguration.create() conf: org.apache.hadoop.conf.Configuration = Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml, hbase-default.xml, hbase-site.xml scala> conf.set("hbase.zookeeper.quorum", "myserver01.dns.fr:2181,myserver03.dns.fr:2181,myserver03.dns.fr:2181") scala> conf.set(TableInputFormat.INPUT_TABLE, "pbeauvois:death_causes") scala> val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD: org.apache.spark.rdd.RDD[(org.apache.hadoop.hbase.io.ImmutableBytesWritable, org.apache.hadoop.hbase.client.Result)] = NewHadoopRDD[0] at newAPIHadoopRDD at <console>:40 scala> hBaseRDD.collect().foreach(println) (33,keyvalues={1/case:count/1453278965864/Put/vlen=1/seqid=0, 1/case:death/1453278965830/Put/vlen=3/seqid=0, 1/case:location/1453278965901/Put/vlen=33/seqid=0, 1/case:year/1453278965670/Put/vlen=4/seqid=0, 1/case:zip/1453278965795/Put/vlen=5/seqid=0}) (33,keyvalues={2/case:count/1453278966013/Put/vlen=1/seqid=0, 2/case:death/1453278965985/Put/vlen=3/seqid=0, 2/case:location/1453278966063/Put/vlen=33/seqid=0, 2/case:year/1453278965932/Put/vlen=4/seqid=0, 2/case:zip/1453278965958/Put/vlen=5/seqid=0}) (33,keyvalues={3/case:count/1453278966173/Put/vlen=1/seqid=0, 3/case:death/1453278966148/Put/vlen=3/seqid=0, 3/case:location/1453278967718/Put/vlen=33/seqid=0, 3/case:year/1453278966101/Put/vlen=4/seqid=0, 3/case:zip/1453278966125/Put/vlen=5/seqid=0}) {code} I tried the same on "yarn-client" mode but I had the follwoing error stack: {code} scala> hBaseRDD.collect().foreach(println) 16/02/01 10:31:23 ERROR TaskSetManager: Task 0 in stage 0.0 failed 4 times; aborting job org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 4 times, most recent failure: Lost task 0.3 in stage 0.0 (TID 3, datanode04.dns.fr): java.lang.IllegalStateException: unread block data at java.io.ObjectInputStream$BlockDataInputStream.setBlockDataMode(ObjectInputStream.java:2421) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1382) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:115) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:193) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1431) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1419) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1418) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1418) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:799) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:799) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1640) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:620) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1832) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1845) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1858) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1929) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:927) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:111) at org.apache.spark.rdd.RDD.withScope(RDD.scala:316) at org.apache.spark.rdd.RDD.collect(RDD.scala:926) at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:43) at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:48) at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:50) at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:52) at $iwC$$iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:54) at $iwC$$iwC$$iwC$$iwC$$iwC.<init>(<console>:56) at $iwC$$iwC$$iwC$$iwC.<init>(<console>:58) at $iwC$$iwC$$iwC.<init>(<console>:60) at $iwC$$iwC.<init>(<console>:62) at $iwC.<init>(<console>:64) at <init>(<console>:66) at .<init>(<console>:70) at .<clinit>(<console>) at .<init>(<console>:7) at .<clinit>(<console>) at $print(<console>) 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.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:1065) at org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1346) at org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:840) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:871) at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:819) at org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:857) at org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:902) at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:814) at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:657) at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:665) at org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$loop(SparkILoop.scala:670) at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply$mcZ$sp(SparkILoop.scala:997) at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) at org.apache.spark.repl.SparkILoop$$anonfun$org$apache$spark$repl$SparkILoop$$process$1.apply(SparkILoop.scala:945) at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) at org.apache.spark.repl.SparkILoop.org$apache$spark$repl$SparkILoop$$process(SparkILoop.scala:945) at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:1059) at org.apache.spark.repl.Main$.main(Main.scala:31) at org.apache.spark.repl.Main.main(Main.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.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731) at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181) at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.IllegalStateException: unread block data at java.io.ObjectInputStream$BlockDataInputStream.setBlockDataMode(ObjectInputStream.java:2421) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1382) at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1990) at java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1915) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:115) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:193) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {code} Any ideas of the source of this error? Did I do a mistake in the configuration? > Requesting a HBase table with kerberos is not working > ----------------------------------------------------- > > Key: SPARK-12279 > URL: https://issues.apache.org/jira/browse/SPARK-12279 > Project: Spark > Issue Type: Bug > Components: YARN > Affects Versions: 1.5.2, 1.6.0 > Environment: Spark 1.6.0 / HBase 1.1.2 / Hadoop 2.7.1 / Zookeeper > 3.4.5 / Authentication done through Kerberos > Reporter: Pierre Beauvois > > I can't read a HBase table with Spark 1.5.2. > I added the option "spark.driver.extraClassPath" in the spark-defaults.conf > which contains the HBASE_CONF_DIR as below: > spark.driver.extraClassPath = /opt/application/Hbase/current/conf/ > On the driver, I started spark-shell (I was running it in yarn-client mode) > {code} > [my_user@uabigspark01 ~]$ spark-shell -v --name HBaseTest --jars > /opt/application/Hbase/current/lib/hbase-common-1.1.2.jar,/opt/application/Hbase/current/lib/hbase-server-1.1.2.jar,/opt/application/Hbase/current/lib/hbase-client-1.1.2.jar,/opt/application/Hbase/current/lib/hbase-protocol-1.1.2.jar,/opt/application/Hbase/current/lib/protobuf-java-2.5.0.jar,/opt/application/Hbase/current/lib/htrace-core-3.1.0-incubating.jar,/opt/application/Hbase/current/lib/hbase-annotations-1.1.2.jar,/opt/application/Hbase/current/lib/guava-12.0.1.jar > {code} > Then I ran the following lines: > {code} > scala> import org.apache.spark._ > import org.apache.spark._ > scala> import org.apache.spark.rdd.NewHadoopRDD > import org.apache.spark.rdd.NewHadoopRDD > scala> import org.apache.hadoop.fs.Path > import org.apache.hadoop.fs.Path > scala> import org.apache.hadoop.hbase.util.Bytes > import org.apache.hadoop.hbase.util.Bytes > scala> import org.apache.hadoop.hbase.HColumnDescriptor > import org.apache.hadoop.hbase.HColumnDescriptor > scala> import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} > import org.apache.hadoop.hbase.{HBaseConfiguration, HTableDescriptor} > scala> import org.apache.hadoop.hbase.client.{HBaseAdmin, Put, HTable, Result} > import org.apache.hadoop.hbase.client.{HBaseAdmin, Put, HTable, Result} > scala> import org.apache.hadoop.hbase.mapreduce.TableInputFormat > import org.apache.hadoop.hbase.mapreduce.TableInputFormat > scala> import org.apache.hadoop.hbase.io.ImmutableBytesWritable > import org.apache.hadoop.hbase.io.ImmutableBytesWritable > scala> val conf = HBaseConfiguration.create() > conf: org.apache.hadoop.conf.Configuration = Configuration: core-default.xml, > core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, > yarn-site.xml, hdfs-default.xml, hdfs-site.xml, hbase-default.xml, > hbase-site.xml > scala> conf.addResource(new > Path("/opt/application/Hbase/current/conf/hbase-site.xml")) > scala> conf.set("hbase.zookeeper.quorum", "FQDN1:2181,FQDN2:2181,FQDN3:2181") > scala> conf.set(TableInputFormat.INPUT_TABLE, "user:noheader") > scala> val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], > classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], > classOf[org.apache.hadoop.hbase.client.Result]) > 2015-12-09 15:17:58,890 INFO [main] storage.MemoryStore: > ensureFreeSpace(266248) called with curMem=0, maxMem=556038881 > 2015-12-09 15:17:58,892 INFO [main] storage.MemoryStore: Block broadcast_0 > stored as values in memory (estimated size 260.0 KB, free 530.0 MB) > 2015-12-09 15:17:59,196 INFO [main] storage.MemoryStore: > ensureFreeSpace(32808) called with curMem=266248, maxMem=556038881 > 2015-12-09 15:17:59,197 INFO [main] storage.MemoryStore: Block > broadcast_0_piece0 stored as bytes in memory (estimated size 32.0 KB, free > 530.0 MB) > 2015-12-09 15:17:59,199 INFO [sparkDriver-akka.actor.default-dispatcher-2] > storage.BlockManagerInfo: Added broadcast_0_piece0 in memory on > 192.168.200.208:60217 (size: 32.0 KB, free: 530.2 MB) > 2015-12-09 15:17:59,203 INFO [main] spark.SparkContext: Created broadcast 0 > from newAPIHadoopRDD at <console>:34 > hBaseRDD: > org.apache.spark.rdd.RDD[(org.apache.hadoop.hbase.io.ImmutableBytesWritable, > org.apache.hadoop.hbase.client.Result)] = NewHadoopRDD[0] at newAPIHadoopRDD > at <console>:34 > scala> hBaseRDD.count() > 2015-12-09 15:18:09,441 INFO [main] zookeeper.RecoverableZooKeeper: Process > identifier=hconnection-0x4a52ac6 connecting to ZooKeeper > ensemble=FQDN1:2181,FQDN2:2181,FQDN3:2181 > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:zookeeper.version=3.4.5-1392090, built on 09/30/2012 17:52 GMT > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:host.name=DRIVER.FQDN > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.version=1.7.0_85 > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.vendor=Oracle Corporation > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.home=/usr/lib/jvm/java-1.7.0-openjdk-1.7.0.85.x86_64/jre > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.class.path=/opt/application/Hbase/current/conf/:/opt/application/Spark/current/conf/:/opt/application/Spark/current/assembly/target/scala-2.10/spark-assembly-1.5.2-hadoop2.7.1.jar:/opt/application/Spark/current/lib_managed/jars/datanucleus-core-3.2.10.jar:/opt/application/Spark/current/lib_managed/jars/datanucleus-rdbms-3.2.9.jar:/opt/application/Spark/current/lib_managed/jars/datanucleus-api-jdo-3.2.6.jar:/opt/application/Hadoop/current/etc/hadoop/:/opt/application/Hadoop/current/etc/hadoop/ > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.io.tmpdir=/tmp > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:java.compiler=<NA> > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:os.name=Linux > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:os.arch=amd64 > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:os.version=2.6.32-573.3.1.el6.x86_64 > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:user.name=my_user > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:user.home=/home/my_user > 2015-12-09 15:18:09,456 INFO [main] zookeeper.ZooKeeper: Client > environment:user.dir=/home/my_user > 2015-12-09 15:18:09,457 INFO [main] zookeeper.ZooKeeper: Initiating client > connection, connectString=FQDN1:2181,FQDN2:2181,FQDN3:2181 > sessionTimeout=90000 watcher=hconnection-0x4a52ac60x0, > quorum=FQDN1:2181,FQDN2:2181,FQDN3:2181, baseZNode=/hbase > 2015-12-09 15:18:09,523 INFO [main-SendThread(FQDN2:2181)] > zookeeper.ClientCnxn: Opening socket connection to server > FQDN2/192.168.200.24:2181. Will not attempt to authenticate using SASL > (unknown error) > 2015-12-09 15:18:09,526 INFO [main-SendThread(FQDN2:2181)] > zookeeper.ClientCnxn: Socket connection established to > FQDN2/192.168.200.24:2181, initiating session > 2015-12-09 15:18:09,540 INFO [main-SendThread(FQDN2:2181)] > zookeeper.ClientCnxn: Session establishment complete on server > FQDN2/192.168.200.24:2181, sessionid = 0x25181db91c90010, negotiated timeout > = 40000 > 2015-12-09 15:18:09,685 INFO [main] util.RegionSizeCalculator: Calculating > region sizes for table "user:noheader". > 2015-12-09 15:18:10,192 INFO [main] > client.ConnectionManager$HConnectionImplementation: Closing master protocol: > MasterService > 2015-12-09 15:18:10,192 INFO [main] > client.ConnectionManager$HConnectionImplementation: Closing zookeeper > sessionid=0x25181db91c90010 > 2015-12-09 15:18:10,196 INFO [main-EventThread] zookeeper.ClientCnxn: > EventThread shut down > 2015-12-09 15:18:10,196 INFO [main] zookeeper.ZooKeeper: Session: > 0x25181db91c90010 closed > 2015-12-09 15:18:10,335 INFO [main] spark.SparkContext: Starting job: count > at <console>:37 > 2015-12-09 15:18:10,351 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Got job 0 (count at <console>:37) with 1 output > partitions > 2015-12-09 15:18:10,351 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Final stage: ResultStage 0(count at <console>:37) > 2015-12-09 15:18:10,352 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Parents of final stage: List() > 2015-12-09 15:18:10,353 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Missing parents: List() > 2015-12-09 15:18:10,361 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Submitting ResultStage 0 (NewHadoopRDD[0] at > newAPIHadoopRDD at <console>:34), which has no missing parents > 2015-12-09 15:18:10,402 INFO [dag-scheduler-event-loop] storage.MemoryStore: > ensureFreeSpace(1688) called with curMem=299056, maxMem=556038881 > 2015-12-09 15:18:10,403 INFO [dag-scheduler-event-loop] storage.MemoryStore: > Block broadcast_1 stored as values in memory (estimated size 1688.0 B, free > 530.0 MB) > 2015-12-09 15:18:10,412 INFO [dag-scheduler-event-loop] storage.MemoryStore: > ensureFreeSpace(1067) called with curMem=300744, maxMem=556038881 > 2015-12-09 15:18:10,412 INFO [dag-scheduler-event-loop] storage.MemoryStore: > Block broadcast_1_piece0 stored as bytes in memory (estimated size 1067.0 B, > free 530.0 MB) > 2015-12-09 15:18:10,413 INFO [sparkDriver-akka.actor.default-dispatcher-2] > storage.BlockManagerInfo: Added broadcast_1_piece0 in memory on > 192.168.200.208:60217 (size: 1067.0 B, free: 530.2 MB) > 2015-12-09 15:18:10,414 INFO [dag-scheduler-event-loop] spark.SparkContext: > Created broadcast 1 from broadcast at DAGScheduler.scala:861 > 2015-12-09 15:18:10,418 INFO [dag-scheduler-event-loop] > scheduler.DAGScheduler: Submitting 1 missing tasks from ResultStage 0 > (NewHadoopRDD[0] at newAPIHadoopRDD at <console>:34) > 2015-12-09 15:18:10,420 INFO [dag-scheduler-event-loop] > cluster.YarnScheduler: Adding task set 0.0 with 1 tasks > 2015-12-09 15:18:10,486 INFO [sparkDriver-akka.actor.default-dispatcher-2] > scheduler.TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, > EXECUTOR1.FQDN, RACK_LOCAL, 2716 bytes) > 2015-12-09 15:18:12,336 INFO [sparkDriver-akka.actor.default-dispatcher-2] > storage.BlockManagerInfo: Added broadcast_1_piece0 in memory on > EXECUTOR1.FQDN:44528 (size: 1067.0 B, free: 1589.8 MB) > 2015-12-09 15:18:12,888 INFO [sparkDriver-akka.actor.default-dispatcher-2] > storage.BlockManagerInfo: Added broadcast_0_piece0 in memory on > EXECUTOR1.FQDN:44528 (size: 32.0 KB, free: 1589.7 MB) > 2015-12-09 15:54:48,806 WARN [task-result-getter-0] > spark.ThrowableSerializationWrapper: Task exception could not be deserialized > java.lang.ClassNotFoundException: > org.apache.hadoop.hbase.client.RetriesExhaustedException > at java.net.URLClassLoader$1.run(URLClassLoader.java:366) > at java.net.URLClassLoader$1.run(URLClassLoader.java:355) > at java.security.AccessController.doPrivileged(Native Method) > at java.net.URLClassLoader.findClass(URLClassLoader.java:354) > at java.lang.ClassLoader.loadClass(ClassLoader.java:425) > at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) > at java.lang.ClassLoader.loadClass(ClassLoader.java:358) > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:278) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:67) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1612) > at > java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1517) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1771) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) > at > org.apache.spark.ThrowableSerializationWrapper.readObject(TaskEndReason.scala:167) > 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 > java.io.ObjectStreamClass.invokeReadObject(ObjectStreamClass.java:1017) > at > java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1897) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) > at > java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1997) > at > java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1921) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) > at > java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.java:1997) > at > java.io.ObjectInputStream.readSerialData(ObjectInputStream.java:1921) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1798) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1350) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:370) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:72) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:98) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$3$$anonfun$run$2.apply$mcV$sp(TaskResultGetter.scala:108) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$3$$anonfun$run$2.apply(TaskResultGetter.scala:105) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$3$$anonfun$run$2.apply(TaskResultGetter.scala:105) > at > org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1699) > at > org.apache.spark.scheduler.TaskResultGetter$$anon$3.run(TaskResultGetter.scala:105) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > 2015-12-09 15:54:48,813 ERROR [task-result-getter-0] > scheduler.TaskResultGetter: Could not deserialize TaskEndReason: > ClassNotFound with classloader > org.apache.spark.repl.SparkIMain$TranslatingClassLoader@6735e614 > 2015-12-09 15:54:48,814 WARN [task-result-getter-0] > scheduler.TaskSetManager: Lost task 0.0 in stage 0.0 (TID 0, EXECUTOR1.FQDN): > UnknownReason > 2015-12-09 15:54:48,819 INFO [sparkDriver-akka.actor.default-dispatcher-16] > scheduler.TaskSetManager: Starting task 0.1 in stage 0.0 (TID 1, > EXECUTOR2.FQDN, RACK_LOCAL, 2716 bytes) > 2015-12-09 15:54:50,546 INFO [sparkDriver-akka.actor.default-dispatcher-16] > storage.BlockManagerInfo: Added broadcast_1_piece0 in memory on > EXECUTOR2.FQDN:45968 (size: 1067.0 B, free: 1589.8 MB) > 2015-12-09 15:54:51,128 INFO [sparkDriver-akka.actor.default-dispatcher-16] > storage.BlockManagerInfo: Added broadcast_0_piece0 in memory on > EXECUTOR2.FQDN:45968 (size: 32.0 KB, free: 1589.7 MB) > {code} > During the execution of the "hBaseRDD.count()", the process was regularly > paused (check the dates). > At the same i saw the following errors on an executor (check the dates again). > {code} > SLF4J: Class path contains multiple SLF4J bindings. > SLF4J: Found binding in > [jar:file:/mnt/hd4/hadoop/yarn/local/usercache/my_user/filecache/766/spark-assembly-1.5.2-hadoop2.7.1.jar!/org/slf4j/impl/StaticLoggerBinder.class] > SLF4J: Found binding in > [jar:file:/opt/application/Hadoop/hadoop-2.7.1/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class] > SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an > explanation. > SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] > 15/12/09 15:17:04 INFO executor.CoarseGrainedExecutorBackend: Registered > signal handlers for [TERM, HUP, INT] > 15/12/09 15:17:05 INFO spark.SecurityManager: Changing view acls to: my_user > 15/12/09 15:17:05 INFO spark.SecurityManager: Changing modify acls to: my_user > 15/12/09 15:17:05 INFO spark.SecurityManager: SecurityManager: authentication > disabled; ui acls disabled; users with view permissions: Set(my_user); users > with modify permissions: Set(my_user) > 15/12/09 15:17:05 INFO slf4j.Slf4jLogger: Slf4jLogger started > 15/12/09 15:17:05 INFO Remoting: Starting remoting > 15/12/09 15:17:05 INFO Remoting: Remoting started; listening on addresses > :[akka.tcp://driverpropsfetc...@executor1.fqdn:36241] > 15/12/09 15:17:05 INFO util.Utils: Successfully started service > 'driverPropsFetcher' on port 36241. > 15/12/09 15:17:06 INFO spark.SecurityManager: Changing view acls to: my_user > 15/12/09 15:17:06 INFO spark.SecurityManager: Changing modify acls to: my_user > 15/12/09 15:17:06 INFO spark.SecurityManager: SecurityManager: authentication > disabled; ui acls disabled; users with view permissions: Set(my_user); users > with modify permissions: Set(my_user) > 15/12/09 15:17:06 INFO remote.RemoteActorRefProvider$RemotingTerminator: > Shutting down remote daemon. > 15/12/09 15:17:06 INFO remote.RemoteActorRefProvider$RemotingTerminator: > Remote daemon shut down; proceeding with flushing remote transports. > 15/12/09 15:17:06 INFO remote.RemoteActorRefProvider$RemotingTerminator: > Remoting shut down. > 15/12/09 15:17:06 INFO slf4j.Slf4jLogger: Slf4jLogger started > 15/12/09 15:17:06 INFO Remoting: Starting remoting > 15/12/09 15:17:06 INFO Remoting: Remoting started; listening on addresses > :[akka.tcp://sparkexecu...@executor1.fqdn:56581] > 15/12/09 15:17:06 INFO util.Utils: Successfully started service > 'sparkExecutor' on port 56581. > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-20cd4d44-6f07-4ea8-ab3d-6341e1bd94cd > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd1/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-9a447538-ec8b-4803-a683-a7527ed9e00f > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd2/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-bbd7b671-30a5-49eb-9a05-4fdfa1d86de7 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd3/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-a4987072-5a10-4b96-a2bc-79477f5a8bd0 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd4/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-3c85fdac-d29c-43b5-bcca-2296274f6381 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd5/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-c8bb3422-f9d7-4e0b-8bb7-39c844d04bba > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd6/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-e5de8172-1476-46ba-867a-288a5436efb9 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-fee9b2e2-2e03-4855-8f2f-44bd43006e01 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd8/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-69b1f7d3-d35e-40b4-8cb4-437fae50fcfa > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd9/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-11b53137-59ba-4e16-8faa-1f2a4c75c721 > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd10/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-ade507aa-9b4c-44b6-b27d-1485cd7dd66a > 15/12/09 15:17:07 INFO storage.DiskBlockManager: Created local directory at > /mnt/hd11/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/blockmgr-443240c7-895a-4241-9e4d-89de48e0bca1 > 15/12/09 15:17:07 INFO storage.MemoryStore: MemoryStore started with capacity > 1589.8 MB > 15/12/09 15:17:08 INFO executor.CoarseGrainedExecutorBackend: Connecting to > driver: akka.tcp://sparkdri...@driver.fqdn:62433/user/CoarseGrainedScheduler > 15/12/09 15:17:08 INFO executor.CoarseGrainedExecutorBackend: Successfully > registered with driver > 15/12/09 15:17:08 INFO executor.Executor: Starting executor ID 2 on host > EXECUTOR1.FQDN > 15/12/09 15:17:08 INFO util.Utils: Successfully started service > 'org.apache.spark.network.netty.NettyBlockTransferService' on port 44528. > 15/12/09 15:17:08 INFO netty.NettyBlockTransferService: Server created on > 44528 > 15/12/09 15:17:08 INFO storage.BlockManagerMaster: Trying to register > BlockManager > 15/12/09 15:17:08 INFO storage.BlockManagerMaster: Registered BlockManager > 15/12/09 15:17:08 INFO executor.Executor: Using REPL class URI: > http://DRIVER.FQDN:53441 > 15/12/09 15:18:10 INFO executor.CoarseGrainedExecutorBackend: Got assigned > task 0 > 15/12/09 15:18:10 INFO executor.Executor: Running task 0.0 in stage 0.0 (TID > 0) > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/protobuf-java-2.5.0.jar with timestamp > 1449670596931 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/protobuf-java-2.5.0.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp154889812017775736.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/21289125271449670596931_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./protobuf-java-2.5.0.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./protobuf-java-2.5.0.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/hbase-common-1.1.2.jar with timestamp > 1449670596905 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/hbase-common-1.1.2.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp3207469898154433839.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/17557465671449670596905_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-common-1.1.2.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-common-1.1.2.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/hbase-server-1.1.2.jar with timestamp > 1449670596916 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/hbase-server-1.1.2.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp1412976626726073257.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/-7756328011449670596916_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-server-1.1.2.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-server-1.1.2.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/hbase-annotations-1.1.2.jar with timestamp > 1449670596935 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/hbase-annotations-1.1.2.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp2679836512883108007.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/4608020401449670596935_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-annotations-1.1.2.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-annotations-1.1.2.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/htrace-core-3.1.0-incubating.jar with timestamp > 1449670596935 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/htrace-core-3.1.0-incubating.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp1461961544034139965.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/-10342791481449670596935_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./htrace-core-3.1.0-incubating.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./htrace-core-3.1.0-incubating.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/hbase-client-1.1.2.jar with timestamp > 1449670596919 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/hbase-client-1.1.2.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp9217964731951772544.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/-7183065851449670596919_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-client-1.1.2.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-client-1.1.2.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/hbase-protocol-1.1.2.jar with timestamp > 1449670596930 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/hbase-protocol-1.1.2.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp8280539781420543251.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/-5481020281449670596930_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-protocol-1.1.2.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./hbase-protocol-1.1.2.jar > to class loader > 15/12/09 15:18:10 INFO executor.Executor: Fetching > http://DRIVER.FQDN:58525/jars/guava-12.0.1.jar with timestamp 1449670596939 > 15/12/09 15:18:10 INFO util.Utils: Fetching > http://DRIVER.FQDN:58525/jars/guava-12.0.1.jar to > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/fetchFileTemp6766780549158760761.tmp > 15/12/09 15:18:10 INFO util.Utils: Copying > /mnt/hd0/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/spark-58907fb7-5a3d-49af-b675-37a7cdd6ba27/-14937723981449670596939_cache > to > /mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./guava-12.0.1.jar > 15/12/09 15:18:10 INFO executor.Executor: Adding > file:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/./guava-12.0.1.jar > to class loader > 15/12/09 15:18:11 INFO broadcast.TorrentBroadcast: Started reading broadcast > variable 1 > 15/12/09 15:18:12 INFO storage.MemoryStore: ensureFreeSpace(1067) called with > curMem=0, maxMem=1666984181 > 15/12/09 15:18:12 INFO storage.MemoryStore: Block broadcast_1_piece0 stored > as bytes in memory (estimated size 1067.0 B, free 1589.8 MB) > 15/12/09 15:18:12 INFO broadcast.TorrentBroadcast: Reading broadcast variable > 1 took 1214 ms > 15/12/09 15:18:12 INFO storage.MemoryStore: ensureFreeSpace(1688) called with > curMem=1067, maxMem=1666984181 > 15/12/09 15:18:12 INFO storage.MemoryStore: Block broadcast_1 stored as > values in memory (estimated size 1688.0 B, free 1589.8 MB) > 15/12/09 15:18:12 INFO rdd.NewHadoopRDD: Input split: HBase table split(table > name: my_user:noheader, scan: , start row: , end row: , region location: > EXECUTOR3.FQDN) > 15/12/09 15:18:12 INFO broadcast.TorrentBroadcast: Started reading broadcast > variable 0 > 15/12/09 15:18:12 INFO storage.MemoryStore: ensureFreeSpace(32808) called > with curMem=2755, maxMem=1666984181 > 15/12/09 15:18:12 INFO storage.MemoryStore: Block broadcast_0_piece0 stored > as bytes in memory (estimated size 32.0 KB, free 1589.7 MB) > 15/12/09 15:18:12 INFO broadcast.TorrentBroadcast: Reading broadcast variable > 0 took 25 ms > 15/12/09 15:18:13 INFO storage.MemoryStore: ensureFreeSpace(398840) called > with curMem=35563, maxMem=1666984181 > 15/12/09 15:18:13 INFO storage.MemoryStore: Block broadcast_0 stored as > values in memory (estimated size 389.5 KB, free 1589.3 MB) > 15/12/09 15:18:13 INFO zookeeper.RecoverableZooKeeper: Process > identifier=hconnection-0x46aff06c connecting to ZooKeeper > ensemble=FQDN1:2181,FQDN2:2181,FQDN3:2181 > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:zookeeper.version=3.4.5-1392090, built on 09/30/2012 17:52 GMT > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:host.name=EXECUTOR1.FQDN > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.version=1.7.0_79 > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.vendor=Oracle Corporation > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.home=/usr/lib/jvm/java-1.7.0-openjdk-1.7.0.79.x86_64/jre > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.class.path=/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003:/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/__spark__.jar:/opt/application/Hadoop/current/etc/hadoop/:/opt/application/Hadoop/current/share/hadoop/common/hadoop-nfs-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/hadoop-common-2.7.1-tests.jar:/opt/application/Hadoop/current/share/hadoop/common/hadoop-common-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-logging-1.1.3.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/java-xmlbuilder-0.4.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jsp-api-2.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/hadoop-auth-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jsr305-3.0.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/httpcore-4.2.5.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-beanutils-1.7.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/avro-1.7.4.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-net-3.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jackson-jaxrs-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/apacheds-kerberos-codec-2.0.0-M15.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/hadoop-annotations-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/stax-api-1.0-2.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-io-2.4.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-lang-2.6.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-httpclient-3.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/protobuf-java-2.5.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-compress-1.4.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jetty-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/paranamer-2.3.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/log4j-1.2.17.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/guava-11.0.2.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/curator-client-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jaxb-api-2.2.2.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/curator-recipes-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-beanutils-core-1.8.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/mockito-all-1.8.5.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-cli-1.2.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jettison-1.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/apacheds-i18n-2.0.0-M15.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/servlet-api-2.5.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/api-util-1.0.0-M20.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jackson-core-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-configuration-1.6.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/curator-framework-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jackson-mapper-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/netty-3.6.2.Final.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jsch-0.1.42.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jersey-core-1.9.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-codec-1.4.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/activation-1.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jersey-json-1.9.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/junit-4.11.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/slf4j-api-1.7.10.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/zookeeper-3.4.6.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/hamcrest-core-1.3.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/snappy-java-1.0.4.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/asm-3.2.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/htrace-core-3.1.0-incubating.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-math3-3.1.1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jaxb-impl-2.2.3-1.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jetty-util-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/xmlenc-0.52.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/xz-1.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/gson-2.2.4.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jets3t-0.9.0.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/httpclient-4.2.5.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jersey-server-1.9.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/jackson-xc-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-digester-1.8.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/api-asn1-api-1.0.0-M20.jar:/opt/application/Hadoop/current/share/hadoop/common/lib/commons-collections-3.2.1.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/hadoop-hdfs-2.7.1-tests.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/hadoop-hdfs-nfs-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/hadoop-hdfs-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-logging-1.1.3.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jsr305-3.0.0.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-io-2.4.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-lang-2.6.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/protobuf-java-2.5.0.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-daemon-1.0.13.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jetty-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/log4j-1.2.17.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/guava-11.0.2.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-cli-1.2.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/servlet-api-2.5.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jackson-core-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/netty-all-4.0.23.Final.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jackson-mapper-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/netty-3.6.2.Final.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jersey-core-1.9.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/commons-codec-1.4.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/leveldbjni-all-1.8.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/asm-3.2.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/htrace-core-3.1.0-incubating.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/xml-apis-1.3.04.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jetty-util-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/xmlenc-0.52.jar:/opt/application/Hadoop/current/share/hadoop/hdfs/lib/jersey-server-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-common-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-client-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-api-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-logging-1.1.3.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/zookeeper-3.4.6-tests.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jsr305-3.0.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jersey-guice-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/stax-api-1.0-2.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-io-2.4.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-lang-2.6.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-compress-1.4.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jetty-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/guice-3.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/log4j-1.2.17.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/guava-11.0.2.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-cli-1.2.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jettison-1.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/servlet-api-2.5.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/aopalliance-1.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/netty-3.6.2.Final.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jersey-core-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-codec-1.4.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/javax.inject-1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/activation-1.1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jersey-json-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/zookeeper-3.4.6.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/asm-3.2.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jersey-client-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jetty-util-6.1.26.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/xz-1.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jersey-server-1.9.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/guice-servlet-3.0.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/yarn/lib/commons-collections-3.2.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-shuffle-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-plugins-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-common-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1-tests.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/hadoop-mapreduce-client-app-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/avro-1.7.4.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/jersey-guice-1.9.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/hadoop-annotations-2.7.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/commons-io-2.4.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/protobuf-java-2.5.0.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/commons-compress-1.4.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/guice-3.0.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/paranamer-2.3.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/log4j-1.2.17.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/jackson-core-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/aopalliance-1.0.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/jackson-mapper-asl-1.9.13.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/netty-3.6.2.Final.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/jersey-core-1.9.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/javax.inject-1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/junit-4.11.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/leveldbjni-all-1.8.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/hamcrest-core-1.3.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/snappy-java-1.0.4.1.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/asm-3.2.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/xz-1.0.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/jersey-server-1.9.jar:/opt/application/Hadoop/current/share/hadoop/mapreduce/lib/guice-servlet-3.0.jar > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.library.path=/opt/application/Hadoop/current/lib/native/ > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.io.tmpdir=/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003/tmp > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:java.compiler=<NA> > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client environment:os.name=Linux > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client environment:os.arch=amd64 > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:os.version=2.6.32-504.8.1.el6.x86_64 > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:user.name=my_user > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:user.home=/home/my_user > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Client > environment:user.dir=/mnt/hd7/hadoop/yarn/local/usercache/my_user/appcache/application_1444912162557_0919/container_1444912162557_0919_01_000003 > 15/12/09 15:18:13 INFO zookeeper.ZooKeeper: Initiating client connection, > connectString=FQDN1:2181,FQDN2:2181,FQDN3:2181 sessionTimeout=90000 > watcher=hconnection-0x46aff06c0x0, quorum=FQDN1:2181,FQDN2:2181,FQDN3:2181, > baseZNode=/hbase > 15/12/09 15:18:13 INFO zookeeper.ClientCnxn: Opening socket connection to > server FQDN3/192.168.200.25:2181. Will not attempt to authenticate using SASL > (unknown error) > 15/12/09 15:18:13 INFO zookeeper.ClientCnxn: Socket connection established to > FQDN3/192.168.200.25:2181, initiating session > 15/12/09 15:18:13 INFO zookeeper.ClientCnxn: Session establishment complete > on server FQDN3/192.168.200.25:2181, sessionid = 0x35181db91ef0010, > negotiated timeout = 40000 > 15/12/09 15:18:14 INFO mapreduce.TableInputFormatBase: Input split length: 0 > bytes. > 15/12/09 15:18:14 WARN ipc.AbstractRpcClient: Exception encountered while > connecting to the server : javax.security.sasl.SaslException: GSS initiate > failed [Caused by GSSException: No valid credentials provided (Mechanism > level: Failed to find any Kerberos tgt)] > 15/12/09 15:18:14 ERROR ipc.AbstractRpcClient: SASL authentication failed. > The most likely cause is missing or invalid credentials. Consider 'kinit'. > javax.security.sasl.SaslException: GSS initiate failed [Caused by > GSSException: No valid credentials provided (Mechanism level: Failed to find > any Kerberos tgt)] > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) > at > org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:179) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupSaslConnection(RpcClientImpl.java:642) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.access$600(RpcClientImpl.java:166) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:769) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:766) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupIOstreams(RpcClientImpl.java:766) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.writeRequest(RpcClientImpl.java:920) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.tracedWriteRequest(RpcClientImpl.java:889) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1222) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:213) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:287) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(ClientProtos.java:32651) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:201) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:180) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:346) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:320) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126) > at > org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:64) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos tgt) > at > sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:121) > at > sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187) > at > sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:223) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179) > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193) > ... 25 more > 15/12/09 15:18:18 WARN ipc.AbstractRpcClient: Exception encountered while > connecting to the server : javax.security.sasl.SaslException: GSS initiate > failed [Caused by GSSException: No valid credentials provided (Mechanism > level: Failed to find any Kerberos tgt)] > 15/12/09 15:18:18 ERROR ipc.AbstractRpcClient: SASL authentication failed. > The most likely cause is missing or invalid credentials. Consider 'kinit'. > javax.security.sasl.SaslException: GSS initiate failed [Caused by > GSSException: No valid credentials provided (Mechanism level: Failed to find > any Kerberos tgt)] > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) > at > org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:179) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupSaslConnection(RpcClientImpl.java:642) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.access$600(RpcClientImpl.java:166) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:769) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:766) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupIOstreams(RpcClientImpl.java:766) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.writeRequest(RpcClientImpl.java:920) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.tracedWriteRequest(RpcClientImpl.java:889) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1222) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:213) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:287) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(ClientProtos.java:32651) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:201) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:180) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:346) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:320) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126) > at > org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:64) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos tgt) > at > sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:121) > at > sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187) > at > sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:223) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179) > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193) > ... 25 more > ... > MORE THAN ONE HUNDRED THOUSAND LINES LIKE THE PREVIOUS ONE... > ... > 15/12/09 15:54:00 WARN ipc.AbstractRpcClient: Exception encountered while > connecting to the server : javax.security.sasl.SaslException: GSS initiate > failed [Caused by GSSException: No valid credentials provided (Mechanism > level: Failed to find any Kerberos tgt)] > 15/12/09 15:54:00 ERROR ipc.AbstractRpcClient: SASL authentication failed. > The most likely cause is missing or invalid credentials. Consider 'kinit'. > javax.security.sasl.SaslException: GSS initiate failed [Caused by > GSSException: No valid credentials provided (Mechanism level: Failed to find > any Kerberos tgt)] > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) > at > org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:179) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupSaslConnection(RpcClientImpl.java:642) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.access$600(RpcClientImpl.java:166) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:769) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:766) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupIOstreams(RpcClientImpl.java:766) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.writeRequest(RpcClientImpl.java:920) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.tracedWriteRequest(RpcClientImpl.java:889) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1222) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:213) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:287) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(ClientProtos.java:32651) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:201) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:180) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:346) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:320) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126) > at > org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:64) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos tgt) > at > sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:121) > at > sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187) > at > sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:223) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179) > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193) > ... 25 more > 15/12/09 15:54:04 WARN ipc.AbstractRpcClient: Exception encountered while > connecting to the server : javax.security.sasl.SaslException: GSS initiate > failed [Caused by GSSException: No valid credentials provided (Mechanism > level: Failed to find any Kerberos tgt)] > 15/12/09 15:54:04 ERROR ipc.AbstractRpcClient: SASL authentication failed. > The most likely cause is missing or invalid credentials. Consider 'kinit'. > javax.security.sasl.SaslException: GSS initiate failed [Caused by > GSSException: No valid credentials provided (Mechanism level: Failed to find > any Kerberos tgt)] > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212) > at > org.apache.hadoop.hbase.security.HBaseSaslRpcClient.saslConnect(HBaseSaslRpcClient.java:179) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupSaslConnection(RpcClientImpl.java:642) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.access$600(RpcClientImpl.java:166) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:769) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run(RpcClientImpl.java:766) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupIOstreams(RpcClientImpl.java:766) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.writeRequest(RpcClientImpl.java:920) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.tracedWriteRequest(RpcClientImpl.java:889) > at > org.apache.hadoop.hbase.ipc.RpcClientImpl.call(RpcClientImpl.java:1222) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(AbstractRpcClient.java:213) > at > org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(AbstractRpcClient.java:287) > at > org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(ClientProtos.java:32651) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:201) > at > org.apache.hadoop.hbase.client.ClientSmallScanner$SmallScannerCallable.call(ClientSmallScanner.java:180) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:346) > at > org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(ScannerCallableWithReplicas.java:320) > at > org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(RpcRetryingCaller.java:126) > at > org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run(ResultBoundedCompletionService.java:64) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Caused by: GSSException: No valid credentials provided (Mechanism level: > Failed to find any Kerberos tgt) > at > sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147) > at > sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:121) > at > sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187) > at > sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:223) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212) > at > sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179) > at > com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193) > ... 25 more > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org