Hi David,

This is not a bug. Maximum transfer size for SPDK is 128KB, i.e. you have to set your slicesize in the Crail configuration to 128KB. Does the Spark log not show any exception in this case? Have you checked your executor logs?

Regards,
Jonas


On Fri, 16 Aug 2019 21:10:35 +0000
 David Crespi <[email protected]> wrote:
Jonas,

I don’t know if this is your problem or not, so I thought I provide some info.

This behavior started after running the latest crail code, and it does appear

to have something to do with opening and closing files.


A few more things to note.

Running terasort with hdfs for tmp data & crail shuffle is okay

Running terrasort with crail for tmp data & not using crail shuffle is okay

Running terasort with crail for tmp data & crail shuffle causes this problem.

So when crail is used for everything, it falls apart.


What spdk is showing:

Aug 16 14:00:30 minnie a3f92fbd8b97[2136]: rdma.c:1422:spdk_nvmf_rdma_request_parse_sgl: *ERROR*: SGL length 0x77400 exceeds max io size 0x20000

Aug 16 14:00:30 minnie a3f92fbd8b97[2136]: rdma.c:2501:spdk_nvmf_process_ib_event: *NOTICE*: Async event: last WQE reached

Aug 16 14:00:30 minnie a3f92fbd8b97[2136]: rdma.c:2501:spdk_nvmf_process_ib_event: *NOTICE*: Async event: last WQE reached


From the client

************************************

        TeraGen

************************************

SLF4J: Class path contains multiple SLF4J bindings.

SLF4J: Found binding in [jar:file:/crail/jars/slf4j-log4j12-1.7.12.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/jnvmf-1.7-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/disni-2.1-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/usr/spark-2.4.2/jars/slf4j-log4j12-1.7.16.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]

19/08/16 13:50:11 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable

===========================================================================

===========================================================================

Input size: 1000MB

Total number of records: 10000000

Number of output partitions: 2

Number of records/output partition: 5000000

===========================================================================

===========================================================================

Number of records written: 10000000


real    0m13.046s

user    0m13.633s

sys     0m1.578s

************************************

        TeraSort

************************************

SLF4J: Class path contains multiple SLF4J bindings.

SLF4J: Found binding in [jar:file:/crail/jars/slf4j-log4j12-1.7.12.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/jnvmf-1.7-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/disni-2.1-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/usr/spark-2.4.2/jars/slf4j-log4j12-1.7.16.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]

19/08/16 13:50:24 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable

19/08/16 13:51:34 ERROR TaskSchedulerImpl: Lost executor 1 on 192.168.2.9: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:35 ERROR TaskSchedulerImpl: Lost executor 2 on 192.168.2.8: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:35 ERROR TaskSchedulerImpl: Lost executor 4 on 192.168.2.5: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:35 ERROR TaskSchedulerImpl: Lost executor 0 on 192.168.2.7: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:35 ERROR TaskSchedulerImpl: Lost executor 3 on 192.168.2.6: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:43 ERROR TaskSchedulerImpl: Lost executor 5 on 192.168.2.9: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:44 ERROR TaskSchedulerImpl: Lost executor 6 on 192.168.2.8: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

19/08/16 13:51:44 ERROR TaskSetManager: Task 6 in stage 0.0 failed 4 times; aborting job

19/08/16 13:51:44 ERROR SparkHadoopWriter: Aborting job job_20190816135127_0001.

org.apache.spark.SparkException: Job aborted due to stage failure: Task 6 in stage 0.0 failed 4 times, most recent failure: Lost task 6.3 in stage 0.0 (TID 26, 192.168.2.8, executor 6): ExecutorLostFailure (executor 6 exited caused by one of the running tasks) Reason: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

Driver stacktrace:

at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:1889)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:1877)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:1876)

at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)

at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)

at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)

at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1876)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:926)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:926)

       at scala.Option.foreach(Option.scala:274)

at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:926)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2110)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2059)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2048)

at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)

at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:737)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2114)

at org.apache.spark.internal.io.SparkHadoopWriter$.write(SparkHadoopWriter.scala:78)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopDataset$1(PairRDDFunctions.scala:1083)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopDataset(PairRDDFunctions.scala:1081)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopFile$2(PairRDDFunctions.scala:1000)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopFile(PairRDDFunctions.scala:991)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopFile$1(PairRDDFunctions.scala:979)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopFile(PairRDDFunctions.scala:979)

at com.github.ehiggs.spark.terasort.TeraSort$.main(TeraSort.scala:63)

at com.github.ehiggs.spark.terasort.TeraSort.main(TeraSort.scala)

at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)

at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)

at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)

       at java.lang.reflect.Method.invoke(Method.java:498)

at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)

at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:849)

at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167)

at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:195)

at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)

at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:924)

at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:933)

at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

Exception in thread "main" org.apache.spark.SparkException: Job aborted.

at org.apache.spark.internal.io.SparkHadoopWriter$.write(SparkHadoopWriter.scala:100)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopDataset$1(PairRDDFunctions.scala:1083)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopDataset(PairRDDFunctions.scala:1081)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopFile$2(PairRDDFunctions.scala:1000)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopFile(PairRDDFunctions.scala:991)

at org.apache.spark.rdd.PairRDDFunctions.$anonfun$saveAsNewAPIHadoopFile$1(PairRDDFunctions.scala:979)

at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)

at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)

       at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)

at org.apache.spark.rdd.PairRDDFunctions.saveAsNewAPIHadoopFile(PairRDDFunctions.scala:979)

at com.github.ehiggs.spark.terasort.TeraSort$.main(TeraSort.scala:63)

at com.github.ehiggs.spark.terasort.TeraSort.main(TeraSort.scala)

at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)

at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)

at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)

       at java.lang.reflect.Method.invoke(Method.java:498)

at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)

at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:849)

at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:167)

at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:195)

at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)

at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:924)

at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:933)

at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

Caused by: org.apache.spark.SparkException: Job aborted due to stage failure: Task 6 in stage 0.0 failed 4 times, most recent failure: Lost task 6.3 in stage 0.0 (TID 26, 192.168.2.8, executor 6): ExecutorLostFailure (executor 6 exited caused by one of the running tasks) Reason: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.

Driver stacktrace:

at org.apache.spark.scheduler.DAGScheduler.failJobAndIndependentStages(DAGScheduler.scala:1889)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2(DAGScheduler.scala:1877)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$abortStage$2$adapted(DAGScheduler.scala:1876)

at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)

at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)

at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)

at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1876)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1(DAGScheduler.scala:926)

at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleTaskSetFailed$1$adapted(DAGScheduler.scala:926)

       at scala.Option.foreach(Option.scala:274)

at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:926)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2110)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2059)

at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2048)

at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)

at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:737)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2082)

at org.apache.spark.SparkContext.runJob(SparkContext.scala:2114)

at org.apache.spark.internal.io.SparkHadoopWriter$.write(SparkHadoopWriter.scala:78)

       ... 32 more


hduser@master:/conf$ crail fs -ls -R /

SLF4J: Class path contains multiple SLF4J bindings.

SLF4J: Found binding in [jar:file:/crail/jars/slf4j-log4j12-1.7.12.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/jnvmf-1.7-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]

SLF4J: Found binding in [jar:file:/crail/jars/disni-2.1-jar-with-dependencies.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]

19/08/16 14:00:29 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable

19/08/16 14:00:29 INFO crail: CrailHadoopFileSystem construction

19/08/16 14:00:30 INFO crail: creating singleton crail file system

19/08/16 14:00:30 INFO crail: crail.version 3101

19/08/16 14:00:30 INFO crail: crail.directorydepth 16

19/08/16 14:00:30 INFO crail: crail.tokenexpiration 10

19/08/16 14:00:30 INFO crail: crail.blocksize 1048576

19/08/16 14:00:30 INFO crail: crail.cachelimit 0

19/08/16 14:00:30 INFO crail: crail.cachepath /dev/hugepages/cache

19/08/16 14:00:30 INFO crail: crail.user crail

19/08/16 14:00:30 INFO crail: crail.shadowreplication 1

19/08/16 14:00:30 INFO crail: crail.debug true

19/08/16 14:00:30 INFO crail: crail.statistics true

19/08/16 14:00:30 INFO crail: crail.rpctimeout 1000

19/08/16 14:00:30 INFO crail: crail.datatimeout 1000

19/08/16 14:00:30 INFO crail: crail.buffersize 1048576

19/08/16 14:00:30 INFO crail: crail.slicesize 65536

19/08/16 14:00:30 INFO crail: crail.singleton true

19/08/16 14:00:30 INFO crail: crail.regionsize 1073741824

19/08/16 14:00:30 INFO crail: crail.directoryrecord 512

19/08/16 14:00:30 INFO crail: crail.directoryrandomize true

19/08/16 14:00:30 INFO crail: crail.cacheimpl org.apache.crail.memory.MappedBufferCache

19/08/16 14:00:30 INFO crail: crail.locationmap

19/08/16 14:00:30 INFO crail: crail.namenode.address crail://192.168.1.164:9060

19/08/16 14:00:30 INFO crail: crail.namenode.blockselection roundrobin

19/08/16 14:00:30 INFO crail: crail.namenode.fileblocks 16

19/08/16 14:00:30 INFO crail: crail.namenode.rpctype org.apache.crail.namenode.rpc.tcp.TcpNameNode

19/08/16 14:00:30 INFO crail: crail.namenode.log

19/08/16 14:00:30 INFO crail: crail.storage.types org.apache.crail.storage.nvmf.NvmfStorageTier

19/08/16 14:00:30 INFO crail: crail.storage.classes 1

19/08/16 14:00:30 INFO crail: crail.storage.rootclass 0

19/08/16 14:00:30 INFO crail: crail.storage.keepalive 2

19/08/16 14:00:30 INFO crail: buffer cache, allocationCount 0, bufferCount 1024

19/08/16 14:00:30 INFO crail: Initialize Nvmf storage client

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.ip 192.168.2.100

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.port 4420

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.nqn nqn.2018-12.com.StorEdgeSystems:cntlr13

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.hostnqn nqn.2014-08.org.nvmexpress:uuid:1b4e28ba-2fa1-11d2-883f-0016d3cca420

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.allocationsize 1073741824

19/08/16 14:00:30 INFO crail: crail.storage.nvmf.queueSize 64

19/08/16 14:00:30 INFO narpc: new NaRPC server group v1.0, queueDepth 32, messageSize 512, nodealy true

19/08/16 14:00:30 INFO crail: crail.namenode.tcp.queueDepth 32

19/08/16 14:00:30 INFO crail: crail.namenode.tcp.messageSize 512

19/08/16 14:00:30 INFO crail: crail.namenode.tcp.cores 2

19/08/16 14:00:30 INFO crail: connected to namenode(s) /192.168.1.164:9060

19/08/16 14:00:30 INFO crail: CrailHadoopFileSystem fs initialization done..

19/08/16 14:00:30 INFO crail: lookupDirectory: path /

19/08/16 14:00:30 INFO crail: lookup: name /, success, fd 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /

19/08/16 14:00:30 INFO crail: lookup: name /, success, fd 0

19/08/16 14:00:30 INFO crail: getDirectoryList: /

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /, fd 0, streamId 1, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: Connecting to NVMf target at Transport address = /192.168.2.100:4420, subsystem NQN = nqn.2018-12.com.StorEdgeSystems:cntlr13

19/08/16 14:00:30 INFO crail: EndpointCache miss /192.168.2.100:4420, fsId 0, cache size 1

19/08/16 14:00:30 INFO crail: lookupDirectory: path /David

19/08/16 14:00:30 INFO crail: lookup: name /David, success, fd 1

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_sort

19/08/16 14:00:30 INFO crail: lookup: name /data_sort, success, fd 24128

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark

19/08/16 14:00:30 INFO crail: lookup: name /spark, success, fd 24131

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_in

19/08/16 14:00:30 INFO crail: lookup: name /data_in, success, fd 24118

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /, fd 0, streamId 1

drwxrwxrwx   - crail crail          0 2019-08-16 13:14 /David

19/08/16 14:00:30 INFO crail: lookupDirectory: path /David

19/08/16 14:00:30 INFO crail: lookup: name /David, success, fd 1

19/08/16 14:00:30 INFO crail: getDirectoryList: /David

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /David, fd 1, streamId 2, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /David, fd 1, streamId 2

drwxrwxrwx   - crail crail       2048 2019-08-16 13:50 /data_in

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_in

19/08/16 14:00:30 INFO crail: lookup: name /data_in, success, fd 24118

19/08/16 14:00:30 INFO crail: getDirectoryList: /data_in

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /data_in, fd 24118, streamId 3, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_in/_SUCCESS

19/08/16 14:00:30 INFO crail: lookup: name /data_in/_SUCCESS, success, fd 24127

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_in/part-r-00000

19/08/16 14:00:30 INFO crail: lookup: name /data_in/part-r-00000, success, fd 24126

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_in/part-r-00001

19/08/16 14:00:30 INFO crail: lookup: name /data_in/part-r-00001, success, fd 24125

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /data_in, fd 24118, streamId 3

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:50 /data_in/_SUCCESS

-rw-rw-rw- 1 crail crail 500000000 2019-08-16 13:50 /data_in/part-r-00000

-rw-rw-rw- 1 crail crail 500000000 2019-08-16 13:50 /data_in/part-r-00001

drwxrwxrwx   - crail crail        512 2019-08-16 13:51 /data_sort

19/08/16 14:00:30 INFO crail: lookupDirectory: path /data_sort

19/08/16 14:00:30 INFO crail: lookup: name /data_sort, success, fd 24128

19/08/16 14:00:30 INFO crail: getDirectoryList: /data_sort

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /data_sort, fd 24128, streamId 4, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /data_sort, fd 24128, streamId 4

drwxrwxrwx   - crail crail       2560 2019-08-16 13:51 /spark

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark

19/08/16 14:00:30 INFO crail: lookup: name /spark, success, fd 24131

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark, fd 24131, streamId 5, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/rdd

19/08/16 14:00:30 INFO crail: lookup: name /spark/rdd, success, fd 24134

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/tmp

19/08/16 14:00:30 INFO crail: lookup: name /spark/tmp, success, fd 24135

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/broadcast

19/08/16 14:00:30 INFO crail: lookup: name /spark/broadcast, success, fd 24132

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta, success, fd 24136

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/shuffle

19/08/16 14:00:30 INFO crail: lookup: name /spark/shuffle, success, fd 24133

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark, fd 24131, streamId 5

drwxrwxrwx - crail crail 0 2019-08-16 13:51 /spark/broadcast

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/broadcast

19/08/16 14:00:30 INFO crail: lookup: name /spark/broadcast, success, fd 24132

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/broadcast

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/broadcast, fd 24132, streamId 6, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/broadcast, fd 24132, streamId 6

drwxrwxrwx   - crail crail        512 2019-08-16 13:51 /spark/meta

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta, success, fd 24136

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/meta

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/meta, fd 24136, streamId 7, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts, success, fd 24137

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/meta, fd 24136, streamId 7

drwxrwxrwx - crail crail 3072 2019-08-16 13:51 /spark/meta/hosts

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts, success, fd 24137

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/meta/hosts

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/meta/hosts, fd 24137, streamId 8, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/35352998

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/35352998, success, fd 25096

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/35352997

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/35352997, success, fd 25097

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/35352995

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/35352995, success, fd 25098

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/35352994

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/35352994, success, fd 25095

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/35352996

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/35352996, success, fd 27836

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/meta/hosts/-1081267614

19/08/16 14:00:30 INFO crail: lookup: name /spark/meta/hosts/-1081267614, success, fd 24138

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/meta/hosts, fd 24137, streamId 8

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/-1081267614

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/35352994

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/35352995

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/35352996

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/35352997

-rw-rw-rw- 1 crail crail 0 2019-08-16 13:51 /spark/meta/hosts/35352998

drwxrwxrwx   - crail crail          0 2019-08-16 13:51 /spark/rdd

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/rdd

19/08/16 14:00:30 INFO crail: lookup: name /spark/rdd, success, fd 24134

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/rdd

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/rdd, fd 24134, streamId 9, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/rdd, fd 24134, streamId 9

drwxrwxrwx - crail crail 512 2019-08-16 13:51 /spark/shuffle

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/shuffle

19/08/16 14:00:30 INFO crail: lookup: name /spark/shuffle, success, fd 24133

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/shuffle

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/shuffle, fd 24133, streamId 10, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/shuffle/shuffle_0

19/08/16 14:00:30 INFO crail: lookup: name /spark/shuffle/shuffle_0, success, fd 24140

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/shuffle, fd 24133, streamId 10

drwxrwxrwx - crail crail 488448 2019-08-16 13:51 /spark/shuffle/shuffle_0

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/shuffle/shuffle_0

19/08/16 14:00:30 INFO crail: lookup: name /spark/shuffle/shuffle_0, success, fd 24140

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/shuffle/shuffle_0

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/shuffle/shuffle_0, fd 24140, streamId 11, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: Cannot close, pending operations, opcount 1, path /spark/shuffle/shuffle_0

19/08/16 14:00:30 INFO crail: error when closing directory stream java.io.IOException: Cannot close, pending operations, opcount 1

drwxrwxrwx   - crail crail       5632 2019-08-16 13:51 /spark/tmp

19/08/16 14:00:30 INFO crail: lookupDirectory: path /spark/tmp

19/08/16 14:00:30 INFO crail: lookup: name /spark/tmp, success, fd 24135

19/08/16 14:00:30 INFO crail: getDirectoryList: /spark/tmp

19/08/16 14:00:30 INFO crail: CoreInputStream: open, path /spark/tmp, fd 24135, streamId 12, isDir true, readHint 0

19/08/16 14:00:30 INFO crail: EndpointCache hit /192.168.2.100:4420, fsId 0

19/08/16 14:00:30 INFO crail: CoreInputStream, close, path /spark/tmp, fd 24135, streamId 12

19/08/16 14:00:30 INFO crail: Closing CrailHadoopFileSystem

19/08/16 14:00:30 INFO crail: Closing CrailFS singleton

19/08/16 14:00:30 INFO crail: Cannot close, pending operations, opcount 1, path /spark/shuffle/shuffle_0

java.io.IOException: java.io.IOException: Cannot close, pending operations, opcount 1

       at org.apache.crail.CrailStore.close(CrailStore.java:55)

at org.apache.crail.hdfs.CrailHadoopFileSystem.close(CrailHadoopFileSystem.java:290)

at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:2760)

at org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer.run(FileSystem.java:2777)

at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)

Caused by: java.io.IOException: Cannot close, pending operations, opcount 1

at org.apache.crail.core.CoreInputStream.close(CoreInputStream.java:108)

at org.apache.crail.core.CoreDataStore.closeFileSystem(CoreDataStore.java:515)

       at org.apache.crail.CrailStore.close(CrailStore.java:52)

       ... 4 more


Regards,


          David


C: 714-476-2692


________________________________
From: Jonas Pfefferle <[email protected]>
Sent: Friday, August 16, 2019 7:30:03 AM
To: [email protected] <[email protected]>; David Crespi <[email protected]> Subject: Re: [GitHub] [incubator-crail] PepperJo opened a new pull request #82: [NVMf] Make keepalive thread a daemon thread

Thanks! You too.
Let me know how it goes.


Regards,
Jonas

 On Fri, 16 Aug 2019 14:22:43 +0000
 David Crespi <[email protected]> wrote:
No, I haven’t tried the latest as there wasn’t any update to the
bug, so I wasn’t sure if you were successful

or not.  I will build new images with the latest and give it a shot.
Thanks and enjoy your weekend… your

a lot closer to it than me ??!


Regards,


          David


________________________________
From: Jonas Pfefferle <[email protected]>
Sent: Friday, August 16, 2019 12:24:30 AM
To: [email protected] <[email protected]>; David Crespi
<[email protected]>
Subject: Re: [GitHub] [incubator-crail] PepperJo opened a new pull
request #82: [NVMf] Make keepalive thread a daemon thread

Hi David,


at least for me, this pull request fixes the closing problem with
Spark.
Did you experience the hang at the start before or just with the
latest
Crail version?

Regards,
Jonas

 On Thu, 15 Aug 2019 19:25:05 +0000
 David Crespi <[email protected]> wrote:
Hi Jonas,

Did you ever get this to work?  I see the bug is still open, and no
update there.

Is there a way to work around this?  It appears when the file size
of teragen is large,

then crail just hangs when starting the sort.  I’ve also tried to do
it in parts,

with the same results.


Regards,


          David


________________________________
From: GitBox <[email protected]>
Sent: Monday, August 5, 2019 1:41:10 AM
To: [email protected] <[email protected]>
Subject: [GitHub] [incubator-crail] PepperJo opened a new pull
request #82: [NVMf] Make keepalive thread a daemon thread

PepperJo opened a new pull request #82: [NVMf] Make keepalive thread
a daemon thread
URL: https://github.com/apache/incubator-crail/pull/82


  Daemonize the keepalive thread to allow applications to
  exit when the main method returns without closing the
  storage client explicitly. For example, Spark has this
  requirement.

  https://issues.apache.org/jira/browse/CRAIL-98

  Signed-off-by: Jonas Pfefferle <[email protected]>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services



Reply via email to