[jira] [Commented] (SPARK-15390) Memory management issue in complex DataFrame join and filter
[ https://issues.apache.org/jira/browse/SPARK-15390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15547955#comment-15547955 ] Iulian Dragos commented on SPARK-15390: --- Thanks [~davies]! > Memory management issue in complex DataFrame join and filter > > > Key: SPARK-15390 > URL: https://issues.apache.org/jira/browse/SPARK-15390 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.0.0 > Environment: branch-2.0, 16 workers >Reporter: Joseph K. Bradley >Assignee: Davies Liu > Fix For: 2.0.1 > > > See [SPARK-15389] for a description of the code which produces this bug. I > am filing this as a separate JIRA since the bug in 2.0 is different. > In 2.0, the code fails with some memory management error. Here is the > stacktrace: > {code} > OpenJDK 64-Bit Server VM warning: ignoring option MaxPermSize=512m; support > was removed in 8.0 > 16/05/18 19:23:16 ERROR Uncaught throwable from user code: > org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: > Exchange SinglePartition, None > +- WholeStageCodegen >: +- TungstenAggregate(key=[], > functions=[(count(1),mode=Partial,isDistinct=false)], output=[count#170L]) >: +- Project >:+- BroadcastHashJoin [id#70L], [id#110L], Inner, BuildLeft, None >: :- INPUT >: +- Project [id#110L] >: +- Filter (degree#115 > 200) >: +- TungstenAggregate(key=[id#110L], > functions=[(count(1),mode=Final,isDistinct=false)], > output=[id#110L,degree#115]) >:+- INPUT >:- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint])) >: +- WholeStageCodegen >: : +- Project [row#66.id AS id#70L] >: : +- Filter isnotnull(row#66.id) >: :+- INPUT >: +- Scan ExistingRDD[row#66,uniq_id#67] >+- Exchange hashpartitioning(id#110L, 200), None > +- WholeStageCodegen > : +- TungstenAggregate(key=[id#110L], > functions=[(count(1),mode=Partial,isDistinct=false)], > output=[id#110L,count#136L]) > : +- Filter isnotnull(id#110L) > :+- INPUT > +- Generate explode(array(src#2L, dst#3L)), false, false, [id#110L] > +- WholeStageCodegen >: +- Filter ((isnotnull(src#2L) && isnotnull(dst#3L)) && NOT > (src#2L = dst#3L)) >: +- INPUT >+- InMemoryTableScan [src#2L,dst#3L], > [isnotnull(src#2L),isnotnull(dst#3L),NOT (src#2L = dst#3L)], InMemoryRelation > [src#2L,dst#3L], true, 1, StorageLevel(disk=true, memory=true, > offheap=false, deserialized=true, replication=1), WholeStageCodegen, None > at > org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) > at > org.apache.spark.sql.execution.exchange.ShuffleExchange.doExecute(ShuffleExchange.scala:113) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:233) > at > org.apache.spark.sql.execution.aggregate.TungstenAggregate.inputRDDs(TungstenAggregate.scala:134) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:348) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:240) > at > org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:287) > at > org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2122) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) >
[jira] [Reopened] (SPARK-15390) Memory management issue in complex DataFrame join and filter
[ https://issues.apache.org/jira/browse/SPARK-15390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos reopened SPARK-15390: --- > Memory management issue in complex DataFrame join and filter > > > Key: SPARK-15390 > URL: https://issues.apache.org/jira/browse/SPARK-15390 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 2.0.0 > Environment: branch-2.0, 16 workers >Reporter: Joseph K. Bradley >Assignee: Davies Liu > Fix For: 2.0.0 > > > See [SPARK-15389] for a description of the code which produces this bug. I > am filing this as a separate JIRA since the bug in 2.0 is different. > In 2.0, the code fails with some memory management error. Here is the > stacktrace: > {code} > OpenJDK 64-Bit Server VM warning: ignoring option MaxPermSize=512m; support > was removed in 8.0 > 16/05/18 19:23:16 ERROR Uncaught throwable from user code: > org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: > Exchange SinglePartition, None > +- WholeStageCodegen >: +- TungstenAggregate(key=[], > functions=[(count(1),mode=Partial,isDistinct=false)], output=[count#170L]) >: +- Project >:+- BroadcastHashJoin [id#70L], [id#110L], Inner, BuildLeft, None >: :- INPUT >: +- Project [id#110L] >: +- Filter (degree#115 > 200) >: +- TungstenAggregate(key=[id#110L], > functions=[(count(1),mode=Final,isDistinct=false)], > output=[id#110L,degree#115]) >:+- INPUT >:- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint])) >: +- WholeStageCodegen >: : +- Project [row#66.id AS id#70L] >: : +- Filter isnotnull(row#66.id) >: :+- INPUT >: +- Scan ExistingRDD[row#66,uniq_id#67] >+- Exchange hashpartitioning(id#110L, 200), None > +- WholeStageCodegen > : +- TungstenAggregate(key=[id#110L], > functions=[(count(1),mode=Partial,isDistinct=false)], > output=[id#110L,count#136L]) > : +- Filter isnotnull(id#110L) > :+- INPUT > +- Generate explode(array(src#2L, dst#3L)), false, false, [id#110L] > +- WholeStageCodegen >: +- Filter ((isnotnull(src#2L) && isnotnull(dst#3L)) && NOT > (src#2L = dst#3L)) >: +- INPUT >+- InMemoryTableScan [src#2L,dst#3L], > [isnotnull(src#2L),isnotnull(dst#3L),NOT (src#2L = dst#3L)], InMemoryRelation > [src#2L,dst#3L], true, 1, StorageLevel(disk=true, memory=true, > offheap=false, deserialized=true, replication=1), WholeStageCodegen, None > at > org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) > at > org.apache.spark.sql.execution.exchange.ShuffleExchange.doExecute(ShuffleExchange.scala:113) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:233) > at > org.apache.spark.sql.execution.aggregate.TungstenAggregate.inputRDDs(TungstenAggregate.scala:134) > at > org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:348) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) > at > org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) > at > org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) > at > org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) > at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) > at > org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:240) > at > org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:287) > at > org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2122) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) > at
[jira] [Commented] (SPARK-15390) Memory management issue in complex DataFrame join and filter
[ https://issues.apache.org/jira/browse/SPARK-15390?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15513206#comment-15513206 ] Iulian Dragos commented on SPARK-15390: --- I'm still seeing a similar stack trace with the 2.0 release. {code} scala> res.count org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: Exchange SinglePartition +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#286L]) +- *Project +- *BroadcastHashJoin [userId#0L], [selUserId#169L], Inner, BuildRight :- *Project [userId#0L] : +- *Filter isnotnull(userId#0L) : +- *Scan avro [userId#0L] Format: com.databricks.spark.avro.DefaultSource@451b7faf, InputPaths: file:/Users/dragos/workspace/consulting/teralytics/11-000.avro, PushedFilters: [IsNotNull(userId)], ReadSchema: struct +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, bigint, true])) +- *GlobalLimit 1 +- Exchange SinglePartition +- *LocalLimit 1 +- *Project [userId#0L AS selUserId#169L] +- *Filter isnotnull(userId#0L) +- *Scan avro [userId#0L] Format: com.databricks.spark.avro.DefaultSource@451b7faf, InputPaths: file:/Users/dragos/workspace/consulting/teralytics/11-000.avro, PushedFilters: [IsNotNull(userId)], ReadSchema: struct at org.apache.spark.sql.catalyst.errors.package$.attachTree(package.scala:50) at org.apache.spark.sql.execution.exchange.ShuffleExchange.doExecute(ShuffleExchange.scala:113) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) at org.apache.spark.sql.execution.InputAdapter.inputRDDs(WholeStageCodegenExec.scala:233) at org.apache.spark.sql.execution.aggregate.HashAggregateExec.inputRDDs(HashAggregateExec.scala:138) at org.apache.spark.sql.execution.WholeStageCodegenExec.doExecute(WholeStageCodegenExec.scala:361) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:115) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:114) at org.apache.spark.sql.execution.SparkPlan.getByteArrayRdd(SparkPlan.scala:240) at org.apache.spark.sql.execution.SparkPlan.executeCollect(SparkPlan.scala:287) at org.apache.spark.sql.Dataset$$anonfun$org$apache$spark$sql$Dataset$$execute$1$1.apply(Dataset.scala:2183) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:57) at org.apache.spark.sql.Dataset.withNewExecutionId(Dataset.scala:2532) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$execute$1(Dataset.scala:2182) at org.apache.spark.sql.Dataset.org$apache$spark$sql$Dataset$$collect(Dataset.scala:2189) at org.apache.spark.sql.Dataset$$anonfun$count$1.apply(Dataset.scala:2217) at org.apache.spark.sql.Dataset$$anonfun$count$1.apply(Dataset.scala:2216) at org.apache.spark.sql.Dataset.withCallback(Dataset.scala:2545) at org.apache.spark.sql.Dataset.count(Dataset.scala:2216) ... 50 elided Caused by: org.apache.spark.SparkException: Exception thrown in awaitResult: at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:194) at org.apache.spark.sql.execution.exchange.BroadcastExchangeExec.doExecuteBroadcast(BroadcastExchangeExec.scala:120) at org.apache.spark.sql.execution.InputAdapter.doExecuteBroadcast(WholeStageCodegenExec.scala:229) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:125) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeBroadcast$1.apply(SparkPlan.scala:125) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:136) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:133) at org.apache.spark.sql.execution.SparkPlan.executeBroadcast(SparkPlan.scala:124) at org.apache.spark.sql.execution.joins.BroadcastHashJoinExec.prepareBroadcast(BroadcastHashJoinExec.scala:98) at
[jira] [Commented] (SPARK-17109) When we serialize UserDefinedGenerator to json, scala reflection throws an error
[ https://issues.apache.org/jira/browse/SPARK-17109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15425082#comment-15425082 ] Iulian Dragos commented on SPARK-17109: --- So the problem is somewhere along the caller list :) There might be a workaround for 2.10 if you find the exact method that has different results. > When we serialize UserDefinedGenerator to json, scala reflection throws an > error > > > Key: SPARK-17109 > URL: https://issues.apache.org/jira/browse/SPARK-17109 > Project: Spark > Issue Type: Bug > Components: SQL >Reporter: Wenchen Fan > > {code} > [info] - simple explode *** FAILED *** (19 milliseconds) > [info] Failed to parse logical plan to JSON: > [info] Project [word#128] > [info] +- Generate UserDefinedGenerator(words#126), true, false, None, > [word#128] > [info] +- Project [_1#125 AS words#126] > [info] +- LocalRelation [_1#125], [[a b c],[d e]] > (QueryTest.scala:214) > [info] org.scalatest.exceptions.TestFailedException: > [info] at > org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:496) > [info] at > org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) > [info] at org.scalatest.Assertions$class.fail(Assertions.scala:1348) > [info] at org.scalatest.FunSuite.fail(FunSuite.scala:1555) > [info] at > org.apache.spark.sql.QueryTest.checkJsonFormat(QueryTest.scala:214) > [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:137) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply$mcV$sp(DataFrameSuite.scala:122) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply(DataFrameSuite.scala:119) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply(DataFrameSuite.scala:119) > [info] at > org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) > [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) > [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) > [info] at org.scalatest.Transformer.apply(Transformer.scala:22) > [info] at org.scalatest.Transformer.apply(Transformer.scala:20) > [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) > [info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42) > [info] at > org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) > [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) > [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) > [info] at org.scalatest.FunSuite.runTest(FunSuite.scala:1555) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) > [info] at > org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) > [info] at > org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) > [info] at scala.collection.immutable.List.foreach(List.scala:318) > [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) > [info] at > org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) > [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) > [info] at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) > [info] at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) > [info] at org.scalatest.Suite$class.run(Suite.scala:1424) > [info] at > org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) > [info] at > org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) > [info] at > org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) > [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:545) > [info] at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) > [info] at > org.apache.spark.sql.DataFrameSuite.org$scalatest$BeforeAndAfterAll$$super$run(DataFrameSuite.scala:36) > [info] at > org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) > [info] at > org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) > [info] at org.apache.spark.sql.DataFrameSuite.run(DataFrameSuite.scala:36) > [info] at > org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) > [info] at > org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) > [info] at
[jira] [Commented] (SPARK-17109) When we serialize UserDefinedGenerator to json, scala reflection throws an error
[ https://issues.apache.org/jira/browse/SPARK-17109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15424979#comment-15424979 ] Iulian Dragos commented on SPARK-17109: --- No, it seems strange. I suggest you print out the `tpe` when it arrives in this method and see if it's the same in 2.10 and 2.11. The error may happen earlier, but it blows up here. > When we serialize UserDefinedGenerator to json, scala reflection throws an > error > > > Key: SPARK-17109 > URL: https://issues.apache.org/jira/browse/SPARK-17109 > Project: Spark > Issue Type: Bug > Components: SQL >Reporter: Wenchen Fan > > {code} > [info] - simple explode *** FAILED *** (19 milliseconds) > [info] Failed to parse logical plan to JSON: > [info] Project [word#128] > [info] +- Generate UserDefinedGenerator(words#126), true, false, None, > [word#128] > [info] +- Project [_1#125 AS words#126] > [info] +- LocalRelation [_1#125], [[a b c],[d e]] > (QueryTest.scala:214) > [info] org.scalatest.exceptions.TestFailedException: > [info] at > org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:496) > [info] at > org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) > [info] at org.scalatest.Assertions$class.fail(Assertions.scala:1348) > [info] at org.scalatest.FunSuite.fail(FunSuite.scala:1555) > [info] at > org.apache.spark.sql.QueryTest.checkJsonFormat(QueryTest.scala:214) > [info] at org.apache.spark.sql.QueryTest.checkAnswer(QueryTest.scala:137) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply$mcV$sp(DataFrameSuite.scala:122) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply(DataFrameSuite.scala:119) > [info] at > org.apache.spark.sql.DataFrameSuite$$anonfun$10.apply(DataFrameSuite.scala:119) > [info] at > org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) > [info] at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) > [info] at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) > [info] at org.scalatest.Transformer.apply(Transformer.scala:22) > [info] at org.scalatest.Transformer.apply(Transformer.scala:20) > [info] at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) > [info] at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42) > [info] at > org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) > [info] at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) > [info] at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) > [info] at org.scalatest.FunSuite.runTest(FunSuite.scala:1555) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) > [info] at > org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) > [info] at > org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) > [info] at > org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) > [info] at scala.collection.immutable.List.foreach(List.scala:318) > [info] at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) > [info] at > org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) > [info] at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) > [info] at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) > [info] at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) > [info] at org.scalatest.Suite$class.run(Suite.scala:1424) > [info] at > org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) > [info] at > org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) > [info] at > org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) > [info] at org.scalatest.SuperEngine.runImpl(Engine.scala:545) > [info] at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) > [info] at > org.apache.spark.sql.DataFrameSuite.org$scalatest$BeforeAndAfterAll$$super$run(DataFrameSuite.scala:36) > [info] at > org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) > [info] at > org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) > [info] at org.apache.spark.sql.DataFrameSuite.run(DataFrameSuite.scala:36) > [info] at > org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) > [info] at > org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) > [info] at
[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15159111#comment-15159111 ] Iulian Dragos commented on SPARK-13431: --- I think the only way is to revert the offending changes, as mentioned [here|https://issues.apache.org/jira/browse/SPARK-13431?focusedCommentId=15157182=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15157182] > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15157213#comment-15157213 ] Iulian Dragos commented on SPARK-13431: --- Sbt assembly is using a different library, I think it's called jarjar. Nothing to do with the maven shade plugin. > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15157119#comment-15157119 ] Iulian Dragos commented on SPARK-13431: --- Probably the easiest fix is to break some grammar elements in more files. I *think* the problem is deeper though, and there's some state explosion generated by an ambiguous part of the grammar. The Scala compiler would show an error, but the ANTLR generator spits out Java files. Definitely those methods are very close to the limit, I saw javac failing when using the Eclipse compiler, but the Oracle one was slightly better. > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15157044#comment-15157044 ] Iulian Dragos commented on SPARK-13431: --- Well, the class file format isn't embedding any names inside the code section, that's what the ConstantPool is for. What the shade plugin does or not is slightly irrelevant, the fact is that the failure happens (only) during shading, and all "compile" integration builds in Jenkins are passing without problem. Only the ones that run tests (and run `assembly`) fail. > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Comment Edited] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15156902#comment-15156902 ] Iulian Dragos edited comment on SPARK-13431 at 2/22/16 1:29 PM: In my experience the 'shade' plugin does a lot more than it says (constant propagation, control-flow optimizations). I didn't find a way to turn them off. ... and I don't know if it's only the last one. The first Jenkins build that failed had only this commit in the list of changes. was (Author: dragos): In my experience the 'shade' plugin does a lot more than it says (constant propagation, control-flow optimizations). I didn't find a way to turn them off. > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Commented] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15156902#comment-15156902 ] Iulian Dragos commented on SPARK-13431: --- In my experience the 'shade' plugin does a lot more than it says (constant propagation, control-flow optimizations). I didn't find a way to turn them off. > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Updated] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-13431: -- Description: Cannot build the project when run the normal build commands: eg. {code} build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package ./make-distribution.sh --name test --tgz -Phadoop-2.6 {code} Integration builds are also failing: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console It looks like this is the commit that introduced the issue: https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 was: Cannot build the project when run the normal build commands: eg. {code} build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package ./make-distribution.sh --name test --tgz -Phadoop-2.6 {code} Check bellow the failure when shading: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console This is blocking i think... > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Integration builds are also failing: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > It looks like this is the commit that introduced the issue: > https://github.com/apache/spark/commit/7925071280bfa1570435bde3e93492eaf2167d56 -- 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
[jira] [Updated] (SPARK-13431) Maven build fails due to: Method code too large! in Catalyst
[ https://issues.apache.org/jira/browse/SPARK-13431?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-13431: -- Description: Cannot build the project when run the normal build commands: eg. {code} build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package ./make-distribution.sh --name test --tgz -Phadoop-2.6 {code} Check bellow the failure when shading: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console This is blocking i think... was: Cannot build the project when run the normal build commands: eg. build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package ./make-distribution.sh --name test --tgz -Phadoop-2.6 Check bellow the failure when shading: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console This is blocking i think... > Maven build fails due to: Method code too large! in Catalyst > > > Key: SPARK-13431 > URL: https://issues.apache.org/jira/browse/SPARK-13431 > Project: Spark > Issue Type: Bug > Components: Build, Optimizer >Affects Versions: 2.0.0 >Reporter: Stavros Kontopoulos >Priority: Blocker > > Cannot build the project when run the normal build commands: > eg. > {code} > build/mvn -Phadoop-2.6 -Dhadoop.version=2.6.0 clean package > ./make-distribution.sh --name test --tgz -Phadoop-2.6 > {code} > Check bellow the failure when shading: > https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-maven-hadoop-2.6/229/console > https://ci.typesafe.com/job/mit-docker-test-zk-ref/12/console > This is blocking i think... -- 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
[jira] [Commented] (SPARK-12430) Temporary folders do not get deleted after Task completes causing problems with disk space.
[ https://issues.apache.org/jira/browse/SPARK-12430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15133887#comment-15133887 ] Iulian Dragos commented on SPARK-12430: --- The PR I linked to was just merged. Normally this should fix the race condition, so please give it a try. Regarding the reasons behind moving the directory out of {{spark-id...}}, it's all in that comment, but here are some pointers: - a well-behaved framework should only store things in the Mesos sandbox - the spark temporary directory is deleted on shutdown (using a shutdown hook, a VM-level callback), including everything underneath, recursively - when the external shuffle service is enabled the shuffle files should not be deleted, even after the executor exits. That's because the (external) shuffle service reads and serves them to other executors. The executor may exit early due to dynamic allocation. So shuffle files are moved out of there - when dynamic allocation is disabled, shuffle files are deleted as part of the standard shutdown procedure (*not* the VM-level shutdown hook). This part seems flaky, and what the PR I linked to is fixing (Mesos apparently kills the executors if the driver exits first). We might move those files under {{spark-id}} when the external shuffle server is disabled, but it seemed simpler to put them in the same place all the time, but delete/keep them depending on this flag. > Temporary folders do not get deleted after Task completes causing problems > with disk space. > --- > > Key: SPARK-12430 > URL: https://issues.apache.org/jira/browse/SPARK-12430 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.5.1, 1.5.2, 1.6.0 > Environment: Ubuntu server >Reporter: Fede Bar > > We are experiencing an issue with automatic /tmp folder deletion after > framework completes. Completing a M/R job using Spark 1.5.2 (same behavior as > Spark 1.5.1) over Mesos will not delete some temporary folders causing free > disk space on server to exhaust. > Behavior of M/R job using Spark 1.4.1 over Mesos cluster: > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/slaves/id#* , */tmp/spark-#/* , > */tmp/spark-#/blockmgr-#* > - When task is completed */tmp/spark-#/* gets deleted along with > */tmp/spark-#/blockmgr-#* sub-folder. > Behavior of M/R job using Spark 1.5.2 over Mesos cluster (same identical job): > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/mesos/slaves/id** * , > */tmp/spark-***/ * ,{color:red} /tmp/blockmgr-***{color} > - When task is completed */tmp/spark-***/ * gets deleted but NOT shuffle > container folder {color:red} /tmp/blockmgr-***{color} > Unfortunately, {color:red} /tmp/blockmgr-***{color} can account for several > GB depending on the job that ran. Over time this causes disk space to become > full with consequences that we all know. > Running a shell script would probably work but it is difficult to identify > folders in use by a running M/R or stale folders. I did notice similar issues > opened by other users marked as "resolved", but none seems to exactly match > the above behavior. > I really hope someone has insights on how to fix it. > Thank you very much! -- 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
[jira] [Commented] (SPARK-12430) Temporary folders do not get deleted after Task completes causing problems with disk space.
[ https://issues.apache.org/jira/browse/SPARK-12430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15132341#comment-15132341 ] Iulian Dragos commented on SPARK-12430: --- There is a long discussion about this in the dynamic allocation PR, it may be good to go through that before you open the PR. [This comment|https://github.com/apache/spark/pull/4984#discussion_r33714930] is a good start. Roughly, dynamic allocation needs those files *after* the executor exits. Are you running with the external shuffle service enabled? > Temporary folders do not get deleted after Task completes causing problems > with disk space. > --- > > Key: SPARK-12430 > URL: https://issues.apache.org/jira/browse/SPARK-12430 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.5.1, 1.5.2, 1.6.0 > Environment: Ubuntu server >Reporter: Fede Bar > > We are experiencing an issue with automatic /tmp folder deletion after > framework completes. Completing a M/R job using Spark 1.5.2 (same behavior as > Spark 1.5.1) over Mesos will not delete some temporary folders causing free > disk space on server to exhaust. > Behavior of M/R job using Spark 1.4.1 over Mesos cluster: > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/slaves/id#* , */tmp/spark-#/* , > */tmp/spark-#/blockmgr-#* > - When task is completed */tmp/spark-#/* gets deleted along with > */tmp/spark-#/blockmgr-#* sub-folder. > Behavior of M/R job using Spark 1.5.2 over Mesos cluster (same identical job): > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/mesos/slaves/id** * , > */tmp/spark-***/ * ,{color:red} /tmp/blockmgr-***{color} > - When task is completed */tmp/spark-***/ * gets deleted but NOT shuffle > container folder {color:red} /tmp/blockmgr-***{color} > Unfortunately, {color:red} /tmp/blockmgr-***{color} can account for several > GB depending on the job that ran. Over time this causes disk space to become > full with consequences that we all know. > Running a shell script would probably work but it is difficult to identify > folders in use by a running M/R or stale folders. I did notice similar issues > opened by other users marked as "resolved", but none seems to exactly match > the above behavior. > I really hope someone has insights on how to fix it. > Thank you very much! -- 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
[jira] [Commented] (SPARK-12430) Temporary folders do not get deleted after Task completes causing problems with disk space.
[ https://issues.apache.org/jira/browse/SPARK-12430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15132640#comment-15132640 ] Iulian Dragos commented on SPARK-12430: --- Are you using 1.6? In that case, the blockmgr directory should really be inside your Mesos sandbox, not under {{/tmp}}. At least, that's what I see when I try out. Note that, in case the external shuffle service isn't running that directory should be deleted *even under the Mesos sandbox*. In my tests I see that it's not always deleted, so I think there's a race condition somewhere. There's a PR that would probably fix that: https://github.com/apache/spark/pull/10319 > Temporary folders do not get deleted after Task completes causing problems > with disk space. > --- > > Key: SPARK-12430 > URL: https://issues.apache.org/jira/browse/SPARK-12430 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.5.1, 1.5.2, 1.6.0 > Environment: Ubuntu server >Reporter: Fede Bar > > We are experiencing an issue with automatic /tmp folder deletion after > framework completes. Completing a M/R job using Spark 1.5.2 (same behavior as > Spark 1.5.1) over Mesos will not delete some temporary folders causing free > disk space on server to exhaust. > Behavior of M/R job using Spark 1.4.1 over Mesos cluster: > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/slaves/id#* , */tmp/spark-#/* , > */tmp/spark-#/blockmgr-#* > - When task is completed */tmp/spark-#/* gets deleted along with > */tmp/spark-#/blockmgr-#* sub-folder. > Behavior of M/R job using Spark 1.5.2 over Mesos cluster (same identical job): > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/mesos/slaves/id** * , > */tmp/spark-***/ * ,{color:red} /tmp/blockmgr-***{color} > - When task is completed */tmp/spark-***/ * gets deleted but NOT shuffle > container folder {color:red} /tmp/blockmgr-***{color} > Unfortunately, {color:red} /tmp/blockmgr-***{color} can account for several > GB depending on the job that ran. Over time this causes disk space to become > full with consequences that we all know. > Running a shell script would probably work but it is difficult to identify > folders in use by a running M/R or stale folders. I did notice similar issues > opened by other users marked as "resolved", but none seems to exactly match > the above behavior. > I really hope someone has insights on how to fix it. > Thank you very much! -- 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
[jira] [Closed] (SPARK-13159) External shuffle service broken w/ Mesos
[ https://issues.apache.org/jira/browse/SPARK-13159?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos closed SPARK-13159. - Resolution: Duplicate > External shuffle service broken w/ Mesos > > > Key: SPARK-13159 > URL: https://issues.apache.org/jira/browse/SPARK-13159 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 2.0.0 >Reporter: Iulian Dragos > > Dynamic allocation and external shuffle service won't work together on Mesos > for applications longer than {{spark.network.timeout}}. > After two minutes (default value for {{spark.network.timeout}}), I see a lot > of FileNotFoundExceptions and spark jobs just fail. > {code} > 16/02/03 15:26:51 WARN TaskSetManager: Lost task 728.0 in stage 3.0 (TID > 2755, 10.0.1.208): java.io.FileNotFoundException: > /tmp/blockmgr-ea5b2392-626a-4278-8ae3-fb2c4262d758/02/shuffle_1_728_0.data.57efd66e-7662-4810-a5b1-56d7e2d7a9f0 > (No such file or directory) > at java.io.FileOutputStream.open(Native Method) > at java.io.FileOutputStream.(FileOutputStream.java:221) > at > org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:88) > at > org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:181) > at > org.apache.spark.util.collection.WritablePartitionedPairCollection$$anon$1.writeNext(WritablePartitionedPairCollection.scala:56) > at > org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:661) > at > org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:71) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:77) > ... > {code} > h3. Analysis > The Mesos external shuffle service needs a way to know when it's safe to > delete shuffle files for a given application. The current solution (that > seemed to work fine while the RPC transport was based on Akka) was to open a > TCP connection between the driver and each external shuffle service. Once the > driver went down (graciously or crashed), the shuffle service would > eventually get a notification from the network layer, and delete the > corresponding files. > This solution stopped working because it relies on an idle connection, and > the new Netty-based RPC layer is closing the connection after > {{spark.network.timeout}}. -- 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
[jira] [Created] (SPARK-13159) External shuffle service broken w/ Mesos
Iulian Dragos created SPARK-13159: - Summary: External shuffle service broken w/ Mesos Key: SPARK-13159 URL: https://issues.apache.org/jira/browse/SPARK-13159 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 2.0.0 Reporter: Iulian Dragos Dynamic allocation and external shuffle service won't work together on Mesos for applications longer than {{spark.network.timeout}}. After two minutes (default value for {{spark.network.timeout}}), I see a lot of FileNotFoundExceptions and spark jobs just fail. {code} 16/02/03 15:26:51 WARN TaskSetManager: Lost task 728.0 in stage 3.0 (TID 2755, 10.0.1.208): java.io.FileNotFoundException: /tmp/blockmgr-ea5b2392-626a-4278-8ae3-fb2c4262d758/02/shuffle_1_728_0.data.57efd66e-7662-4810-a5b1-56d7e2d7a9f0 (No such file or directory) at java.io.FileOutputStream.open(Native Method) at java.io.FileOutputStream.(FileOutputStream.java:221) at org.apache.spark.storage.DiskBlockObjectWriter.open(DiskBlockObjectWriter.scala:88) at org.apache.spark.storage.DiskBlockObjectWriter.write(DiskBlockObjectWriter.scala:181) at org.apache.spark.util.collection.WritablePartitionedPairCollection$$anon$1.writeNext(WritablePartitionedPairCollection.scala:56) at org.apache.spark.util.collection.ExternalSorter.writePartitionedFile(ExternalSorter.scala:661) at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:71) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:77) ... {code} h3. Analysis The Mesos external shuffle service needs a way to know when it's safe to delete shuffle files for a given application. The current solution (that seemed to work fine while the RPC transport was based on Akka) was to open a TCP connection between the driver and each external shuffle service. Once the driver went down (graciously or crashed), the shuffle service would eventually get a notification from the network layer, and delete the corresponding files. This solution stopped working because it relies on an idle connection, and the new Netty-based RPC layer is closing the connection after {{spark.network.timeout}}. -- 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
[jira] [Commented] (SPARK-12430) Temporary folders do not get deleted after Task completes causing problems with disk space.
[ https://issues.apache.org/jira/browse/SPARK-12430?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15130554#comment-15130554 ] Iulian Dragos commented on SPARK-12430: --- I guess the thinking was that Mesos would clean up those sandboxes. If you're experiencing disk space issues, I think it would make sense to tweak Mesos: {quote} Garbage collection is scheduled based on the --gc_delay agent flag. By default, this is one week since the sandbox was last modified. After the delay, the files are deleted. {quote} https://mesos.apache.org/documentation/latest/sandbox/ > Temporary folders do not get deleted after Task completes causing problems > with disk space. > --- > > Key: SPARK-12430 > URL: https://issues.apache.org/jira/browse/SPARK-12430 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.5.1, 1.5.2, 1.6.0 > Environment: Ubuntu server >Reporter: Fede Bar > > We are experiencing an issue with automatic /tmp folder deletion after > framework completes. Completing a M/R job using Spark 1.5.2 (same behavior as > Spark 1.5.1) over Mesos will not delete some temporary folders causing free > disk space on server to exhaust. > Behavior of M/R job using Spark 1.4.1 over Mesos cluster: > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/slaves/id#* , */tmp/spark-#/* , > */tmp/spark-#/blockmgr-#* > - When task is completed */tmp/spark-#/* gets deleted along with > */tmp/spark-#/blockmgr-#* sub-folder. > Behavior of M/R job using Spark 1.5.2 over Mesos cluster (same identical job): > - Launched using spark-submit on one cluster node. > - Following folders are created: */tmp/mesos/mesos/slaves/id** * , > */tmp/spark-***/ * ,{color:red} /tmp/blockmgr-***{color} > - When task is completed */tmp/spark-***/ * gets deleted but NOT shuffle > container folder {color:red} /tmp/blockmgr-***{color} > Unfortunately, {color:red} /tmp/blockmgr-***{color} can account for several > GB depending on the job that ran. Over time this causes disk space to become > full with consequences that we all know. > Running a shell script would probably work but it is difficult to identify > folders in use by a running M/R or stale folders. I did notice similar issues > opened by other users marked as "resolved", but none seems to exactly match > the above behavior. > I really hope someone has insights on how to fix it. > Thank you very much! -- 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
[jira] [Updated] (SPARK-12423) Mesos executor home should not be resolved on the driver's file system
[ https://issues.apache.org/jira/browse/SPARK-12423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-12423: -- Fix Version/s: 2.0.0 > Mesos executor home should not be resolved on the driver's file system > -- > > Key: SPARK-12423 > URL: https://issues.apache.org/jira/browse/SPARK-12423 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Iulian Dragos > Fix For: 2.0.0 > > > {{spark.mesos.executor.home}} should be an uninterpreted string. It is very > possible that this path does not exist on the driver, and if it does, it may > be a symlink that should not be resolved. Currently, this leads to failures > in client mode. > For example, setting it to {{/var/spark/spark-1.6.0-bin-hadoop2.6/}} leads to > executors failing: > {code} > sh: 1: /private/var/spark/spark-1.6.0-bin-hadoop2.6/bin/spark-class: not found > {code} > {{getCanonicalPath}} transforms {{/var/spark...}} into {{/private/var..}} > because on my system there is a symlink from one to the other. -- 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
[jira] [Resolved] (SPARK-12423) Mesos executor home should not be resolved on the driver's file system
[ https://issues.apache.org/jira/browse/SPARK-12423?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos resolved SPARK-12423. --- Resolution: Fixed > Mesos executor home should not be resolved on the driver's file system > -- > > Key: SPARK-12423 > URL: https://issues.apache.org/jira/browse/SPARK-12423 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Iulian Dragos > > {{spark.mesos.executor.home}} should be an uninterpreted string. It is very > possible that this path does not exist on the driver, and if it does, it may > be a symlink that should not be resolved. Currently, this leads to failures > in client mode. > For example, setting it to {{/var/spark/spark-1.6.0-bin-hadoop2.6/}} leads to > executors failing: > {code} > sh: 1: /private/var/spark/spark-1.6.0-bin-hadoop2.6/bin/spark-class: not found > {code} > {{getCanonicalPath}} transforms {{/var/spark...}} into {{/private/var..}} > because on my system there is a symlink from one to the other. -- 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
[jira] [Commented] (SPARK-12423) Mesos executor home should not be resolved on the driver's file system
[ https://issues.apache.org/jira/browse/SPARK-12423?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15128036#comment-15128036 ] Iulian Dragos commented on SPARK-12423: --- Yes, closing this. > Mesos executor home should not be resolved on the driver's file system > -- > > Key: SPARK-12423 > URL: https://issues.apache.org/jira/browse/SPARK-12423 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Iulian Dragos > Fix For: 2.0.0 > > > {{spark.mesos.executor.home}} should be an uninterpreted string. It is very > possible that this path does not exist on the driver, and if it does, it may > be a symlink that should not be resolved. Currently, this leads to failures > in client mode. > For example, setting it to {{/var/spark/spark-1.6.0-bin-hadoop2.6/}} leads to > executors failing: > {code} > sh: 1: /private/var/spark/spark-1.6.0-bin-hadoop2.6/bin/spark-class: not found > {code} > {{getCanonicalPath}} transforms {{/var/spark...}} into {{/private/var..}} > because on my system there is a symlink from one to the other. -- 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
[jira] [Created] (SPARK-13086) Spark Shell for 2.11 does not allow loading files via '-i'
Iulian Dragos created SPARK-13086: - Summary: Spark Shell for 2.11 does not allow loading files via '-i' Key: SPARK-13086 URL: https://issues.apache.org/jira/browse/SPARK-13086 Project: Spark Issue Type: Bug Components: Spark Shell Affects Versions: 1.6.0 Reporter: Iulian Dragos Spark Shell for 2.10 allows one to load files via '-i': {code} spark-shell -i file1.scala {code} the same results in an error in the 2.11 version of Spark: {code} $ bin/spark-shell -i test.scala bad option: '-i' {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
[jira] [Created] (SPARK-12979) Paths are resolved relative to the local file system
Iulian Dragos created SPARK-12979: - Summary: Paths are resolved relative to the local file system Key: SPARK-12979 URL: https://issues.apache.org/jira/browse/SPARK-12979 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.6.0 Reporter: Iulian Dragos Spark properties that refer to paths on the cluster (for example, `spark.mesos.executor.home`) should be un-interpreted strings. Currently, such a path is resolved relative to the local (client) file system, and symlinks are resolved, etc. (by calling `getCanonicalPath`). -- 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
[jira] [Commented] (SPARK-11857) Remove Mesos fine-grained mode subject to discussions
[ https://issues.apache.org/jira/browse/SPARK-11857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15098355#comment-15098355 ] Iulian Dragos commented on SPARK-11857: --- Sure, but that seems to be done already... by you: https://github.com/apache/spark/pull/9795 > Remove Mesos fine-grained mode subject to discussions > - > > Key: SPARK-11857 > URL: https://issues.apache.org/jira/browse/SPARK-11857 > Project: Spark > Issue Type: Sub-task > Components: Mesos >Reporter: Reynold Xin >Assignee: Reynold Xin > > See discussions in > http://apache-spark-developers-list.1001551.n3.nabble.com/Removing-the-Mesos-fine-grained-mode-td15277.html > and > http://apache-spark-developers-list.1001551.n3.nabble.com/Please-reply-if-you-use-Mesos-fine-grained-mode-td14930.html -- 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
[jira] [Comment Edited] (SPARK-11857) Remove Mesos fine-grained mode subject to discussions
[ https://issues.apache.org/jira/browse/SPARK-11857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15095975#comment-15095975 ] Iulian Dragos edited comment on SPARK-11857 at 1/13/16 10:35 AM: - There's one pending issue in the email thread, regarding a 20-30% performance drop by switching to the coarse-grained mode. Maybe it makes sense to just deprecate it as a first step, until we figure out the issue? PS: http://apache-spark-developers-list.1001551.n3.nabble.com/Removing-the-Mesos-fine-grained-mode-tp15277p15388.html was (Author: dragos): There's one pending issue in the email thread, regarding a 20-30% performance drop by switching to the coarse-grained mode. Maybe it makes sense to just deprecate it as a first step, until we figure out the issue? > Remove Mesos fine-grained mode subject to discussions > - > > Key: SPARK-11857 > URL: https://issues.apache.org/jira/browse/SPARK-11857 > Project: Spark > Issue Type: Sub-task > Components: Mesos >Reporter: Reynold Xin >Assignee: Reynold Xin > > See discussions in > http://apache-spark-developers-list.1001551.n3.nabble.com/Removing-the-Mesos-fine-grained-mode-td15277.html > and > http://apache-spark-developers-list.1001551.n3.nabble.com/Please-reply-if-you-use-Mesos-fine-grained-mode-td14930.html -- 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
[jira] [Commented] (SPARK-11857) Remove Mesos fine-grained mode subject to discussions
[ https://issues.apache.org/jira/browse/SPARK-11857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15095975#comment-15095975 ] Iulian Dragos commented on SPARK-11857: --- There's one pending issue in the email thread, regarding a 20-30% performance drop by switching to the coarse-grained mode. Maybe it makes sense to just deprecate it as a first step, until we figure out the issue? > Remove Mesos fine-grained mode subject to discussions > - > > Key: SPARK-11857 > URL: https://issues.apache.org/jira/browse/SPARK-11857 > Project: Spark > Issue Type: Sub-task > Components: Mesos >Reporter: Reynold Xin >Assignee: Reynold Xin > > See discussions in > http://apache-spark-developers-list.1001551.n3.nabble.com/Removing-the-Mesos-fine-grained-mode-td15277.html > and > http://apache-spark-developers-list.1001551.n3.nabble.com/Please-reply-if-you-use-Mesos-fine-grained-mode-td14930.html -- 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
[jira] [Created] (SPARK-12423) Mesos executor home should not be resolved on the driver's file system
Iulian Dragos created SPARK-12423: - Summary: Mesos executor home should not be resolved on the driver's file system Key: SPARK-12423 URL: https://issues.apache.org/jira/browse/SPARK-12423 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.6.0 Reporter: Iulian Dragos {{spark.mesos.executor.home}} should be an uninterpreted string. It is very possible that this path does not exist on the driver, and if it does, it may be a symlink that should not be resolved. Currently, this leads to failures in client mode. For example, setting it to {{/var/spark/spark-1.6.0-bin-hadoop2.6/}} leads to executors failing: {code} sh: 1: /private/var/spark/spark-1.6.0-bin-hadoop2.6/bin/spark-class: not found {code} {{getCanonicalPath}} transforms {{/var/spark...}} into {{/private/var..}} because on my system there is a symlink from one to the other. -- 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
[jira] [Commented] (SPARK-12345) Mesos cluster mode is broken
[ https://issues.apache.org/jira/browse/SPARK-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15060236#comment-15060236 ] Iulian Dragos commented on SPARK-12345: --- I'd prefer filtering it at at the submit side, if everything else works. Doing it in the scheduler will be confusing for users. The Driver environment would still show SPARK_HOME (in the dispatcher UI), but in fact it would be filtered out in practice. > Mesos cluster mode is broken > > > Key: SPARK-12345 > URL: https://issues.apache.org/jira/browse/SPARK-12345 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Andrew Or >Assignee: Apache Spark >Priority: Critical > > The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. > The driver is confused about where SPARK_HOME is. It resolves > `mesos.executor.uri` or `spark.mesos.executor.home` relative to the > filesystem where the driver runs, which is wrong. > {code} > I1215 15:00:39.411212 28032 exec.cpp:134] Version: 0.25.0 > I1215 15:00:39.413512 28037 exec.cpp:208] Executor registered on slave > 130bdc39-44e7-4256-8c22-602040d337f1-S1 > bin/spark-submit: line 27: > /Users/dragos/workspace/Spark/dev/rc-tests/spark-1.6.0-bin-hadoop2.6/bin/spark-class: > No such file or directory > {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
[jira] [Commented] (SPARK-12345) Mesos cluster mode is broken
[ https://issues.apache.org/jira/browse/SPARK-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15059717#comment-15059717 ] Iulian Dragos commented on SPARK-12345: --- There's isn't any {{SPARK_HOME}} set on any of the Mesos slaves. Here's what I think happens: the {{SPARK_HOME}} variable is exported by {{spark-submit}}, and copied [here|https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala#L369-L372] to the driver environment. > Mesos cluster mode is broken > > > Key: SPARK-12345 > URL: https://issues.apache.org/jira/browse/SPARK-12345 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Andrew Or >Assignee: Apache Spark >Priority: Critical > > The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. > The driver is confused about where SPARK_HOME is. It resolves > `mesos.executor.uri` or `spark.mesos.executor.home` relative to the > filesystem where the driver runs, which is wrong. > {code} > I1215 15:00:39.411212 28032 exec.cpp:134] Version: 0.25.0 > I1215 15:00:39.413512 28037 exec.cpp:208] Executor registered on slave > 130bdc39-44e7-4256-8c22-602040d337f1-S1 > bin/spark-submit: line 27: > /Users/dragos/workspace/Spark/dev/rc-tests/spark-1.6.0-bin-hadoop2.6/bin/spark-class: > No such file or directory > {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
[jira] [Commented] (SPARK-12345) Mesos cluster mode is broken
[ https://issues.apache.org/jira/browse/SPARK-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15058873#comment-15058873 ] Iulian Dragos commented on SPARK-12345: --- [~skonto] pointed out this commit: https://github.com/apache/spark/commit/8aff36e91de0fee2f3f56c6d240bb203b5bb48ba it could be part of the problem. > Mesos cluster mode is broken > > > Key: SPARK-12345 > URL: https://issues.apache.org/jira/browse/SPARK-12345 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Andrew Or >Assignee: Apache Spark >Priority: Critical > > The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. > The driver is confused about where SPARK_HOME is. It resolves > `mesos.executor.uri` or `spark.mesos.executor.home` relative to the > filesystem where the driver runs, which is wrong. > {code} > I1215 15:00:39.411212 28032 exec.cpp:134] Version: 0.25.0 > I1215 15:00:39.413512 28037 exec.cpp:208] Executor registered on slave > 130bdc39-44e7-4256-8c22-602040d337f1-S1 > bin/spark-submit: line 27: > /Users/dragos/workspace/Spark/dev/rc-tests/spark-1.6.0-bin-hadoop2.6/bin/spark-class: > No such file or directory > {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
[jira] [Updated] (SPARK-12345) Mesos cluster mode is broken
[ https://issues.apache.org/jira/browse/SPARK-12345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-12345: -- Description: The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. The driver is confused about where SPARK_HOME is. It resolves `mesos.executor.uri` or `spark.mesos.executor.home` relative to the filesystem where the driver runs, which is wrong. {code} I1215 15:00:39.411212 28032 exec.cpp:134] Version: 0.25.0 I1215 15:00:39.413512 28037 exec.cpp:208] Executor registered on slave 130bdc39-44e7-4256-8c22-602040d337f1-S1 bin/spark-submit: line 27: /Users/dragos/workspace/Spark/dev/rc-tests/spark-1.6.0-bin-hadoop2.6/bin/spark-class: No such file or directory {code} was: The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. (Iulian: please edit this to provide more detail) > Mesos cluster mode is broken > > > Key: SPARK-12345 > URL: https://issues.apache.org/jira/browse/SPARK-12345 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.6.0 >Reporter: Andrew Or >Assignee: Iulian Dragos >Priority: Critical > > The same setup worked in 1.5.2 but is now failing for 1.6.0-RC2. > The driver is confused about where SPARK_HOME is. It resolves > `mesos.executor.uri` or `spark.mesos.executor.home` relative to the > filesystem where the driver runs, which is wrong. > {code} > I1215 15:00:39.411212 28032 exec.cpp:134] Version: 0.25.0 > I1215 15:00:39.413512 28037 exec.cpp:208] Executor registered on slave > 130bdc39-44e7-4256-8c22-602040d337f1-S1 > bin/spark-submit: line 27: > /Users/dragos/workspace/Spark/dev/rc-tests/spark-1.6.0-bin-hadoop2.6/bin/spark-class: > No such file or directory > {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
[jira] [Created] (SPARK-12265) Spark calls System.exit inside driver instead of throwing exception
Iulian Dragos created SPARK-12265: - Summary: Spark calls System.exit inside driver instead of throwing exception Key: SPARK-12265 URL: https://issues.apache.org/jira/browse/SPARK-12265 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.6.0 Reporter: Iulian Dragos Spark may call {{System.exit}} if Mesos sends an error code back to the MesosSchedulerDriver. This makes Spark very hard to test, since this effectively kills the driver application under test. Such tests may run under ScalaTest, that doesn't get a chance to collect a result and populate a report. Relevant code is in MesosSchedulerUtils.scala: {code} val ret = mesosDriver.run() logInfo("driver.run() returned with code " + ret) if (ret != null && ret.equals(Status.DRIVER_ABORTED)) { System.exit(1) } {code} Errors should be signaled with a {{SparkException}} in the correct thread. -- 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
[jira] [Created] (SPARK-12202) Pass additional Scala REPL options to the underlying REPL
Iulian Dragos created SPARK-12202: - Summary: Pass additional Scala REPL options to the underlying REPL Key: SPARK-12202 URL: https://issues.apache.org/jira/browse/SPARK-12202 Project: Spark Issue Type: Improvement Components: Spark Shell Reporter: Iulian Dragos Sometimes it is useful to be able to pass Scala options to the underlying Spark REPL (like {{-target}} or {{-Xprint:parse}} when debugging). A simple way is to pass them through an additional environment variable {{SPARK_REPL_OPTS}}. -- 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
[jira] [Updated] (SPARK-12202) Pass additional Scala REPL options to the underlying REPL (2.11)
[ https://issues.apache.org/jira/browse/SPARK-12202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-12202: -- Description: Sometimes it is useful to be able to pass Scala options to the underlying Spark REPL (like {{-target}} or {{-Xprint:parse}} when debugging). Currently, only the 2.10 version of the REPL allows that (as normal arguments to the spark-shell command) was: Sometimes it is useful to be able to pass Scala options to the underlying Spark REPL (like {{-target}} or {{-Xprint:parse}} when debugging). A simple way is to pass them through an additional environment variable {{SPARK_REPL_OPTS}}. Summary: Pass additional Scala REPL options to the underlying REPL (2.11) (was: Pass additional Scala REPL options to the underlying REPL) > Pass additional Scala REPL options to the underlying REPL (2.11) > > > Key: SPARK-12202 > URL: https://issues.apache.org/jira/browse/SPARK-12202 > Project: Spark > Issue Type: Improvement > Components: Spark Shell >Reporter: Iulian Dragos > > Sometimes it is useful to be able to pass Scala options to the underlying > Spark REPL (like {{-target}} or {{-Xprint:parse}} when debugging). > Currently, only the 2.10 version of the REPL allows that (as normal arguments > to the spark-shell command) -- 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
[jira] [Commented] (SPARK-7286) Precedence of operator not behaving properly
[ https://issues.apache.org/jira/browse/SPARK-7286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15015418#comment-15015418 ] Iulian Dragos commented on SPARK-7286: -- I agree with Jakob's analysis. For reference, Slick uses {{=!=}}, which wouldn't be a bad option and gives you the same precedence. > Precedence of operator not behaving properly > > > Key: SPARK-7286 > URL: https://issues.apache.org/jira/browse/SPARK-7286 > Project: Spark > Issue Type: Bug > Components: SQL >Affects Versions: 1.3.1 > Environment: Linux >Reporter: DevilJetha >Priority: Critical > > The precedence of the operators ( especially with !== and && ) in Dataframe > Columns seems to be messed up. > Example Snippet > .where( $"col1" === "val1" && ($"col2" !== "val2") ) works fine. > whereas .where( $"col1" === "val1" && $"col2" !== "val2" ) > evaluates as ( $"col1" === "val1" && $"col2" ) !== "val2" -- 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
[jira] [Commented] (SPARK-11327) spark-dispatcher doesn't pass along some spark properties
[ https://issues.apache.org/jira/browse/SPARK-11327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15011339#comment-15011339 ] Iulian Dragos commented on SPARK-11327: --- [~abraithwaite] it's a valid question, but it's nevertheless a bug in Mesos cluster-mode.. > spark-dispatcher doesn't pass along some spark properties > - > > Key: SPARK-11327 > URL: https://issues.apache.org/jira/browse/SPARK-11327 > Project: Spark > Issue Type: Bug > Components: Mesos >Reporter: Alan Braithwaite > > I haven't figured out exactly what's going on yet, but there's something in > the spark-dispatcher which is failing to pass along properties to the > spark-driver when using spark-submit in a clustered mesos docker environment. > Most importantly, it's not passing along spark.mesos.executor.docker.image... > cli: > {code} > docker run -t -i --rm --net=host > --entrypoint=/usr/local/spark/bin/spark-submit > docker.example.com/spark:2015.10.2 --conf spark.driver.memory=8G --conf > spark.mesos.executor.docker.image=docker.example.com/spark:2015.10.2 --master > mesos://spark-dispatcher.example.com:31262 --deploy-mode cluster > --properties-file /usr/local/spark/conf/spark-defaults.conf --class > com.example.spark.streaming.MyApp > http://jarserver.example.com:8000/sparkapp.jar zk1.example.com:2181 > spark-testing my-stream 40 > {code} > submit output: > {code} > 15/10/26 22:03:53 INFO RestSubmissionClient: Submitting a request to launch > an application in mesos://compute1.example.com:31262. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Sending POST request to server > at http://compute1.example.com:31262/v1/submissions/create: > { > "action" : "CreateSubmissionRequest", > "appArgs" : [ "zk1.example.com:2181", "spark-testing", "requests", "40" ], > "appResource" : "http://jarserver.example.com:8000/sparkapp.jar;, > "clientSparkVersion" : "1.5.0", > "environmentVariables" : { > "SPARK_SCALA_VERSION" : "2.10", > "SPARK_CONF_DIR" : "/usr/local/spark/conf", > "SPARK_HOME" : "/usr/local/spark", > "SPARK_ENV_LOADED" : "1" > }, > "mainClass" : "com.example.spark.streaming.MyApp", > "sparkProperties" : { > "spark.serializer" : "org.apache.spark.serializer.KryoSerializer", > "spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY" : > "/usr/local/lib/libmesos.so", > "spark.history.fs.logDirectory" : "hdfs://hdfsha.example.com/spark/logs", > "spark.eventLog.enabled" : "true", > "spark.driver.maxResultSize" : "0", > "spark.mesos.deploy.recoveryMode" : "ZOOKEEPER", > "spark.mesos.deploy.zookeeper.url" : > "zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181,zk4.example.com:2181,zk5.example.com:2181", > "spark.jars" : "http://jarserver.example.com:8000/sparkapp.jar;, > "spark.driver.supervise" : "false", > "spark.app.name" : "com.example.spark.streaming.MyApp", > "spark.driver.memory" : "8G", > "spark.logConf" : "true", > "spark.deploy.zookeeper.dir" : "/spark_mesos_dispatcher", > "spark.mesos.executor.docker.image" : > "docker.example.com/spark-prod:2015.10.2", > "spark.submit.deployMode" : "cluster", > "spark.master" : "mesos://compute1.example.com:31262", > "spark.executor.memory" : "8G", > "spark.eventLog.dir" : "hdfs://hdfsha.example.com/spark/logs", > "spark.mesos.docker.executor.network" : "HOST", > "spark.mesos.executor.home" : "/usr/local/spark" > } > } > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Response from the server: > { > "action" : "CreateSubmissionResponse", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > 15/10/26 22:03:53 INFO RestSubmissionClient: Submission successfully created > as driver-20151026220353-0011. Polling submission state... > 15/10/26 22:03:53 INFO RestSubmissionClient: Submitting a request for the > status of submission driver-20151026220353-0011 in > mesos://compute1.example.com:31262. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Sending GET request to server > at > http://compute1.example.com:31262/v1/submissions/status/driver-20151026220353-0011. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Response from the server: > { > "action" : "SubmissionStatusResponse", > "driverState" : "QUEUED", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > 15/10/26 22:03:53 INFO RestSubmissionClient: State of driver > driver-20151026220353-0011 is now QUEUED. > 15/10/26 22:03:53 INFO RestSubmissionClient: Server responded with > CreateSubmissionResponse: > { > "action" : "CreateSubmissionResponse", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > {code} > driver log: > {code} >
[jira] [Commented] (SPARK-11327) spark-dispatcher doesn't pass along some spark properties
[ https://issues.apache.org/jira/browse/SPARK-11327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14989280#comment-14989280 ] Iulian Dragos commented on SPARK-11327: --- I think this is related to (possibly a duplicate of) SPARK-11280. > spark-dispatcher doesn't pass along some spark properties > - > > Key: SPARK-11327 > URL: https://issues.apache.org/jira/browse/SPARK-11327 > Project: Spark > Issue Type: Bug > Components: Mesos >Reporter: Alan Braithwaite > > I haven't figured out exactly what's going on yet, but there's something in > the spark-dispatcher which is failing to pass along properties to the > spark-driver when using spark-submit in a clustered mesos docker environment. > Most importantly, it's not passing along spark.mesos.executor.docker.image... > cli: > {code} > docker run -t -i --rm --net=host > --entrypoint=/usr/local/spark/bin/spark-submit > docker.example.com/spark:2015.10.2 --conf spark.driver.memory=8G --conf > spark.mesos.executor.docker.image=docker.example.com/spark:2015.10.2 --master > mesos://spark-dispatcher.example.com:31262 --deploy-mode cluster > --properties-file /usr/local/spark/conf/spark-defaults.conf --class > com.example.spark.streaming.MyApp > http://jarserver.example.com:8000/sparkapp.jar zk1.example.com:2181 > spark-testing my-stream 40 > {code} > submit output: > {code} > 15/10/26 22:03:53 INFO RestSubmissionClient: Submitting a request to launch > an application in mesos://compute1.example.com:31262. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Sending POST request to server > at http://compute1.example.com:31262/v1/submissions/create: > { > "action" : "CreateSubmissionRequest", > "appArgs" : [ "zk1.example.com:2181", "spark-testing", "requests", "40" ], > "appResource" : "http://jarserver.example.com:8000/sparkapp.jar;, > "clientSparkVersion" : "1.5.0", > "environmentVariables" : { > "SPARK_SCALA_VERSION" : "2.10", > "SPARK_CONF_DIR" : "/usr/local/spark/conf", > "SPARK_HOME" : "/usr/local/spark", > "SPARK_ENV_LOADED" : "1" > }, > "mainClass" : "com.example.spark.streaming.MyApp", > "sparkProperties" : { > "spark.serializer" : "org.apache.spark.serializer.KryoSerializer", > "spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY" : > "/usr/local/lib/libmesos.so", > "spark.history.fs.logDirectory" : "hdfs://hdfsha.example.com/spark/logs", > "spark.eventLog.enabled" : "true", > "spark.driver.maxResultSize" : "0", > "spark.mesos.deploy.recoveryMode" : "ZOOKEEPER", > "spark.mesos.deploy.zookeeper.url" : > "zk1.example.com:2181,zk2.example.com:2181,zk3.example.com:2181,zk4.example.com:2181,zk5.example.com:2181", > "spark.jars" : "http://jarserver.example.com:8000/sparkapp.jar;, > "spark.driver.supervise" : "false", > "spark.app.name" : "com.example.spark.streaming.MyApp", > "spark.driver.memory" : "8G", > "spark.logConf" : "true", > "spark.deploy.zookeeper.dir" : "/spark_mesos_dispatcher", > "spark.mesos.executor.docker.image" : > "docker.example.com/spark-prod:2015.10.2", > "spark.submit.deployMode" : "cluster", > "spark.master" : "mesos://compute1.example.com:31262", > "spark.executor.memory" : "8G", > "spark.eventLog.dir" : "hdfs://hdfsha.example.com/spark/logs", > "spark.mesos.docker.executor.network" : "HOST", > "spark.mesos.executor.home" : "/usr/local/spark" > } > } > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Response from the server: > { > "action" : "CreateSubmissionResponse", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > 15/10/26 22:03:53 INFO RestSubmissionClient: Submission successfully created > as driver-20151026220353-0011. Polling submission state... > 15/10/26 22:03:53 INFO RestSubmissionClient: Submitting a request for the > status of submission driver-20151026220353-0011 in > mesos://compute1.example.com:31262. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Sending GET request to server > at > http://compute1.example.com:31262/v1/submissions/status/driver-20151026220353-0011. > 15/10/26 22:03:53 DEBUG RestSubmissionClient: Response from the server: > { > "action" : "SubmissionStatusResponse", > "driverState" : "QUEUED", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > 15/10/26 22:03:53 INFO RestSubmissionClient: State of driver > driver-20151026220353-0011 is now QUEUED. > 15/10/26 22:03:53 INFO RestSubmissionClient: Server responded with > CreateSubmissionResponse: > { > "action" : "CreateSubmissionResponse", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151026220353-0011", > "success" : true > } > {code} > driver log: > {code} > 15/10/26 22:08:08 INFO
[jira] [Commented] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14974480#comment-14974480 ] Iulian Dragos commented on SPARK-10986: --- Digging a bit deeper. The problem is that the context class loader is not set when running in fine-grained mode. When the Java serializer is created, it uses a {{null}} classloader, leading to {{ClassNotFoundException}} ({{Class.forName}} with a null classloader uses the primordial class loader, meaning only the JDK is in there). In coarse-grained, the context class loader is set by some hadoop classes dealing with {{UserGroupInformation}}, via {{runAsSparkUser}}. It's probably totally accidental that it works. I'm not sure where is the right moment to set the context class loader, and who else relies on it. This part is totally undocumented. > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu >Priority: Blocker > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > Works fine in coarse-grained mode, only fails in *fine-grained mode*. > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:244) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at >
[jira] [Comment Edited] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14974480#comment-14974480 ] Iulian Dragos edited comment on SPARK-10986 at 10/26/15 4:17 PM: - Digging a bit deeper. The problem is that the context class loader is not set when running in fine-grained mode. When the Java serializer is created, it uses a {{null}} classloader, leading to {{ClassNotFoundException}} ({{Class.forName}} with a null classloader uses the primordial class loader, meaning only the JDK is in there). In coarse-grained, the context class loader is set by some hadoop classes dealing with {{UserGroupInformation}}, via {{runAsSparkUser}}. It's probably totally accidental that it works. I'm not sure where is the right place to set the context class loader, and who else relies on it. This part is totally undocumented. was (Author: dragos): Digging a bit deeper. The problem is that the context class loader is not set when running in fine-grained mode. When the Java serializer is created, it uses a {{null}} classloader, leading to {{ClassNotFoundException}} ({{Class.forName}} with a null classloader uses the primordial class loader, meaning only the JDK is in there). In coarse-grained, the context class loader is set by some hadoop classes dealing with {{UserGroupInformation}}, via {{runAsSparkUser}}. It's probably totally accidental that it works. I'm not sure where is the right moment to set the context class loader, and who else relies on it. This part is totally undocumented. > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu >Priority: Blocker > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > Works fine in coarse-grained mode, only fails in *fine-grained mode*. > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at >
[jira] [Commented] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14974673#comment-14974673 ] Iulian Dragos commented on SPARK-10986: --- [~kaysoky] can you try my PR and see if it solves it for you? > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu >Priority: Blocker > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > Works fine in coarse-grained mode, only fails in *fine-grained mode*. > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:244) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at
[jira] [Commented] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14974777#comment-14974777 ] Iulian Dragos commented on SPARK-10986: --- Great! Please leave a note on the PR as well.. It's been particularly difficult to get attention from committers lately. > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu >Priority: Blocker > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > Works fine in coarse-grained mode, only fails in *fine-grained mode*. > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:244) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at >
[jira] [Updated] (SPARK-11280) Mesos cluster deployment using only one node
[ https://issues.apache.org/jira/browse/SPARK-11280?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-11280: -- Attachment: Screen Shot 2015-10-23 at 11.37.43.png > Mesos cluster deployment using only one node > > > Key: SPARK-11280 > URL: https://issues.apache.org/jira/browse/SPARK-11280 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.5.1, 1.6.0 >Reporter: Iulian Dragos > Attachments: Screen Shot 2015-10-23 at 11.37.43.png > > > I submit the SparkPi example in Mesos cluster mode, and I notice that all > tasks fail except the ones that run on the same node as the driver. The > others fail with > {code} > sh: 1: > /tmp/mesos/slaves/1521e408-d8fe-416d-898b-3801e73a8293-S0/frameworks/1521e408-d8fe-416d-898b-3801e73a8293-0003/executors/driver-20151023113121-0006/runs/2abefd29-7386-4d81-a025-9d794780db23/spark-1.5.0-bin-hadoop2.6/bin/spark-class: > not found > {code} > The path exists only on the machine that launched the driver, and the sandbox > of the executor where this task died is completely empty. > I launch the task like this: > {code} > $ spark-submit --deploy-mode cluster --master mesos://sagitarius.local:7077 > --conf > spark.executor.uri="ftp://sagitarius.local/ftp/spark-1.5.0-bin-hadoop2.6.tgz; > --conf spark.mesos.coarse=true --class org.apache.spark.examples.SparkPi > ftp://sagitarius.local/ftp/spark-examples-1.5.0-hadoop2.6.0.jar > Using Spark's default log4j profile: > org/apache/spark/log4j-defaults.properties > 15/10/23 11:31:21 INFO RestSubmissionClient: Submitting a request to launch > an application in mesos://sagitarius.local:7077. > 15/10/23 11:31:21 INFO RestSubmissionClient: Submission successfully created > as driver-20151023113121-0006. Polling submission state... > 15/10/23 11:31:21 INFO RestSubmissionClient: Submitting a request for the > status of submission driver-20151023113121-0006 in > mesos://sagitarius.local:7077. > 15/10/23 11:31:21 INFO RestSubmissionClient: State of driver > driver-20151023113121-0006 is now QUEUED. > 15/10/23 11:31:21 INFO RestSubmissionClient: Server responded with > CreateSubmissionResponse: > { > "action" : "CreateSubmissionResponse", > "serverSparkVersion" : "1.5.0", > "submissionId" : "driver-20151023113121-0006", > "success" : true > } > {code} > I can see the driver in the Dispatcher UI and the job succeeds eventually, > but running only on the node where the driver was launched (see attachment). -- 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
[jira] [Created] (SPARK-11280) Mesos cluster deployment using only one node
Iulian Dragos created SPARK-11280: - Summary: Mesos cluster deployment using only one node Key: SPARK-11280 URL: https://issues.apache.org/jira/browse/SPARK-11280 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.5.1, 1.6.0 Reporter: Iulian Dragos I submit the SparkPi example in Mesos cluster mode, and I notice that all tasks fail except the ones that run on the same node as the driver. The others fail with {code} sh: 1: /tmp/mesos/slaves/1521e408-d8fe-416d-898b-3801e73a8293-S0/frameworks/1521e408-d8fe-416d-898b-3801e73a8293-0003/executors/driver-20151023113121-0006/runs/2abefd29-7386-4d81-a025-9d794780db23/spark-1.5.0-bin-hadoop2.6/bin/spark-class: not found {code} The path exists only on the machine that launched the driver, and the sandbox of the executor where this task died is completely empty. I launch the task like this: {code} $ spark-submit --deploy-mode cluster --master mesos://sagitarius.local:7077 --conf spark.executor.uri="ftp://sagitarius.local/ftp/spark-1.5.0-bin-hadoop2.6.tgz; --conf spark.mesos.coarse=true --class org.apache.spark.examples.SparkPi ftp://sagitarius.local/ftp/spark-examples-1.5.0-hadoop2.6.0.jar Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties 15/10/23 11:31:21 INFO RestSubmissionClient: Submitting a request to launch an application in mesos://sagitarius.local:7077. 15/10/23 11:31:21 INFO RestSubmissionClient: Submission successfully created as driver-20151023113121-0006. Polling submission state... 15/10/23 11:31:21 INFO RestSubmissionClient: Submitting a request for the status of submission driver-20151023113121-0006 in mesos://sagitarius.local:7077. 15/10/23 11:31:21 INFO RestSubmissionClient: State of driver driver-20151023113121-0006 is now QUEUED. 15/10/23 11:31:21 INFO RestSubmissionClient: Server responded with CreateSubmissionResponse: { "action" : "CreateSubmissionResponse", "serverSparkVersion" : "1.5.0", "submissionId" : "driver-20151023113121-0006", "success" : true } {code} I can see the driver in the Dispatcher UI and the job succeeds eventually, but running only on the node where the driver was launched (see attachment). -- 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
[jira] [Commented] (SPARK-10975) Shuffle files left behind on Mesos without dynamic allocation
[ https://issues.apache.org/jira/browse/SPARK-10975?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14971268#comment-14971268 ] Iulian Dragos commented on SPARK-10975: --- No, it's not a duplicate, but fixed by the same PR :) > Shuffle files left behind on Mesos without dynamic allocation > - > > Key: SPARK-10975 > URL: https://issues.apache.org/jira/browse/SPARK-10975 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.5.1 >Reporter: Iulian Dragos >Priority: Blocker > > (from mailing list) > Running on Mesos in coarse-grained mode. No dynamic allocation or shuffle > service. > I see that there are two types of temporary files under /tmp folder > associated with every executor: /tmp/spark- and /tmp/blockmgr-. > When job is finished /tmp/spark- is gone, but blockmgr directory is > left with all gigabytes in it. > The reason is that logic to clean up files is only enabled when the shuffle > service is running, see https://github.com/apache/spark/pull/7820 > The shuffle files should be placed in the Mesos sandbox or under `tmp/spark` > unless the shuffle service is enabled. -- 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
[jira] [Updated] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-10986: -- Priority: Blocker (was: Major) > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu >Priority: Blocker > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:244) > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) > at > io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) > at > io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) > at > io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) > at > io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) > at > io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) > at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) > at >
[jira] [Commented] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14950507#comment-14950507 ] Iulian Dragos commented on SPARK-10986: --- I can confirm this issue. It only happens in *fine-grained mode*. It's weird, because I downloaded the Spark assembly jar from the slave where it failed (it fails on all slaves), and the class *is* there: {code} $ javap -classpath ~/Downloads/spark-assembly-1.6.0-SNAPSHOT-hadoop2.6.0.jar org.apache.spark.rpc.netty.AskResponse Compiled from "NettyRpcEnv.scala" public class org.apache.spark.rpc.netty.AskResponse implements org.apache.spark.rpc.netty.ResponseMessage,scala.Product,scala.Serializable { public static scala.Option> unapply(org.apache.spark.rpc.netty.AskResponse); public static org.apache.spark.rpc.netty.AskResponse apply(org.apache.spark.rpc.netty.NettyRpcEndpointRef, java.lang.Object); public static scala.Function1 , org.apache.spark.rpc.netty.AskResponse> tupled(); public static scala.Function1 > curried(); public org.apache.spark.rpc.netty.NettyRpcEndpointRef sender(); public java.lang.Object reply(); public org.apache.spark.rpc.netty.AskResponse copy(org.apache.spark.rpc.netty.NettyRpcEndpointRef, java.lang.Object); public org.apache.spark.rpc.netty.NettyRpcEndpointRef copy$default$1(); public java.lang.Object copy$default$2(); public java.lang.String productPrefix(); public int productArity(); public java.lang.Object productElement(int); public scala.collection.Iterator productIterator(); public boolean canEqual(java.lang.Object); public int hashCode(); public java.lang.String toString(); public boolean equals(java.lang.Object); public org.apache.spark.rpc.netty.AskResponse(org.apache.spark.rpc.netty.NettyRpcEndpointRef, java.lang.Object); } {code} > ClassNotFoundException when running on Client mode, with a Mesos master. > > > Key: SPARK-10986 > URL: https://issues.apache.org/jira/browse/SPARK-10986 > Project: Spark > Issue Type: Bug > Components: Mesos > Environment: OSX, Java 8, Mesos 0.25.0 > HEAD of Spark (`f5d154bc731aedfc2eecdb4ed6af8cac820511c9`) > Built from source: > build/mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package >Reporter: Joseph Wu > Labels: mesos, spark > > When running an example task on a Mesos cluster (local master, local agent), > any Spark tasks will stall with the following error (in the executor's > stderr): > {code} > 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on > port 53689. > 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from > /10.0.79.8:53673 > java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) > at > java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) > at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) > at > java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) > at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) > at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) > at > org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) > at > org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) > at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) > at > org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) > at > org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) > at > org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) > at > org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) > at >
[jira] [Updated] (SPARK-10986) ClassNotFoundException when running on Client mode, with a Mesos master.
[ https://issues.apache.org/jira/browse/SPARK-10986?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-10986: -- Description: When running an example task on a Mesos cluster (local master, local agent), any Spark tasks will stall with the following error (in the executor's stderr): Works fine in coarse-grained mode, only fails in *fine-grained mode*. {code} 15/10/07 15:21:14 INFO Utils: Successfully started service 'sparkExecutor' on port 53689. 15/10/07 15:21:14 WARN TransportChannelHandler: Exception in connection from /10.0.79.8:53673 java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:76) at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:109) at org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$deserialize$1.apply(NettyRpcEnv.scala:227) at scala.util.DynamicVariable.withValue(DynamicVariable.scala:57) at org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:265) at org.apache.spark.rpc.netty.NettyRpcEnv.deserialize(NettyRpcEnv.scala:226) at org.apache.spark.rpc.netty.NettyRpcEnv$$anon$3$$anon$4.onSuccess(NettyRpcEnv.scala:196) at org.apache.spark.network.client.TransportResponseHandler.handle(TransportResponseHandler.java:152) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:103) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:51) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) at io.netty.handler.timeout.IdleStateHandler.channelRead(IdleStateHandler.java:266) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:244) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111) at java.lang.Thread.run(Thread.java:745) 15/10/07 15:21:14 ERROR NettyRpcHandler: org/apache/spark/rpc/netty/AskResponse java.lang.ClassNotFoundException: org/apache/spark/rpc/netty/AskResponse at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:68) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) at
[jira] [Created] (SPARK-10975) Shuffle files left behind on Mesos without dynamic allocation
Iulian Dragos created SPARK-10975: - Summary: Shuffle files left behind on Mesos without dynamic allocation Key: SPARK-10975 URL: https://issues.apache.org/jira/browse/SPARK-10975 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.5.1 Reporter: Iulian Dragos Priority: Blocker (from mailing list) Running on Mesos in coarse-grained mode. No dynamic allocation or shuffle service. I see that there are two types of temporary files under /tmp folder associated with every executor: /tmp/spark- and /tmp/blockmgr-. When job is finished /tmp/spark- is gone, but blockmgr directory is left with all gigabytes in it. The reason is that logic to clean up files is only enabled when the shuffle service is running, see https://github.com/apache/spark/pull/7820 The shuffle files should be placed in the Mesos sandbox or under `tmp/spark` unless the shuffle service is enabled. -- 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
[jira] [Commented] (SPARK-7398) Add back-pressure to Spark Streaming (umbrella JIRA)
[ https://issues.apache.org/jira/browse/SPARK-7398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14940184#comment-14940184 ] Iulian Dragos commented on SPARK-7398: -- Hey, except the last point, everything is available in 1.5. You can go ahead and tackle the remaining ticket, of course. > Add back-pressure to Spark Streaming (umbrella JIRA) > > > Key: SPARK-7398 > URL: https://issues.apache.org/jira/browse/SPARK-7398 > Project: Spark > Issue Type: Improvement > Components: Streaming >Affects Versions: 1.3.1 >Reporter: François Garillot >Assignee: Tathagata Das >Priority: Critical > Labels: streams > > Spark Streaming has trouble dealing with situations where > batch processing time > batch interval > Meaning a high throughput of input data w.r.t. Spark's ability to remove data > from the queue. > If this throughput is sustained for long enough, it leads to an unstable > situation where the memory of the Receiver's Executor is overflowed. > This aims at transmitting a back-pressure signal back to data ingestion to > help with dealing with that high throughput, in a backwards-compatible way. > The original design doc can be found here: > https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing > The second design doc, focusing [on the first > sub-task|https://issues.apache.org/jira/browse/SPARK-8834] (without all the > background info, and more centered on the implementation) can be found here: > https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing -- 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
[jira] [Commented] (SPARK-7841) Spark build should not use lib_managed for dependencies
[ https://issues.apache.org/jira/browse/SPARK-7841?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14791674#comment-14791674 ] Iulian Dragos commented on SPARK-7841: -- Yes, there are a few build scripts (including make-distribution IIRC) that depend on having things in `lib_managed`. For the moment I'm applying a patch locally, I hope to have some time to look at this in the next week or two. > Spark build should not use lib_managed for dependencies > --- > > Key: SPARK-7841 > URL: https://issues.apache.org/jira/browse/SPARK-7841 > Project: Spark > Issue Type: Bug > Components: Build >Affects Versions: 1.3.1 >Reporter: Iulian Dragos > Labels: easyfix, sbt > > - unnecessary duplication (I will have those libraries under ./m2, via maven > anyway) > - every time I call make-distribution I lose lib_managed (via mvn clean > install) and have to wait to download again all jars next time I use sbt > - Eclipse does not handle relative paths very well (source attachments from > lib_managed don’t always work) > - it's not the default configuration. If we stray from defaults I think there > should be a clear advantage. > Digging through history, the only reference to `retrieveManaged := true` I > found was in f686e3d, from July 2011 ("Initial work on converting build to > SBT 0.10.1"). My guess this is purely an accident of porting the build form > Sbt 0.7.x and trying to keep the old project layout. > If there are reasons for keeping it, please comment (I didn't get any answers > on the [dev mailing > list|http://apache-spark-developers-list.1001551.n3.nabble.com/Why-use-quot-lib-managed-quot-for-the-Sbt-build-td12361.html]) -- 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
[jira] [Commented] (SPARK-10444) Remove duplication in Mesos schedulers
[ https://issues.apache.org/jira/browse/SPARK-10444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14736434#comment-14736434 ] Iulian Dragos commented on SPARK-10444: --- Another example of duplicated logic: https://github.com/apache/spark/pull/8639 > Remove duplication in Mesos schedulers > -- > > Key: SPARK-10444 > URL: https://issues.apache.org/jira/browse/SPARK-10444 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.5.0 >Reporter: Iulian Dragos > Labels: refactoring > > Currently coarse-grained and fine-grained Mesos schedulers don't share much > code, and that leads to inconsistencies. For instance: > - only coarse-grained mode respects {{spark.cores.max}}, see SPARK-9873 > - only coarse-grained mode blacklists slaves that fail repeatedly, but that > seams like generally useful > - constraints and memory checking are done on both sides (code is shared > though) > - framework re-registration (master election) is only done for cluster-mode > deployment > We should find a better design that groups together common concerns and > generally improves the 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
[jira] [Comment Edited] (SPARK-6350) Make mesosExecutorCores configurable in mesos "fine-grained" mode
[ https://issues.apache.org/jira/browse/SPARK-6350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14734575#comment-14734575 ] Iulian Dragos edited comment on SPARK-6350 at 9/8/15 10:26 AM: --- I'm re-opening this, since in the meantime this regressed. Original fix was in commit d86bbb, which regressed in [PR 4960|https://github.com/apache/spark/pull/4960]. [~tnachen] you might want to take a look at this. was (Author: dragos): I'm re-opening this, since in the meantime this regressed. See changes in d86bbb, which regressed in [PR 4960|https://github.com/apache/spark/pull/4960]. [~tnachen] you might want to take a look at this. > Make mesosExecutorCores configurable in mesos "fine-grained" mode > - > > Key: SPARK-6350 > URL: https://issues.apache.org/jira/browse/SPARK-6350 > Project: Spark > Issue Type: Improvement > Components: Mesos >Affects Versions: 1.0.0 >Reporter: Jongyoul Lee >Assignee: Jongyoul Lee >Priority: Minor > Fix For: 1.4.0 > > > When spark runs in mesos fine-grained mode, mesos slave launches executor > with # of cpus and memories. By the way, # of executor's cores is always > CPU_PER_TASKS as same as spark.task.cpus. If I set that values as 5 for > running intensive task, mesos executor always consume 5 cores without any > running task. This waste resources. We should set executor core as a > configuration variable. -- 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
[jira] [Reopened] (SPARK-6350) Make mesosExecutorCores configurable in mesos "fine-grained" mode
[ https://issues.apache.org/jira/browse/SPARK-6350?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos reopened SPARK-6350: -- I'm re-opening this, since in the meantime this regressed. See changes in d86bbb, which regressed in [PR 4960|https://github.com/apache/spark/pull/4960]. [~tnachen] you might want to take a look at this. > Make mesosExecutorCores configurable in mesos "fine-grained" mode > - > > Key: SPARK-6350 > URL: https://issues.apache.org/jira/browse/SPARK-6350 > Project: Spark > Issue Type: Improvement > Components: Mesos >Affects Versions: 1.0.0 >Reporter: Jongyoul Lee >Assignee: Jongyoul Lee >Priority: Minor > Fix For: 1.4.0 > > > When spark runs in mesos fine-grained mode, mesos slave launches executor > with # of cpus and memories. By the way, # of executor's cores is always > CPU_PER_TASKS as same as spark.task.cpus. If I set that values as 5 for > running intensive task, mesos executor always consume 5 cores without any > running task. This waste resources. We should set executor core as a > configuration variable. -- 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
[jira] [Commented] (SPARK-9708) Spark should create local temporary directories in Mesos sandbox when launched with Mesos
[ https://issues.apache.org/jira/browse/SPARK-9708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14735011#comment-14735011 ] Iulian Dragos commented on SPARK-9708: -- This won't work when the external shuffle service is enabled, so that could be the flag here (actually, it's negation: enable this behavior only when the external shuffle service is disabled). > Spark should create local temporary directories in Mesos sandbox when > launched with Mesos > - > > Key: SPARK-9708 > URL: https://issues.apache.org/jira/browse/SPARK-9708 > Project: Spark > Issue Type: Bug > Components: Mesos >Reporter: Timothy Chen > > Currently Spark creates temporary directories with > Utils.getConfiguredLocalDirs, and it writes to YARN directories if YARN is > detected, otherwise just writes in a temporary directory in the host. > However, Mesos does create a directory per task and ideally Spark should use > that directory to create its local temporary directories since it then can be > cleaned up when the task is gone and not left on the host or cleaned until > reboot. -- 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
[jira] [Commented] (SPARK-4940) Support more evenly distributing cores for Mesos mode
[ https://issues.apache.org/jira/browse/SPARK-4940?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14730401#comment-14730401 ] Iulian Dragos commented on SPARK-4940: -- [~doctapp] I guess there is no perfect scheduling for every application, so some level of configuration may help. We could implement some form of queue based scheduling, with applications assigned to queues, and queues having different algorithms. But I fear the additional complexity may not pay off. What is the case you had in mind? I'm wondering if different applications specify their preferred allocation strategy would really help. Assuming 2 apps on the cluster, one is streaming (prefers round-robin), the other one is... something else, that prefers "fill-up-slave". Would this really buy anything to the second one? And what type of application prefers to have multiple executors on the same slave, given the choice? I'd imagine in most cases having more slaves is better (for instance IO interference is lower). > Support more evenly distributing cores for Mesos mode > - > > Key: SPARK-4940 > URL: https://issues.apache.org/jira/browse/SPARK-4940 > Project: Spark > Issue Type: Improvement > Components: Mesos >Reporter: Timothy Chen > Attachments: mesos-config-difference-3nodes-vs-2nodes.png > > > Currently in Coarse grain mode the spark scheduler simply takes all the > resources it can on each node, but can cause uneven distribution based on > resources available on each slave. -- 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
[jira] [Created] (SPARK-10444) Remove duplication in Mesos schedulers
Iulian Dragos created SPARK-10444: - Summary: Remove duplication in Mesos schedulers Key: SPARK-10444 URL: https://issues.apache.org/jira/browse/SPARK-10444 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.5.0 Reporter: Iulian Dragos Currently coarse-grained and fine-grained Mesos schedulers don't share much code, and that leads to inconsistencies. For instance: - only coarse-grained mode respects {{spark.cores.max}}, see SPARK-9873 - only coarse-grained mode blacklists slaves that fail repeatedly, but that seams like generally useful - constraints and memory checking are done on both sides (code is shared though) - framework re-registration (master election) is only done for cluster-mode deployment We should find a better design that groups together common concerns and generally improves the 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
[jira] [Commented] (SPARK-7874) Add a global setting for the fine-grained mesos scheduler that limits the number of concurrent tasks of a job
[ https://issues.apache.org/jira/browse/SPARK-7874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14729314#comment-14729314 ] Iulian Dragos commented on SPARK-7874: -- [~tnachen] I'm going to implement `spark.cores.max` as the limit, is that what you had in mind? > Add a global setting for the fine-grained mesos scheduler that limits the > number of concurrent tasks of a job > - > > Key: SPARK-7874 > URL: https://issues.apache.org/jira/browse/SPARK-7874 > Project: Spark > Issue Type: Wish > Components: Mesos >Affects Versions: 1.3.1 >Reporter: Thomas Dudziak >Priority: Minor > > This would be a very simple yet effective way to prevent a job dominating the > cluster. A way to override it per job would also be nice but not required. -- 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
[jira] [Commented] (SPARK-7874) Add a global setting for the fine-grained mesos scheduler that limits the number of concurrent tasks of a job
[ https://issues.apache.org/jira/browse/SPARK-7874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726775#comment-14726775 ] Iulian Dragos commented on SPARK-7874: -- [~tomdz] do you mean respecting `spark.cores.max`, as it is the case in coarse-grained mode? > Add a global setting for the fine-grained mesos scheduler that limits the > number of concurrent tasks of a job > - > > Key: SPARK-7874 > URL: https://issues.apache.org/jira/browse/SPARK-7874 > Project: Spark > Issue Type: Wish > Components: Mesos >Affects Versions: 1.3.1 >Reporter: Thomas Dudziak >Priority: Minor > > This would be a very simple yet effective way to prevent a job dominating the > cluster. A way to override it per job would also be nice but not required. -- 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
[jira] [Commented] (SPARK-3871) compute-classpath.sh does not escape :
[ https://issues.apache.org/jira/browse/SPARK-3871?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726785#comment-14726785 ] Iulian Dragos commented on SPARK-3871: -- There's no more compute-classpath.sh. Ok to close this? > compute-classpath.sh does not escape : > -- > > Key: SPARK-3871 > URL: https://issues.apache.org/jira/browse/SPARK-3871 > Project: Spark > Issue Type: Bug > Components: Mesos >Affects Versions: 1.1.0 >Reporter: Hector Yee >Priority: Minor > > Chronos jobs on Mesos schedule jobs in temp directories such as > /tmp/mesos/slaves/20140926-142803-3852091146-5050-3487-375/frameworks/20140719-203536-160311562-5050-10655-0007/executors/ct:1412815902180:2:search_ranking_scoring/runs/f1e0d058-3ef0-4838-816e-e3fa5e179dd8 > The compute-classpath.sh does not properly escape the : in the temp dirs > generated by mesos and so the spark-submit gets a broken classpath -- 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
[jira] [Commented] (SPARK-4940) Support more evenly distributing cores for Mesos mode
[ https://issues.apache.org/jira/browse/SPARK-4940?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726784#comment-14726784 ] Iulian Dragos commented on SPARK-4940: -- Would it make sense to allocate resources in a round-robin fashion? Supposing Spark gets several offers at the same time, it would have enough info to balance executors on the available resources (or optionally, define an interval during which it holds on to resources it receives to accumulate a larger set of slaves). The algorithm may proceed by allocating a multiple `spark.task.cores` (below cap, see SPARK-9873 which might help on its own) on each slave in the set of resources, until it can't allocate anymore. > Support more evenly distributing cores for Mesos mode > - > > Key: SPARK-4940 > URL: https://issues.apache.org/jira/browse/SPARK-4940 > Project: Spark > Issue Type: Improvement > Components: Mesos >Reporter: Timothy Chen > Attachments: mesos-config-difference-3nodes-vs-2nodes.png > > > Currently in Coarse grain mode the spark scheduler simply takes all the > resources it can on each node, but can cause uneven distribution based on > resources available on each slave. -- 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
[jira] [Commented] (SPARK-9708) Spark should create local temporary directories in Mesos sandbox when launched with Mesos
[ https://issues.apache.org/jira/browse/SPARK-9708?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14725657#comment-14725657 ] Iulian Dragos commented on SPARK-9708: -- I'm not sure if this is the entire story. Remember that shuffle files need to survive the executor when dynamic allocation is enabled. So, with this proposed change, if the scheduler decides to kill an executor its shuffle files will be gone and the external shuffle server won't be able to find them anymore. At least shuffle files need to go on another directory, not under the sandbox. Also, Spark allows one to configure `spark.local.dir`, and that should take precedence. In the Hadoop world, this can be used to specify several directories on different physical disks (to allow fast parallel writes). > Spark should create local temporary directories in Mesos sandbox when > launched with Mesos > - > > Key: SPARK-9708 > URL: https://issues.apache.org/jira/browse/SPARK-9708 > Project: Spark > Issue Type: Bug > Components: Mesos >Reporter: Timothy Chen > > Currently Spark creates temporary directories with > Utils.getConfiguredLocalDirs, and it writes to YARN directories if YARN is > detected, otherwise just writes in a temporary directory in the host. > However, Mesos does create a directory per task and ideally Spark should use > that directory to create its local temporary directories since it then can be > cleaned up when the task is gone and not left on the host or cleaned until > reboot. -- 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
[jira] [Created] (SPARK-8977) Define the RateEstimator interface, and implement the ReceiverRateController
Iulian Dragos created SPARK-8977: Summary: Define the RateEstimator interface, and implement the ReceiverRateController Key: SPARK-8977 URL: https://issues.apache.org/jira/browse/SPARK-8977 Project: Spark Issue Type: Sub-task Components: Streaming Reporter: Iulian Dragos Fix For: 1.5.0 Full [design doc|https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing] Implement a rate controller for receiver-based InputDStreams that estimates a maximum rate and sends it to each receiver supervisor. -- 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
[jira] [Commented] (SPARK-5281) Registering table on RDD is giving MissingRequirementError
[ https://issues.apache.org/jira/browse/SPARK-5281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14622510#comment-14622510 ] Iulian Dragos commented on SPARK-5281: -- Thanks for pointing them out. Glad it wasn't too bad :) Registering table on RDD is giving MissingRequirementError -- Key: SPARK-5281 URL: https://issues.apache.org/jira/browse/SPARK-5281 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.2.0, 1.3.1 Reporter: sarsol Assignee: Iulian Dragos Priority: Critical Fix For: 1.4.0 Application crashes on this line {{rdd.registerTempTable(temp)}} in 1.2 version when using sbt or Eclipse SCALA IDE Stacktrace: {code} Exception in thread main scala.reflect.internal.MissingRequirementError: class org.apache.spark.sql.catalyst.ScalaReflection in JavaMirror with primordial classloader with boot classpath [C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-library.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-reflect.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-actor.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-swing.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-compiler.jar;C:\Program Files\Java\jre7\lib\resources.jar;C:\Program Files\Java\jre7\lib\rt.jar;C:\Program Files\Java\jre7\lib\sunrsasign.jar;C:\Program Files\Java\jre7\lib\jsse.jar;C:\Program Files\Java\jre7\lib\jce.jar;C:\Program Files\Java\jre7\lib\charsets.jar;C:\Program Files\Java\jre7\lib\jfr.jar;C:\Program Files\Java\jre7\classes] not found. at scala.reflect.internal.MissingRequirementError$.signal(MissingRequirementError.scala:16) at scala.reflect.internal.MissingRequirementError$.notFound(MissingRequirementError.scala:17) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:61) at scala.reflect.internal.Mirrors$RootsBase.staticModuleOrClass(Mirrors.scala:72) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:119) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:21) at org.apache.spark.sql.catalyst.ScalaReflection$$typecreator1$1.apply(ScalaReflection.scala:115) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe$lzycompute(TypeTags.scala:231) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe(TypeTags.scala:231) at scala.reflect.api.TypeTags$class.typeOf(TypeTags.scala:335) at scala.reflect.api.Universe.typeOf(Universe.scala:59) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:115) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:100) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.attributesFor(ScalaReflection.scala:94) at org.apache.spark.sql.catalyst.ScalaReflection$.attributesFor(ScalaReflection.scala:33) at org.apache.spark.sql.SQLContext.createSchemaRDD(SQLContext.scala:111) at com.sar.spark.dq.poc.SparkPOC$delayedInit$body.apply(SparkPOC.scala:43) at scala.Function0$class.apply$mcV$sp(Function0.scala:40) at scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:32) at scala.App$class.main(App.scala:71) {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
[jira] [Created] (SPARK-8975) Implement a mechanism to send a new rate from the driver to the block generator
Iulian Dragos created SPARK-8975: Summary: Implement a mechanism to send a new rate from the driver to the block generator Key: SPARK-8975 URL: https://issues.apache.org/jira/browse/SPARK-8975 Project: Spark Issue Type: Sub-task Components: Streaming Reporter: Iulian Dragos Full design doc [here|https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing] - Add a new message, {{RateUpdate(newRate: Long)}} that ReceiverSupervisor handles in its endpoint - Add a new method to ReceiverTracker {{def sendRateUpdate(streamId: Int, newRate: Long): Unit}} this method sends an asynchronous RateUpdate message to the receiver supervisor corresponding to streamId - update the rate in the corresponding block generator. -- 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
[jira] [Created] (SPARK-8978) Implement the DirectKafkaController
Iulian Dragos created SPARK-8978: Summary: Implement the DirectKafkaController Key: SPARK-8978 URL: https://issues.apache.org/jira/browse/SPARK-8978 Project: Spark Issue Type: Sub-task Components: Streaming Reporter: Iulian Dragos Fix For: 1.5.0 Based on this [design doc|https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing]. The DirectKafkaInputDStream should use the rate estimate to control how many records/partition to put in the next batch. -- 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
[jira] [Created] (SPARK-8979) Implement a PIDRateEstimator
Iulian Dragos created SPARK-8979: Summary: Implement a PIDRateEstimator Key: SPARK-8979 URL: https://issues.apache.org/jira/browse/SPARK-8979 Project: Spark Issue Type: Sub-task Components: Streaming Reporter: Iulian Dragos Fix For: 1.5.0 Based on this [design doc|https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing] -- 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
[jira] [Commented] (SPARK-5281) Registering table on RDD is giving MissingRequirementError
[ https://issues.apache.org/jira/browse/SPARK-5281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14620148#comment-14620148 ] Iulian Dragos commented on SPARK-5281: -- [~marmbrus] what were those regressions, out of curiosity? I haven't followed Jira closely, but nothing is linked to this ticket. Registering table on RDD is giving MissingRequirementError -- Key: SPARK-5281 URL: https://issues.apache.org/jira/browse/SPARK-5281 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.2.0, 1.3.1 Reporter: sarsol Assignee: Iulian Dragos Priority: Critical Fix For: 1.4.0 Application crashes on this line {{rdd.registerTempTable(temp)}} in 1.2 version when using sbt or Eclipse SCALA IDE Stacktrace: {code} Exception in thread main scala.reflect.internal.MissingRequirementError: class org.apache.spark.sql.catalyst.ScalaReflection in JavaMirror with primordial classloader with boot classpath [C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-library.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-reflect.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-actor.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-swing.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-compiler.jar;C:\Program Files\Java\jre7\lib\resources.jar;C:\Program Files\Java\jre7\lib\rt.jar;C:\Program Files\Java\jre7\lib\sunrsasign.jar;C:\Program Files\Java\jre7\lib\jsse.jar;C:\Program Files\Java\jre7\lib\jce.jar;C:\Program Files\Java\jre7\lib\charsets.jar;C:\Program Files\Java\jre7\lib\jfr.jar;C:\Program Files\Java\jre7\classes] not found. at scala.reflect.internal.MissingRequirementError$.signal(MissingRequirementError.scala:16) at scala.reflect.internal.MissingRequirementError$.notFound(MissingRequirementError.scala:17) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:61) at scala.reflect.internal.Mirrors$RootsBase.staticModuleOrClass(Mirrors.scala:72) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:119) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:21) at org.apache.spark.sql.catalyst.ScalaReflection$$typecreator1$1.apply(ScalaReflection.scala:115) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe$lzycompute(TypeTags.scala:231) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe(TypeTags.scala:231) at scala.reflect.api.TypeTags$class.typeOf(TypeTags.scala:335) at scala.reflect.api.Universe.typeOf(Universe.scala:59) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:115) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:100) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.attributesFor(ScalaReflection.scala:94) at org.apache.spark.sql.catalyst.ScalaReflection$.attributesFor(ScalaReflection.scala:33) at org.apache.spark.sql.SQLContext.createSchemaRDD(SQLContext.scala:111) at com.sar.spark.dq.poc.SparkPOC$delayedInit$body.apply(SparkPOC.scala:43) at scala.Function0$class.apply$mcV$sp(Function0.scala:40) at scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:32) at scala.App$class.main(App.scala:71) {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
[jira] [Commented] (SPARK-7398) Add back-pressure to Spark Streaming
[ https://issues.apache.org/jira/browse/SPARK-7398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14613240#comment-14613240 ] Iulian Dragos commented on SPARK-7398: -- Document updated. Add back-pressure to Spark Streaming Key: SPARK-7398 URL: https://issues.apache.org/jira/browse/SPARK-7398 Project: Spark Issue Type: Improvement Components: Streaming Affects Versions: 1.3.1 Reporter: François Garillot Priority: Critical Labels: streams Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The original design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing The second design doc (without all the background info, and more centered on the implementation) can be found here: https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing -- 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
[jira] [Updated] (SPARK-7398) Add back-pressure to Spark Streaming
[ https://issues.apache.org/jira/browse/SPARK-7398?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-7398: - Description: Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The original design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing The second design doc (without all the background info, and more centered on the implementation) can be found here: https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing was: Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing Add back-pressure to Spark Streaming Key: SPARK-7398 URL: https://issues.apache.org/jira/browse/SPARK-7398 Project: Spark Issue Type: Improvement Components: Streaming Affects Versions: 1.3.1 Reporter: François Garillot Priority: Critical Labels: streams Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The original design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing The second design doc (without all the background info, and more centered on the implementation) can be found here: https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing -- 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
[jira] [Commented] (SPARK-7398) Add back-pressure to Spark Streaming
[ https://issues.apache.org/jira/browse/SPARK-7398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14605338#comment-14605338 ] Iulian Dragos commented on SPARK-7398: -- [~tdas] here it is: https://docs.google.com/document/d/1ls_g5fFmfbbSTIfQQpUxH56d0f3OksF567zwA00zK9E/edit?usp=sharing Add back-pressure to Spark Streaming Key: SPARK-7398 URL: https://issues.apache.org/jira/browse/SPARK-7398 Project: Spark Issue Type: Improvement Components: Streaming Affects Versions: 1.3.1 Reporter: François Garillot Priority: Critical Labels: streams Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing -- 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
[jira] [Commented] (SPARK-7944) Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path
[ https://issues.apache.org/jira/browse/SPARK-7944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14599689#comment-14599689 ] Iulian Dragos commented on SPARK-7944: -- Unfortunately I don't think there's can be any workaround. The classpath is simply ignored in the current version. Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path Key: SPARK-7944 URL: https://issues.apache.org/jira/browse/SPARK-7944 Project: Spark Issue Type: Bug Components: Spark Shell Affects Versions: 1.3.1, 1.4.0 Environment: scala 2.11 Reporter: Alexander Nakos Priority: Critical Attachments: spark_shell_output.txt, spark_shell_output_2.10.txt When I run the spark-shell with the --jars argument and supply a path to a single jar file, none of the classes in the jar are available in the REPL. I have encountered this same behaviour in both 1.3.1 and 1.4.0_RC-03 builds for scala 2.11. I have yet to do a 1.4.0 RC-03 build for scala 2.10, but the contents of the jar are available in the 1.3.1_2.10 REPL. -- 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
[jira] [Commented] (SPARK-7398) Add back-pressure to Spark Streaming
[ https://issues.apache.org/jira/browse/SPARK-7398?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14595569#comment-14595569 ] Iulian Dragos commented on SPARK-7398: -- [~tdas] thanks for looking into this. I agree that now that we have working code we can be more specific about what changes are needed. We'll prepare another doc with that focus in mind. Add back-pressure to Spark Streaming Key: SPARK-7398 URL: https://issues.apache.org/jira/browse/SPARK-7398 Project: Spark Issue Type: Improvement Components: Streaming Affects Versions: 1.3.1 Reporter: François Garillot Priority: Critical Labels: streams Spark Streaming has trouble dealing with situations where batch processing time batch interval Meaning a high throughput of input data w.r.t. Spark's ability to remove data from the queue. If this throughput is sustained for long enough, it leads to an unstable situation where the memory of the Receiver's Executor is overflowed. This aims at transmitting a back-pressure signal back to data ingestion to help with dealing with that high throughput, in a backwards-compatible way. The design doc can be found here: https://docs.google.com/document/d/1ZhiP_yBHcbjifz8nJEyPJpHqxB1FT6s8-Zk7sAfayQw/edit?usp=sharing -- 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
[jira] [Commented] (SPARK-7944) Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path
[ https://issues.apache.org/jira/browse/SPARK-7944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14592101#comment-14592101 ] Iulian Dragos commented on SPARK-7944: -- I can confirm this is working if the Spark REPL is based on a more recent version of the Scala REPL. I already have a branch that does away with 99% of the code, basically keeping only the initializeSpark and printWelcome calls. You can assign this to me. Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path Key: SPARK-7944 URL: https://issues.apache.org/jira/browse/SPARK-7944 Project: Spark Issue Type: Bug Components: Spark Shell Affects Versions: 1.3.1, 1.4.0 Environment: scala 2.11 Reporter: Alexander Nakos Priority: Critical Attachments: spark_shell_output.txt, spark_shell_output_2.10.txt When I run the spark-shell with the --jars argument and supply a path to a single jar file, none of the classes in the jar are available in the REPL. I have encountered this same behaviour in both 1.3.1 and 1.4.0_RC-03 builds for scala 2.11. I have yet to do a 1.4.0 RC-03 build for scala 2.10, but the contents of the jar are available in the 1.3.1_2.10 REPL. -- 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
[jira] [Commented] (SPARK-7944) Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path
[ https://issues.apache.org/jira/browse/SPARK-7944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14588740#comment-14588740 ] Iulian Dragos commented on SPARK-7944: -- I'll have a look tomorrow, I vaguely remember a bug in the Scala REPL that was fixed. Since the code is forked, the fix may not be in there... Spark-Shell 2.11 1.4.0-RC-03 does not add jars to class path Key: SPARK-7944 URL: https://issues.apache.org/jira/browse/SPARK-7944 Project: Spark Issue Type: Bug Components: Spark Shell Affects Versions: 1.3.1, 1.4.0 Environment: scala 2.11 Reporter: Alexander Nakos Priority: Critical Attachments: spark_shell_output.txt, spark_shell_output_2.10.txt When I run the spark-shell with the --jars argument and supply a path to a single jar file, none of the classes in the jar are available in the REPL. I have encountered this same behaviour in both 1.3.1 and 1.4.0_RC-03 builds for scala 2.11. I have yet to do a 1.4.0 RC-03 build for scala 2.10, but the contents of the jar are available in the 1.3.1_2.10 REPL. -- 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
[jira] [Commented] (SPARK-7890) Document that Spark 2.11 now supports Kafka
[ https://issues.apache.org/jira/browse/SPARK-7890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14567167#comment-14567167 ] Iulian Dragos commented on SPARK-7890: -- [~srowen] thanks for fixing it, and sorry for being unresponsive. I've been traveling a few days without a good internet connection. Document that Spark 2.11 now supports Kafka --- Key: SPARK-7890 URL: https://issues.apache.org/jira/browse/SPARK-7890 Project: Spark Issue Type: Sub-task Components: Documentation Reporter: Patrick Wendell Assignee: Sean Owen Priority: Critical Fix For: 1.4.1, 1.5.0 The building-spark.html page needs to be updated. It's a simple fix, just remove the caveat about Kafka. -- 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
[jira] [Commented] (SPARK-8013) Get JDBC server working with Scala 2.11
[ https://issues.apache.org/jira/browse/SPARK-8013?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14567561#comment-14567561 ] Iulian Dragos commented on SPARK-8013: -- There's a Scala 2.11.7 milestone due in July, hopefully we can get a solution in by then. Get JDBC server working with Scala 2.11 --- Key: SPARK-8013 URL: https://issues.apache.org/jira/browse/SPARK-8013 Project: Spark Issue Type: Sub-task Components: SQL Reporter: Patrick Wendell Assignee: Iulian Dragos Priority: Critical It's worth some investigation here, but I believe the simplest solution is to see if we can get Scala to shade it's use of JLine to avoid JLine conflicts between Hive and the Spark repl. It's also possible that there is a simpler internal solution to the conflict (I haven't looked at it in a long time). So doing some investigation of that would be good. IIRC, there is use of Jline in our own repl code, in addition to in Hive and also in the Scala 2.11 repl. Back when we created the 2.11 build I couldn't harmonize all the versions in a nice way. -- 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
[jira] [Created] (SPARK-7841) Spark build should not use lib_managed for dependencies
Iulian Dragos created SPARK-7841: Summary: Spark build should not use lib_managed for dependencies Key: SPARK-7841 URL: https://issues.apache.org/jira/browse/SPARK-7841 Project: Spark Issue Type: Bug Components: Build Affects Versions: 1.3.1 Reporter: Iulian Dragos - unnecessary duplication (I will have those libraries under ./m2, via maven anyway) - every time I call make-distribution I lose lib_managed (via mvn clean install) and have to wait to download again all jars next time I use sbt - Eclipse does not handle relative paths very well (source attachments from lib_managed don’t always work) - it's not the default configuration. If we stray from defaults I think there should be a clear advantage. Digging through history, the only reference to `retrieveManaged := true` I found was in f686e3d, from July 2011 (Initial work on converting build to SBT 0.10.1). My guess this is purely an accident of porting the build form Sbt 0.7.x and trying to keep the old project layout. If there are reasons for keeping it, please comment (I didn't get any answers on the [dev mailing list|http://apache-spark-developers-list.1001551.n3.nabble.com/Why-use-quot-lib-managed-quot-for-the-Sbt-build-td12361.html]) -- 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
[jira] [Commented] (SPARK-7726) Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher]
[ https://issues.apache.org/jira/browse/SPARK-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550142#comment-14550142 ] Iulian Dragos commented on SPARK-7726: -- This fix in 2.11.3 looks like the culprit (meaning that the error was always there, but ignored): https://issues.scala-lang.org/browse/SI-8885 Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher] - Key: SPARK-7726 URL: https://issues.apache.org/jira/browse/SPARK-7726 Project: Spark Issue Type: Bug Components: Build Reporter: Patrick Wendell Priority: Blocker This one took a long time to track down. The Maven install phase is part of our release process. It runs the scala:doc target to generate doc jars. Between Scala 2.11.2 and Scala 2.11.3, the behavior of this plugin changed in a way that breaks our build. In both cases, it returned an error (there has been a long running error here that we've always ignored), however in 2.11.3 that error became fatal and failed the entire build process. The upgrade occurred in SPARK-7092. Here is a simple reproduction: {code} ./dev/change-version-to-2.11.sh mvn clean install -pl network/common -pl network/shuffle -DskipTests -Dscala-2.11 {code} This command exits success when Spark is at Scala 2.11.2 and fails with 2.11.3 or higher. In either case an error is printed: {code} [INFO] [INFO] --- scala-maven-plugin:3.2.0:doc-jar (attach-scaladocs) @ spark-network-shuffle_2.11 --- /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java:56: error: not found: type Type protected Type type() { return Type.UPLOAD_BLOCK; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java:37: error: not found: type Type protected Type type() { return Type.STREAM_HANDLE; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java:44: error: not found: type Type protected Type type() { return Type.REGISTER_EXECUTOR; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java:40: error: not found: type Type protected Type type() { return Type.OPEN_BLOCKS; } ^ model contains 22 documentable templates four errors found {code} Ideally we'd just dig in and fix this error. Unfortunately it's a very confusing error and I have no idea why it is appearing. I'd propose reverting SPARK-7092 in the mean time. -- 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
[jira] [Commented] (SPARK-7726) Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher]
[ https://issues.apache.org/jira/browse/SPARK-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550170#comment-14550170 ] Iulian Dragos commented on SPARK-7726: -- The problem is different visibility rules in Scala and Java w.r.t to statics (and the fact that scaladoc is run on Java sources). In this particular case, `Type` is a static enumeration inherited from `BlockTransferMessage`. In Scala statics are part of the companion object, and not visible without an import. Since scaladoc needs to resolve all types it documents, this comes up as an error. I fixed this by adding a static import in all those files {code} import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; {code} {code} [INFO] [INFO] Reactor Summary: [INFO] [INFO] Spark Project Networking ... SUCCESS [ 11.395 s] [INFO] Spark Project Shuffle Streaming Service SUCCESS [ 5.460 s] [INFO] [INFO] BUILD SUCCESS [INFO] [INFO] Total time: 17.582 s [INFO] Finished at: 2015-05-19T12:07:01+02:00 [INFO] Final Memory: 42M/456M [INFO] {code} Should I open a pull request reverting the reverting commit and add this fix? Or you'd like to suppress scaladoc running on Java files (assuming that's not intentional...) Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher] - Key: SPARK-7726 URL: https://issues.apache.org/jira/browse/SPARK-7726 Project: Spark Issue Type: Bug Components: Build Reporter: Patrick Wendell Priority: Blocker This one took a long time to track down. The Maven install phase is part of our release process. It runs the scala:doc target to generate doc jars. Between Scala 2.11.2 and Scala 2.11.3, the behavior of this plugin changed in a way that breaks our build. In both cases, it returned an error (there has been a long running error here that we've always ignored), however in 2.11.3 that error became fatal and failed the entire build process. The upgrade occurred in SPARK-7092. Here is a simple reproduction: {code} ./dev/change-version-to-2.11.sh mvn clean install -pl network/common -pl network/shuffle -DskipTests -Dscala-2.11 {code} This command exits success when Spark is at Scala 2.11.2 and fails with 2.11.3 or higher. In either case an error is printed: {code} [INFO] [INFO] --- scala-maven-plugin:3.2.0:doc-jar (attach-scaladocs) @ spark-network-shuffle_2.11 --- /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java:56: error: not found: type Type protected Type type() { return Type.UPLOAD_BLOCK; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java:37: error: not found: type Type protected Type type() { return Type.STREAM_HANDLE; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java:44: error: not found: type Type protected Type type() { return Type.REGISTER_EXECUTOR; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java:40: error: not found: type Type protected Type type() { return Type.OPEN_BLOCKS; } ^ model contains 22 documentable templates four errors found {code} Ideally we'd just dig in and fix this error. Unfortunately it's a very confusing error and I have no idea why it is appearing. I'd propose reverting SPARK-7092 in the mean time. -- 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
[jira] [Comment Edited] (SPARK-7726) Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher]
[ https://issues.apache.org/jira/browse/SPARK-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550170#comment-14550170 ] Iulian Dragos edited comment on SPARK-7726 at 5/19/15 10:11 AM: The problem is different visibility rules in Scala and Java w.r.t to statics (and the fact that scaladoc is invoked on Java sources). In this particular case, `Type` is a static enumeration inherited from `BlockTransferMessage`. In Scala statics are part of the companion object, and not visible without an import. Since scaladoc needs to resolve all types it documents, this comes up as an error. I fixed this by adding a static import in all those files {code} import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; {code} {code} [INFO] [INFO] Reactor Summary: [INFO] [INFO] Spark Project Networking ... SUCCESS [ 11.395 s] [INFO] Spark Project Shuffle Streaming Service SUCCESS [ 5.460 s] [INFO] [INFO] BUILD SUCCESS [INFO] [INFO] Total time: 17.582 s [INFO] Finished at: 2015-05-19T12:07:01+02:00 [INFO] Final Memory: 42M/456M [INFO] {code} Should I open a pull request reverting the reverting commit and add this fix? Or you'd like to suppress scaladoc running on Java files (assuming that's not intentional...) was (Author: dragos): The problem is different visibility rules in Scala and Java w.r.t to statics (and the fact that scaladoc is run on Java sources). In this particular case, `Type` is a static enumeration inherited from `BlockTransferMessage`. In Scala statics are part of the companion object, and not visible without an import. Since scaladoc needs to resolve all types it documents, this comes up as an error. I fixed this by adding a static import in all those files {code} import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; {code} {code} [INFO] [INFO] Reactor Summary: [INFO] [INFO] Spark Project Networking ... SUCCESS [ 11.395 s] [INFO] Spark Project Shuffle Streaming Service SUCCESS [ 5.460 s] [INFO] [INFO] BUILD SUCCESS [INFO] [INFO] Total time: 17.582 s [INFO] Finished at: 2015-05-19T12:07:01+02:00 [INFO] Final Memory: 42M/456M [INFO] {code} Should I open a pull request reverting the reverting commit and add this fix? Or you'd like to suppress scaladoc running on Java files (assuming that's not intentional...) Maven Install Breaks When Upgrading Scala 2.11.2--[2.11.3 or higher] - Key: SPARK-7726 URL: https://issues.apache.org/jira/browse/SPARK-7726 Project: Spark Issue Type: Bug Components: Build Reporter: Patrick Wendell Priority: Blocker This one took a long time to track down. The Maven install phase is part of our release process. It runs the scala:doc target to generate doc jars. Between Scala 2.11.2 and Scala 2.11.3, the behavior of this plugin changed in a way that breaks our build. In both cases, it returned an error (there has been a long running error here that we've always ignored), however in 2.11.3 that error became fatal and failed the entire build process. The upgrade occurred in SPARK-7092. Here is a simple reproduction: {code} ./dev/change-version-to-2.11.sh mvn clean install -pl network/common -pl network/shuffle -DskipTests -Dscala-2.11 {code} This command exits success when Spark is at Scala 2.11.2 and fails with 2.11.3 or higher. In either case an error is printed: {code} [INFO] [INFO] --- scala-maven-plugin:3.2.0:doc-jar (attach-scaladocs) @ spark-network-shuffle_2.11 --- /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java:56: error: not found: type Type protected Type type() { return Type.UPLOAD_BLOCK; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java:37: error: not found: type Type protected Type type() { return Type.STREAM_HANDLE; } ^ /Users/pwendell/Documents/spark/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java:44: error: not
[jira] [Commented] (SPARK-5488) SPARK_LOCAL_IP not read by mesos scheduler
[ https://issues.apache.org/jira/browse/SPARK-5488?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550434#comment-14550434 ] Iulian Dragos commented on SPARK-5488: -- I've noticed something fishy as well, however a cursory look at the Mesos implementation doesn't show any foul play. It seems to go through `Utils.localHostName`. SPARK_LOCAL_IP not read by mesos scheduler -- Key: SPARK-5488 URL: https://issues.apache.org/jira/browse/SPARK-5488 Project: Spark Issue Type: Bug Components: Mesos, Scheduler Affects Versions: 1.1.1 Reporter: Martin Tapp Priority: Minor My environment sets SPARK_LOCAL_IP and my driver sees it. But mesos sees the one from my first available network adapter. I can even see that SPARK_LOCAL_IP is read correctly by Utils.localHostName and Utils.localIpAddress (core/src/main/scala/org/apache/spark/util/Utils.scala). Seems spark mesos framework doesn't use it. Work around for now is to disable my first adapter such that the second one becomes the one seen by Spark. -- 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
[jira] [Resolved] (SPARK-2555) Support configuration spark.scheduler.minRegisteredExecutorsRatio in Mesos mode.
[ https://issues.apache.org/jira/browse/SPARK-2555?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos resolved SPARK-2555. -- Resolution: Invalid As [commented here|https://github.com/apache/spark/pull/1462#issuecomment-76289582] it was decided to drop this feature. Support configuration spark.scheduler.minRegisteredExecutorsRatio in Mesos mode. Key: SPARK-2555 URL: https://issues.apache.org/jira/browse/SPARK-2555 Project: Spark Issue Type: Improvement Components: Mesos, Scheduler Affects Versions: 1.0.0 Reporter: Zhihui In SPARK-1946, configuration spark.scheduler.minRegisteredExecutorsRatio was introduced, but it only support Standalone and Yarn mode. This is try to introduce the configuration to Mesos mode. -- 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
[jira] [Issue Comment Deleted] (SPARK-4899) Support Mesos features: roles and checkpoints
[ https://issues.apache.org/jira/browse/SPARK-4899?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-4899: - Comment: was deleted (was: Seems so, at least.) Support Mesos features: roles and checkpoints - Key: SPARK-4899 URL: https://issues.apache.org/jira/browse/SPARK-4899 Project: Spark Issue Type: New Feature Components: Mesos Affects Versions: 1.2.0 Reporter: Andrew Ash Inspired by https://github.com/apache/spark/pull/60 Mesos has two features that would be nice for Spark to take advantage of: 1. Roles -- a way to specify ACLs and priorities for users 2. Checkpoints -- a way to restart a failed Mesos slave without losing all the work that was happening on the box Some of these may require a Mesos upgrade past our current 0.18.1 -- 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
[jira] [Commented] (SPARK-872) Should revive offer after tasks finish in Mesos fine-grained mode
[ https://issues.apache.org/jira/browse/SPARK-872?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14550473#comment-14550473 ] Iulian Dragos commented on SPARK-872: - I think we should close this due to inactivity. What is the policy w.r.t. to the status to use? Should revive offer after tasks finish in Mesos fine-grained mode -- Key: SPARK-872 URL: https://issues.apache.org/jira/browse/SPARK-872 Project: Spark Issue Type: Improvement Components: Mesos Affects Versions: 0.8.0 Reporter: xiajunluan when running spark on latest Mesos release, I notice that spark on mesos fine-grained could not schedule spark tasks effectively, for example, if slave has 4 cpu cores resource, mesos master will call resourceOffer function of spark until 4 cpu cores are all free. but In my points like standalone scheduler mode, if one task finished and one cpus core is free, Mesos master should call spark resourceOffer to allocate resource to tasks. -- 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
[jira] [Updated] (SPARK-6679) java.lang.ClassNotFoundException on Mesos fine grained mode and input replication
[ https://issues.apache.org/jira/browse/SPARK-6679?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-6679: - Description: Spark Streaming 1.3.0, Mesos 0.21.1 - Only when using fine grained mode and receiver input replication (StorageLevel.MEMORY_ONLY_2, StorageLevel.MEMORY_AND_DISK_2). When using coarse grained mode it works. When not using replication (StorageLevel.MEMORY_ONLY ...) it works. Error: {code} 15/03/26 14:50:00 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() on RPC id 7178767328921933569 java.lang.ClassNotFoundException: org/apache/spark/storage/StorageLevel at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:344) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:65) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) at org.apache.spark.serializer.JavaDeserializationStream.readObject(JavaSerializer.scala:68) at org.apache.spark.serializer.JavaSerializerInstance.deserialize(JavaSerializer.scala:88) at org.apache.spark.network.netty.NettyBlockRpcServer.receive(NettyBlockRpcServer.scala:65) at org.apache.spark.network.server.TransportRequestHandler.processRpcRequest(TransportRequestHandler.java:124) at org.apache.spark.network.server.TransportRequestHandler.handle(TransportRequestHandler.java:97) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:91) at org.apache.spark.network.server.TransportChannelHandler.channelRead0(TransportChannelHandler.java:44) at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:105) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:163) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:130) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) at java.lang.Thread.run(Thread.java:745) {code} was: Spark Streaming 1.3.0, Mesos 0.21.1 - Only when using fine grained mode and receiver input replication (StorageLevel.MEMORY_ONLY_2, StorageLevel.MEMORY_AND_DISK_2). When using coarse grained mode it works. When not using replication (StorageLevel.MEMORY_ONLY ...) it works. Error: 15/03/26 14:50:00 ERROR TransportRequestHandler: Error while invoking RpcHandler#receive() on RPC id 7178767328921933569 java.lang.ClassNotFoundException: org/apache/spark/storage/StorageLevel at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:344) at org.apache.spark.serializer.JavaDeserializationStream$$anon$1.resolveClass(JavaSerializer.scala:65) at java.io.ObjectInputStream.readNonProxyDesc(ObjectInputStream.java:1613) at java.io.ObjectInputStream.readClassDesc(ObjectInputStream.java:1518) at java.io.ObjectInputStream.readOrdinaryObject(ObjectInputStream.java:1774) at java.io.ObjectInputStream.readObject0(ObjectInputStream.java:1351) at java.io.ObjectInputStream.readObject(ObjectInputStream.java:371) at
[jira] [Commented] (SPARK-3334) Spark causes mesos-master memory leak
[ https://issues.apache.org/jira/browse/SPARK-3334?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14547888#comment-14547888 ] Iulian Dragos commented on SPARK-3334: -- This was fixed in Mesos 0.21.0. Should we close this ticket? Spark causes mesos-master memory leak - Key: SPARK-3334 URL: https://issues.apache.org/jira/browse/SPARK-3334 Project: Spark Issue Type: Bug Components: Mesos Affects Versions: 1.0.2 Environment: Mesos 0.16.0/0.19.0 CentOS 6.4 Reporter: Iven Hsu The {{akkaFrameSize}} is set to {{Long.MaxValue}} in MesosBackend to workaround SPARK-1112, this causes all serialized task result is sent using Mesos TaskStatus. mesos-master stores TaskStatus in memory, and when running Spark, its memory grows very fast, and will be OOM killed. See MESOS-1746 for more. I've tried to set {{akkaFrameSize}} to 0, mesos-master won't be killed, however, the driver will block after success unless I use {{sc.stop()}} to quit it manually. Not sure if it's related to SPARK-1112. -- 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
[jira] [Commented] (SPARK-4899) Support Mesos features: roles and checkpoints
[ https://issues.apache.org/jira/browse/SPARK-4899?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14547906#comment-14547906 ] Iulian Dragos commented on SPARK-4899: -- Seems so, at least. Support Mesos features: roles and checkpoints - Key: SPARK-4899 URL: https://issues.apache.org/jira/browse/SPARK-4899 Project: Spark Issue Type: New Feature Components: Mesos Affects Versions: 1.2.0 Reporter: Andrew Ash Inspired by https://github.com/apache/spark/pull/60 Mesos has two features that would be nice for Spark to take advantage of: 1. Roles -- a way to specify ACLs and priorities for users 2. Checkpoints -- a way to restart a failed Mesos slave without losing all the work that was happening on the box Some of these may require a Mesos upgrade past our current 0.18.1 -- 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
[jira] [Commented] (SPARK-5281) Registering table on RDD is giving MissingRequirementError
[ https://issues.apache.org/jira/browse/SPARK-5281?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14530224#comment-14530224 ] Iulian Dragos commented on SPARK-5281: -- Here's my workaround from [this stack overflow quesiton|https://stackoverflow.com/questions/29796928/whats-the-most-efficient-way-to-filter-a-dataframe] - find your launch configuration and go to Classpath - remove Scala Library and Scala Compiler from the Bootstrap entries - add (as external jars) scala-reflect, scala-library and scala-compiler to user entries Make sure to add the right version (2.10.4 at this point). Registering table on RDD is giving MissingRequirementError -- Key: SPARK-5281 URL: https://issues.apache.org/jira/browse/SPARK-5281 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.2.0, 1.3.1 Reporter: sarsol Priority: Critical Application crashes on this line {{rdd.registerTempTable(temp)}} in 1.2 version when using sbt or Eclipse SCALA IDE Stacktrace: {code} Exception in thread main scala.reflect.internal.MissingRequirementError: class org.apache.spark.sql.catalyst.ScalaReflection in JavaMirror with primordial classloader with boot classpath [C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-library.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-reflect.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-actor.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-swing.jar;C:\sar\scala\scala-ide\eclipse\plugins\org.scala-ide.scala210.jars_4.0.0.201407240952\target\jars\scala-compiler.jar;C:\Program Files\Java\jre7\lib\resources.jar;C:\Program Files\Java\jre7\lib\rt.jar;C:\Program Files\Java\jre7\lib\sunrsasign.jar;C:\Program Files\Java\jre7\lib\jsse.jar;C:\Program Files\Java\jre7\lib\jce.jar;C:\Program Files\Java\jre7\lib\charsets.jar;C:\Program Files\Java\jre7\lib\jfr.jar;C:\Program Files\Java\jre7\classes] not found. at scala.reflect.internal.MissingRequirementError$.signal(MissingRequirementError.scala:16) at scala.reflect.internal.MissingRequirementError$.notFound(MissingRequirementError.scala:17) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:48) at scala.reflect.internal.Mirrors$RootsBase.getModuleOrClass(Mirrors.scala:61) at scala.reflect.internal.Mirrors$RootsBase.staticModuleOrClass(Mirrors.scala:72) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:119) at scala.reflect.internal.Mirrors$RootsBase.staticClass(Mirrors.scala:21) at org.apache.spark.sql.catalyst.ScalaReflection$$typecreator1$1.apply(ScalaReflection.scala:115) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe$lzycompute(TypeTags.scala:231) at scala.reflect.api.TypeTags$WeakTypeTagImpl.tpe(TypeTags.scala:231) at scala.reflect.api.TypeTags$class.typeOf(TypeTags.scala:335) at scala.reflect.api.Universe.typeOf(Universe.scala:59) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:115) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.schemaFor(ScalaReflection.scala:100) at org.apache.spark.sql.catalyst.ScalaReflection$.schemaFor(ScalaReflection.scala:33) at org.apache.spark.sql.catalyst.ScalaReflection$class.attributesFor(ScalaReflection.scala:94) at org.apache.spark.sql.catalyst.ScalaReflection$.attributesFor(ScalaReflection.scala:33) at org.apache.spark.sql.SQLContext.createSchemaRDD(SQLContext.scala:111) at com.sar.spark.dq.poc.SparkPOC$delayedInit$body.apply(SparkPOC.scala:43) at scala.Function0$class.apply$mcV$sp(Function0.scala:40) at scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.App$$anonfun$main$1.apply(App.scala:71) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:32) at scala.App$class.main(App.scala:71) {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
[jira] [Created] (SPARK-7399) Master fails on 2.11 with compilation error
Iulian Dragos created SPARK-7399: Summary: Master fails on 2.11 with compilation error Key: SPARK-7399 URL: https://issues.apache.org/jira/browse/SPARK-7399 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 1.4.0 Reporter: Iulian Dragos The current code in master (and 1.4 branch) fails on 2.11 with the following compilation error: {code} [error] /home/ubuntu/workspace/Apache Spark (master) on 2.11/core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala:78: in object RDDOperationScope, multiple overloaded alternatives of method withScope define default arguments. [error] private[spark] object RDDOperationScope { [error] ^ {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
[jira] [Commented] (SPARK-6285) Duplicated code leads to errors
[ https://issues.apache.org/jira/browse/SPARK-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14360937#comment-14360937 ] Iulian Dragos commented on SPARK-6285: -- Thanks, [~lian cheng] Duplicated code leads to errors --- Key: SPARK-6285 URL: https://issues.apache.org/jira/browse/SPARK-6285 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.3.0 Reporter: Iulian Dragos Assignee: Cheng Lian The following class is duplicated inside [ParquetTestData|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala#L39] and [ParquetIOSuite|https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala#L44], with exact same code and fully qualified name: {code} org.apache.spark.sql.parquet.TestGroupWriteSupport {code} The second one was introduced in [3b395e10|https://github.com/apache/spark/commit/3b395e10510782474789c9098084503f98ca4830], but even though it mentions that `ParquetTestData` should be removed later, I couldn't find a corresponding Jira ticket. This duplicate class causes the Eclipse builder to fail (since src/main and src/test are compiled together in Eclipse, unlike Sbt). -- 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
[jira] [Commented] (SPARK-6286) Handle TASK_ERROR in TaskState
[ https://issues.apache.org/jira/browse/SPARK-6286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14357310#comment-14357310 ] Iulian Dragos commented on SPARK-6286: -- Good point. It's been [introduced in 0.21.0|http://mesos.apache.org/blog/mesos-0-21-0-released/]. According to [pom.xml|https://github.com/apache/spark/blob/master/pom.xml#L119], Spark depends on `0.21.0`, so it seems safe to handle it. Feel free to close if you think it's going to break something else. Handle TASK_ERROR in TaskState -- Key: SPARK-6286 URL: https://issues.apache.org/jira/browse/SPARK-6286 Project: Spark Issue Type: Bug Components: Spark Core Reporter: Iulian Dragos Priority: Minor Labels: mesos Scala warning: {code} match may not be exhaustive. It would fail on the following input: TASK_ERROR {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
[jira] [Commented] (SPARK-6286) Handle TASK_ERROR in TaskState
[ https://issues.apache.org/jira/browse/SPARK-6286?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14358905#comment-14358905 ] Iulian Dragos commented on SPARK-6286: -- Sure, I'll issue a PR for handling {{TASK_ERROR = TASK_LOST}} Handle TASK_ERROR in TaskState -- Key: SPARK-6286 URL: https://issues.apache.org/jira/browse/SPARK-6286 Project: Spark Issue Type: Bug Components: Spark Core Reporter: Iulian Dragos Priority: Minor Labels: mesos Scala warning: {code} match may not be exhaustive. It would fail on the following input: TASK_ERROR {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
[jira] [Created] (SPARK-6286) Handle TASK_ERROR in TaskState
Iulian Dragos created SPARK-6286: Summary: Handle TASK_ERROR in TaskState Key: SPARK-6286 URL: https://issues.apache.org/jira/browse/SPARK-6286 Project: Spark Issue Type: Bug Components: Spark Core Reporter: Iulian Dragos Priority: Minor Scala warning: {code} match may not be exhaustive. It would fail on the following input: TASK_ERROR {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
[jira] [Updated] (SPARK-6286) Handle TASK_ERROR in TaskState
[ https://issues.apache.org/jira/browse/SPARK-6286?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Iulian Dragos updated SPARK-6286: - Labels: mesos (was: ) Handle TASK_ERROR in TaskState -- Key: SPARK-6286 URL: https://issues.apache.org/jira/browse/SPARK-6286 Project: Spark Issue Type: Bug Components: Spark Core Reporter: Iulian Dragos Priority: Minor Labels: mesos Scala warning: {code} match may not be exhaustive. It would fail on the following input: TASK_ERROR {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
[jira] [Created] (SPARK-6285) Duplicated code leads to errors
Iulian Dragos created SPARK-6285: Summary: Duplicated code leads to errors Key: SPARK-6285 URL: https://issues.apache.org/jira/browse/SPARK-6285 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.3.0 Reporter: Iulian Dragos The following class is duplicated inside [ParquetTestData|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala#L39] and [ParquetIOSuite|https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala#L44], with exact same code and fully qualified name: {code} org.apache.spark.sql.parquet.TestGroupWriteSupport {code} The second one was introduced in [3b395e10|https://github.com/apache/spark/commit/3b395e10510782474789c9098084503f98ca4830], but even though it mentions that `ParquetTestData` should be removed later, I couldn't find a corresponding Jira ticket. This duplicate class causes the Eclipse builder to fail (since src/main and src/test are compiled together in Eclipse, unlike Sbt). -- 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
[jira] [Commented] (SPARK-6285) Duplicated code leads to errors
[ https://issues.apache.org/jira/browse/SPARK-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14357297#comment-14357297 ] Iulian Dragos commented on SPARK-6285: -- According to the git commit message that introduced the duplicate: {quote} To avoid potential merge conflicts, old testing code are not removed yet. The following classes can be safely removed after most Parquet related PRs are handled: - `ParquetQuerySuite` - `ParquetTestData` {quote} I mentioned the Eclipse build problem in passing, but I can expand: the class *is* a duplicated name, so the Scala compiler is correct in refusing it. It only compiles in Sbt/Maven because the src/main and src/test are compiled in separate compiler runs, and scalac seems to not notice the duplicate name when it comes from bytecode. Eclipse builds src/main and src/test together, and when both classes originate from sources scalac issues an error message. Duplicated code leads to errors --- Key: SPARK-6285 URL: https://issues.apache.org/jira/browse/SPARK-6285 Project: Spark Issue Type: Bug Components: SQL Affects Versions: 1.3.0 Reporter: Iulian Dragos The following class is duplicated inside [ParquetTestData|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala#L39] and [ParquetIOSuite|https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala#L44], with exact same code and fully qualified name: {code} org.apache.spark.sql.parquet.TestGroupWriteSupport {code} The second one was introduced in [3b395e10|https://github.com/apache/spark/commit/3b395e10510782474789c9098084503f98ca4830], but even though it mentions that `ParquetTestData` should be removed later, I couldn't find a corresponding Jira ticket. This duplicate class causes the Eclipse builder to fail (since src/main and src/test are compiled together in Eclipse, unlike Sbt). -- 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
[jira] [Created] (SPARK-6287) Add support for dynamic allocation in the Mesos coarse-grained scheduler
Iulian Dragos created SPARK-6287: Summary: Add support for dynamic allocation in the Mesos coarse-grained scheduler Key: SPARK-6287 URL: https://issues.apache.org/jira/browse/SPARK-6287 Project: Spark Issue Type: Bug Components: Mesos Reporter: Iulian Dragos Add support inside the coarse-grained Mesos scheduler for dynamic allocation. It amounts to implementing two methods that allow scaling up and down the number of executors: {code} def doKillExecutors(executorIds: Seq[String]) def doRequestTotalExecutors(requestedTotal: Int) {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
[jira] [Commented] (SPARK-6005) Flaky test: o.a.s.streaming.kafka.DirectKafkaStreamSuite.offset recovery
[ https://issues.apache.org/jira/browse/SPARK-6005?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14336542#comment-14336542 ] Iulian Dragos commented on SPARK-6005: -- Looks similar, but unless I miss something, that fix didn't fix this one. The failure I report is from a recent build that includes the fixes in 3912d332464dcd124c60b734724c34d9742466a4 Flaky test: o.a.s.streaming.kafka.DirectKafkaStreamSuite.offset recovery Key: SPARK-6005 URL: https://issues.apache.org/jira/browse/SPARK-6005 Project: Spark Issue Type: Bug Components: Streaming Reporter: Iulian Dragos Labels: flaky-test, kafka, streaming [Link to failing test on Jenkins|https://ci.typesafe.com/view/Spark/job/spark-nightly-build/lastCompletedBuild/testReport/org.apache.spark.streaming.kafka/DirectKafkaStreamSuite/offset_recovery/] {code} The code passed to eventually never returned normally. Attempted 208 times over 10.00622791 seconds. Last failure message: strings.forall({ ((elem: Any) = DirectKafkaStreamSuite.collectedData.contains(elem)) }) was false. {code} {code:title=Stack trace} sbt.ForkMain$ForkError: The code passed to eventually never returned normally. Attempted 208 times over 10.00622791 seconds. Last failure message: strings.forall({ ((elem: Any) = DirectKafkaStreamSuite.collectedData.contains(elem)) }) was false. at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) at org.apache.spark.streaming.kafka.KafkaStreamSuiteBase.eventually(KafkaStreamSuite.scala:49) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307) at org.apache.spark.streaming.kafka.KafkaStreamSuiteBase.eventually(KafkaStreamSuite.scala:49) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.org$apache$spark$streaming$kafka$DirectKafkaStreamSuite$$anonfun$$sendDataAndWaitForReceive$1(DirectKafkaStreamSuite.scala:225) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply$mcV$sp(DirectKafkaStreamSuite.scala:287) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply(DirectKafkaStreamSuite.scala:211) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply(DirectKafkaStreamSuite.scala:211) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.scalatest.Suite$class.withFixture(Suite.scala:1122) at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite.org$scalatest$BeforeAndAfter$$super$runTest(DirectKafkaStreamSuite.scala:39) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite.runTest(DirectKafkaStreamSuite.scala:39) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:318) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at
[jira] [Created] (SPARK-6005) Flaky test: o.a.s.s.k.DirectKafkaStreamSuite.offset recovery
Iulian Dragos created SPARK-6005: Summary: Flaky test: o.a.s.s.k.DirectKafkaStreamSuite.offset recovery Key: SPARK-6005 URL: https://issues.apache.org/jira/browse/SPARK-6005 Project: Spark Issue Type: Bug Components: Streaming Reporter: Iulian Dragos [Link to failing test on Jenkins|https://ci.typesafe.com/view/Spark/job/spark-nightly-build/lastCompletedBuild/testReport/org.apache.spark.streaming.kafka/DirectKafkaStreamSuite/offset_recovery/] {code} The code passed to eventually never returned normally. Attempted 208 times over 10.00622791 seconds. Last failure message: strings.forall({ ((elem: Any) = DirectKafkaStreamSuite.collectedData.contains(elem)) }) was false. {code} {code:title=Stack trace} sbt.ForkMain$ForkError: The code passed to eventually never returned normally. Attempted 208 times over 10.00622791 seconds. Last failure message: strings.forall({ ((elem: Any) = DirectKafkaStreamSuite.collectedData.contains(elem)) }) was false. at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) at org.apache.spark.streaming.kafka.KafkaStreamSuiteBase.eventually(KafkaStreamSuite.scala:49) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307) at org.apache.spark.streaming.kafka.KafkaStreamSuiteBase.eventually(KafkaStreamSuite.scala:49) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.org$apache$spark$streaming$kafka$DirectKafkaStreamSuite$$anonfun$$sendDataAndWaitForReceive$1(DirectKafkaStreamSuite.scala:225) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply$mcV$sp(DirectKafkaStreamSuite.scala:287) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply(DirectKafkaStreamSuite.scala:211) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite$$anonfun$5.apply(DirectKafkaStreamSuite.scala:211) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.scalatest.Suite$class.withFixture(Suite.scala:1122) at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite.org$scalatest$BeforeAndAfter$$super$runTest(DirectKafkaStreamSuite.scala:39) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite.runTest(DirectKafkaStreamSuite.scala:39) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:318) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at org.apache.spark.streaming.kafka.DirectKafkaStreamSuite.org$scalatest$BeforeAndAfter$$super$run(DirectKafkaStreamSuite.scala:39) at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) at