Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
I will cut a new one once https://github.com/apache/spark/pull/15774 gets in. On Fri, Nov 4, 2016 at 11:44 AM, Sean Owen wrote: > I guess it's worth explicitly stating that I think we need another RC one > way or the other because this test seems to consistently fail. It was a > (surprising) last-minute regression. I think I'd have to say -1 only for > this. > > Reverting https://github.com/apache/spark/pull/15706 for branch-2.0 would > unblock this. There's also some discussion about an alternative resolution > for the test problem. > > > On Wed, Nov 2, 2016 at 5:44 PM Sean Owen wrote: > >> Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here >> are the 4 issues still open: >> >> SPARK-14387 Enable Hive-1.x ORC compatibility with spark.sql.hive. >> convertMetastoreOrc >> SPARK-17957 Calling outer join and na.fill(0) and then inner join will >> miss rows >> SPARK-17981 Incorrectly Set Nullability to False in FilterExec >> SPARK-18160 spark.files & spark.jars should not be passed to driver in >> yarn mode >> >> Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on >> Ubuntu 16, I am seeing consistent failures in this test below. I think we >> very recently changed this so it could be legitimate. But does anyone else >> see something like this? I have seen other failures in this test due to OOM >> but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. >> >> >> - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** >> FAILED *** >> isContain was true Interpreter output contained 'Exception': >> Welcome to >> __ >>/ __/__ ___ _/ /__ >> _\ \/ _ \/ _ `/ __/ '_/ >> /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 >> /_/ >> >> Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) >> Type in expressions to have them evaluated. >> Type :help for more information. >> >> scala> >> scala> keyValueGrouped: org.apache.spark.sql. >> KeyValueGroupedDataset[Int,(Int, Int)] = org.apache.spark.sql. >> KeyValueGroupedDataset@70c30f72 >> >> scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, >> _2: int] >> >> scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = >> Broadcast(0) >> >> scala> >> scala> >> scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] >> >> scala> org.apache.spark.SparkException: Job aborted due to stage >> failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task >> 0.0 in stage 0.0 (TID 0, localhost): >> com.google.common.util.concurrent.ExecutionError: >> java.lang.ClassCircularityError: io/netty/util/internal/__ >> matchers__/org/apache/spark/network/protocol/MessageMatcher >> at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2261) >> at com.google.common.cache.LocalCache.get(LocalCache.java:4000) >> at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) >> at com.google.common.cache.LocalCache$LocalLoadingCache. >> get(LocalCache.java:4874) >> at org.apache.spark.sql.catalyst.expressions.codegen. >> CodeGenerator$.compile(CodeGenerator.scala:841) >> at org.apache.spark.sql.catalyst.expressions.codegen. >> GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) >> at org.apache.spark.sql.catalyst.expressions.codegen. >> GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) >> at org.apache.spark.sql.catalyst.expressions.codegen. >> CodeGenerator.generate(CodeGenerator.scala:825) >> at org.apache.spark.sql.catalyst.expressions.codegen. >> CodeGenerator.generate(CodeGenerator.scala:822) >> at org.apache.spark.sql.execution.ObjectOperator$. >> deserializeRowToObject(objects.scala:137) >> at org.apache.spark.sql.execution.AppendColumnsExec$$ >> anonfun$9.apply(objects.scala:251) >> at org.apache.spark.sql.execution.AppendColumnsExec$$ >> anonfun$9.apply(objects.scala:250) >> at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$ >> 1$$anonfun$apply$24.apply(RDD.scala:803) >> at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$ >> 1$$anonfun$apply$24.apply(RDD.scala:803) >> at org.apache.spark.rdd.MapPartitionsRDD.compute( >> MapPartitionsRDD.scala:38) >> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) >> at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) >> at org.apache.spark.rdd.MapPartitionsRDD.compute( >> MapPartitionsRDD.scala:38) >> at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) >> at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) >> at org.apache.spark.scheduler.ShuffleMapTask.runTask( >> ShuffleMapTask.scala:79) >> at org.apache.spark.scheduler.ShuffleMapTask.runTask( >> ShuffleMapTask.scala:47) >> at org.apache.spark.scheduler.Task.run(Task.scala:86) >> at org.apache.spark.executor.Executor$TaskRunner.run( >> Executor.scala:274) >> at java.util.concurrent.ThreadPoolExecutor.runWorker( >> ThreadPoolExecutor.java:1142) >> at java.util.
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
+1 On Fri, Nov 4, 2016 at 7:20 PM, Michael Armbrust wrote: > +1 > > On Tue, Nov 1, 2016 at 9:51 PM, Reynold Xin wrote: > >> Please vote on releasing the following candidate as Apache Spark version >> 2.0.2. The vote is open until Fri, Nov 4, 2016 at 22:00 PDT and passes if a >> majority of at least 3+1 PMC votes are cast. >> >> [ ] +1 Release this package as Apache Spark 2.0.2 >> [ ] -1 Do not release this package because ... >> >> >> The tag to be voted on is v2.0.2-rc2 (a6abe1ee22141931614bf27a4f371 >> c46d8379e33) >> >> This release candidate resolves 84 issues: https://s.apache.org/spark-2.0 >> .2-jira >> >> The release files, including signatures, digests, etc. can be found at: >> http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-bin/ >> >> Release artifacts are signed with the following key: >> https://people.apache.org/keys/committer/pwendell.asc >> >> The staging repository for this release can be found at: >> https://repository.apache.org/content/repositories/orgapachespark-1210/ >> >> The documentation corresponding to this release can be found at: >> http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-docs/ >> >> >> Q: How can I help test this release? >> A: If you are a Spark user, you can help us test this release by taking >> an existing Spark workload and running on this release candidate, then >> reporting any regressions from 2.0.1. >> >> Q: What justifies a -1 vote for this release? >> A: This is a maintenance release in the 2.0.x series. Bugs already >> present in 2.0.1, missing features, or bugs related to new features will >> not necessarily block this release. >> >> Q: What fix version should I use for patches merging into branch-2.0 from >> now on? >> A: Please mark the fix version as 2.0.3, rather than 2.0.2. If a new RC >> (i.e. RC3) is cut, I will change the fix version of those patches to 2.0.2. >> > >
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
I guess it's worth explicitly stating that I think we need another RC one way or the other because this test seems to consistently fail. It was a (surprising) last-minute regression. I think I'd have to say -1 only for this. Reverting https://github.com/apache/spark/pull/15706 for branch-2.0 would unblock this. There's also some discussion about an alternative resolution for the test problem. On Wed, Nov 2, 2016 at 5:44 PM Sean Owen wrote: > Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here > are the 4 issues still open: > > SPARK-14387 Enable Hive-1.x ORC compatibility with > spark.sql.hive.convertMetastoreOrc > SPARK-17957 Calling outer join and na.fill(0) and then inner join will > miss rows > SPARK-17981 Incorrectly Set Nullability to False in FilterExec > SPARK-18160 spark.files & spark.jars should not be passed to driver in > yarn mode > > Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on > Ubuntu 16, I am seeing consistent failures in this test below. I think we > very recently changed this so it could be legitimate. But does anyone else > see something like this? I have seen other failures in this test due to OOM > but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. > > > - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** > FAILED *** > isContain was true Interpreter output contained 'Exception': > Welcome to > __ >/ __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ > /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 > /_/ > > Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) > Type in expressions to have them evaluated. > Type :help for more information. > > scala> > scala> keyValueGrouped: > org.apache.spark.sql.KeyValueGroupedDataset[Int,(Int, Int)] = > org.apache.spark.sql.KeyValueGroupedDataset@70c30f72 > > scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, > _2: int] > > scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = > Broadcast(0) > > scala> > scala> > scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] > > scala> org.apache.spark.SparkException: Job aborted due to stage > failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task > 0.0 in stage 0.0 (TID 0, localhost): > com.google.common.util.concurrent.ExecutionError: > java.lang.ClassCircularityError: > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher > at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2261) > at com.google.common.cache.LocalCache.get(LocalCache.java:4000) > at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) > at > com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:841) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:825) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:822) > at > org.apache.spark.sql.execution.ObjectOperator$.deserializeRowToObject(objects.scala:137) > at > org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:251) > at > org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:250) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.ClassCircularityError: > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher > at java.lang.Clas
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
+1 On Fri, Nov 4, 2016 at 11:20 AM, Michael Armbrust wrote: > +1 > > On Tue, Nov 1, 2016 at 9:51 PM, Reynold Xin wrote: > >> Please vote on releasing the following candidate as Apache Spark version >> 2.0.2. The vote is open until Fri, Nov 4, 2016 at 22:00 PDT and passes if a >> majority of at least 3+1 PMC votes are cast. >> >> [ ] +1 Release this package as Apache Spark 2.0.2 >> [ ] -1 Do not release this package because ... >> >> >> The tag to be voted on is v2.0.2-rc2 (a6abe1ee22141931614bf27a4f371 >> c46d8379e33) >> >> This release candidate resolves 84 issues: https://s.apache.org/spark-2.0 >> .2-jira >> >> The release files, including signatures, digests, etc. can be found at: >> http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-bin/ >> >> Release artifacts are signed with the following key: >> https://people.apache.org/keys/committer/pwendell.asc >> >> The staging repository for this release can be found at: >> https://repository.apache.org/content/repositories/orgapachespark-1210/ >> >> The documentation corresponding to this release can be found at: >> http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-docs/ >> >> >> Q: How can I help test this release? >> A: If you are a Spark user, you can help us test this release by taking >> an existing Spark workload and running on this release candidate, then >> reporting any regressions from 2.0.1. >> >> Q: What justifies a -1 vote for this release? >> A: This is a maintenance release in the 2.0.x series. Bugs already >> present in 2.0.1, missing features, or bugs related to new features will >> not necessarily block this release. >> >> Q: What fix version should I use for patches merging into branch-2.0 from >> now on? >> A: Please mark the fix version as 2.0.3, rather than 2.0.2. If a new RC >> (i.e. RC3) is cut, I will change the fix version of those patches to 2.0.2. >> > >
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
+1 On Tue, Nov 1, 2016 at 9:51 PM, Reynold Xin wrote: > Please vote on releasing the following candidate as Apache Spark version > 2.0.2. The vote is open until Fri, Nov 4, 2016 at 22:00 PDT and passes if a > majority of at least 3+1 PMC votes are cast. > > [ ] +1 Release this package as Apache Spark 2.0.2 > [ ] -1 Do not release this package because ... > > > The tag to be voted on is v2.0.2-rc2 (a6abe1ee22141931614bf27a4f371c > 46d8379e33) > > This release candidate resolves 84 issues: https://s.apache.org/spark-2. > 0.2-jira > > The release files, including signatures, digests, etc. can be found at: > http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-bin/ > > Release artifacts are signed with the following key: > https://people.apache.org/keys/committer/pwendell.asc > > The staging repository for this release can be found at: > https://repository.apache.org/content/repositories/orgapachespark-1210/ > > The documentation corresponding to this release can be found at: > http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-docs/ > > > Q: How can I help test this release? > A: If you are a Spark user, you can help us test this release by taking an > existing Spark workload and running on this release candidate, then > reporting any regressions from 2.0.1. > > Q: What justifies a -1 vote for this release? > A: This is a maintenance release in the 2.0.x series. Bugs already present > in 2.0.1, missing features, or bugs related to new features will not > necessarily block this release. > > Q: What fix version should I use for patches merging into branch-2.0 from > now on? > A: Please mark the fix version as 2.0.3, rather than 2.0.2. If a new RC > (i.e. RC3) is cut, I will change the fix version of those patches to 2.0.2. >
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
+1 On Tue, Nov 1, 2016 at 9:51 PM, Reynold Xin wrote: > Please vote on releasing the following candidate as Apache Spark version > 2.0.2. The vote is open until Fri, Nov 4, 2016 at 22:00 PDT and passes if a > majority of at least 3+1 PMC votes are cast. > > [ ] +1 Release this package as Apache Spark 2.0.2 > [ ] -1 Do not release this package because ... > > > The tag to be voted on is v2.0.2-rc2 (a6abe1ee22141931614bf27a4f371c > 46d8379e33) > > This release candidate resolves 84 issues: https://s.apache.org/spark-2. > 0.2-jira > > The release files, including signatures, digests, etc. can be found at: > http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-bin/ > > Release artifacts are signed with the following key: > https://people.apache.org/keys/committer/pwendell.asc > > The staging repository for this release can be found at: > https://repository.apache.org/content/repositories/orgapachespark-1210/ > > The documentation corresponding to this release can be found at: > http://people.apache.org/~pwendell/spark-releases/spark-2.0.2-rc2-docs/ > > > Q: How can I help test this release? > A: If you are a Spark user, you can help us test this release by taking an > existing Spark workload and running on this release candidate, then > reporting any regressions from 2.0.1. > > Q: What justifies a -1 vote for this release? > A: This is a maintenance release in the 2.0.x series. Bugs already present > in 2.0.1, missing features, or bugs related to new features will not > necessarily block this release. > > Q: What fix version should I use for patches merging into branch-2.0 from > now on? > A: Please mark the fix version as 2.0.3, rather than 2.0.2. If a new RC > (i.e. RC3) is cut, I will change the fix version of those patches to 2.0.2. >
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
I'm seeing the same failure but manifesting itself as a stackoverflow, various operating systems and architectures (RHEL 71, CentOS 72, SUSE 12, Ubuntu 14 04 and 16 04 LTS) Build and test options: mvn -T 1C -Psparkr -Pyarn -Phadoop-2.7 -Phive -Phive-thriftserver -DskipTests clean package mvn -Pyarn -Phadoop-2.7 -Phive -Phive-thriftserver -Dtest.exclude.tags=org.apache.spark.tags.DockerTest -fn test -Xss2048k -Dspark.buffer.pageSize=1048576 -Xmx4g Stacktrace (this is with IBM's latest SDK for Java 8): scala> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): com.google.common.util.concurrent.ExecutionError: java.lang.StackOverflowError: operating system stack overflow at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2261) at com.google.common.cache.LocalCache.get(LocalCache.java:4000) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:849) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:833) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:830) at org.apache.spark.sql.execution.ObjectOperator$.deserializeRowToObject(objects.scala:137) ... omitted the rest for brevity Would also be useful to include this small but useful change that looks to have only just missed the cut: https://github.com/apache/spark/pull/14409 From: Reynold Xin To: Dongjoon Hyun Cc: "dev@spark.apache.org" Date: 02/11/2016 18:37 Subject: Re: [VOTE] Release Apache Spark 2.0.2 (RC2) Looks like there is an issue with Maven (likely just the test itself though). We should look into it. On Wed, Nov 2, 2016 at 11:32 AM, Dongjoon Hyun wrote: Hi, Sean. The same failure blocks me, too. - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** FAILED *** I used `-Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver -Dsparkr` on CentOS 7 / OpenJDK1.8.0_111. Dongjoon. On 2016-11-02 10:44 (-0700), Sean Owen wrote: > Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here are > the 4 issues still open: > > SPARK-14387 Enable Hive-1.x ORC compatibility with > spark.sql.hive.convertMetastoreOrc > SPARK-17957 Calling outer join and na.fill(0) and then inner join will miss > rows > SPARK-17981 Incorrectly Set Nullability to False in FilterExec > SPARK-18160 spark.files & spark.jars should not be passed to driver in yarn > mode > > Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on > Ubuntu 16, I am seeing consistent failures in this test below. I think we > very recently changed this so it could be legitimate. But does anyone else > see something like this? I have seen other failures in this test due to OOM > but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. > > > - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** FAILED > *** > isContain was true Interpreter output contained 'Exception': > Welcome to > __ >/ __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ > /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 > /_/ > > Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) > Type in expressions to have them evaluated. > Type :help for more information. > > scala> > scala> keyValueGrouped: > org.apache.spark.sql.KeyValueGroupedDataset[Int,(Int, Int)] = > org.apache.spark.sql.KeyValueGroupedDataset@70c30f72 > > scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, > _2: int] > > scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = > Broadcast(0) > > scala> > scala> > scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] > > scala> org.apache.spark.SparkException: Job aborted due to stage failure: > Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in > stage 0.0 (TID 0, localhost): > com.google.common.util.concurrent.ExecutionError: > java.lang.ClassCircularityError: > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher > at com.google.common.cache.Loc
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Looks like there is an issue with Maven (likely just the test itself though). We should look into it. On Wed, Nov 2, 2016 at 11:32 AM, Dongjoon Hyun wrote: > Hi, Sean. > > The same failure blocks me, too. > > - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** > FAILED *** > > I used `-Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver > -Dsparkr` on CentOS 7 / OpenJDK1.8.0_111. > > Dongjoon. > > On 2016-11-02 10:44 (-0700), Sean Owen wrote: > > Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here > are > > the 4 issues still open: > > > > SPARK-14387 Enable Hive-1.x ORC compatibility with > > spark.sql.hive.convertMetastoreOrc > > SPARK-17957 Calling outer join and na.fill(0) and then inner join will > miss > > rows > > SPARK-17981 Incorrectly Set Nullability to False in FilterExec > > SPARK-18160 spark.files & spark.jars should not be passed to driver in > yarn > > mode > > > > Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on > > Ubuntu 16, I am seeing consistent failures in this test below. I think we > > very recently changed this so it could be legitimate. But does anyone > else > > see something like this? I have seen other failures in this test due to > OOM > > but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. > > > > > > - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** > FAILED > > *** > > isContain was true Interpreter output contained 'Exception': > > Welcome to > > __ > >/ __/__ ___ _/ /__ > > _\ \/ _ \/ _ `/ __/ '_/ > > /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 > > /_/ > > > > Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) > > Type in expressions to have them evaluated. > > Type :help for more information. > > > > scala> > > scala> keyValueGrouped: > > org.apache.spark.sql.KeyValueGroupedDataset[Int,(Int, Int)] = > > org.apache.spark.sql.KeyValueGroupedDataset@70c30f72 > > > > scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, > > _2: int] > > > > scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = > > Broadcast(0) > > > > scala> > > scala> > > scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] > > > > scala> org.apache.spark.SparkException: Job aborted due to stage > failure: > > Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in > > stage 0.0 (TID 0, localhost): > > com.google.common.util.concurrent.ExecutionError: > > java.lang.ClassCircularityError: > > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/ > MessageMatcher > > at com.google.common.cache.LocalCache$Segment.get( > LocalCache.java:2261) > > at com.google.common.cache.LocalCache.get(LocalCache.java:4000) > > at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) > > at > > com.google.common.cache.LocalCache$LocalLoadingCache. > get(LocalCache.java:4874) > > at > > org.apache.spark.sql.catalyst.expressions.codegen. > CodeGenerator$.compile(CodeGenerator.scala:841) > > at > > org.apache.spark.sql.catalyst.expressions.codegen. > GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) > > at > > org.apache.spark.sql.catalyst.expressions.codegen. > GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) > > at > > org.apache.spark.sql.catalyst.expressions.codegen. > CodeGenerator.generate(CodeGenerator.scala:825) > > at > > org.apache.spark.sql.catalyst.expressions.codegen. > CodeGenerator.generate(CodeGenerator.scala:822) > > at > > org.apache.spark.sql.execution.ObjectOperator$.deserializeRowToObject( > objects.scala:137) > > at > > org.apache.spark.sql.execution.AppendColumnsExec$$ > anonfun$9.apply(objects.scala:251) > > at > > org.apache.spark.sql.execution.AppendColumnsExec$$ > anonfun$9.apply(objects.scala:250) > > at > > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$ > 1$$anonfun$apply$24.apply(RDD.scala:803) > > at > > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$ > 1$$anonfun$apply$24.apply(RDD.scala:803) > > at > > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > > at > > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > > at > > org.apache.spark.scheduler.ShuffleMapTask.runTask( > ShuffleMapTask.scala:79) > > at > > org.apache.spark.scheduler.ShuffleMapTask.runTask( > ShuffleMapTask.scala:47) > > at org.apache.spark.scheduler.Task.run(Task.scala:86) > > at org.apache.spark.executor.Executor$TaskRunner.run( > Executor.scala:274) > > at > > java.util.concurrent.ThreadPoolExecutor.runWorker( > ThreadPoolExecutor.java:1142) > > at
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Hi, Sean. The same failure blocks me, too. - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** FAILED *** I used `-Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver -Dsparkr` on CentOS 7 / OpenJDK1.8.0_111. Dongjoon. On 2016-11-02 10:44 (-0700), Sean Owen wrote: > Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here are > the 4 issues still open: > > SPARK-14387 Enable Hive-1.x ORC compatibility with > spark.sql.hive.convertMetastoreOrc > SPARK-17957 Calling outer join and na.fill(0) and then inner join will miss > rows > SPARK-17981 Incorrectly Set Nullability to False in FilterExec > SPARK-18160 spark.files & spark.jars should not be passed to driver in yarn > mode > > Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on > Ubuntu 16, I am seeing consistent failures in this test below. I think we > very recently changed this so it could be legitimate. But does anyone else > see something like this? I have seen other failures in this test due to OOM > but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. > > > - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** FAILED > *** > isContain was true Interpreter output contained 'Exception': > Welcome to > __ >/ __/__ ___ _/ /__ > _\ \/ _ \/ _ `/ __/ '_/ > /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 > /_/ > > Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) > Type in expressions to have them evaluated. > Type :help for more information. > > scala> > scala> keyValueGrouped: > org.apache.spark.sql.KeyValueGroupedDataset[Int,(Int, Int)] = > org.apache.spark.sql.KeyValueGroupedDataset@70c30f72 > > scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, > _2: int] > > scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = > Broadcast(0) > > scala> > scala> > scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] > > scala> org.apache.spark.SparkException: Job aborted due to stage failure: > Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in > stage 0.0 (TID 0, localhost): > com.google.common.util.concurrent.ExecutionError: > java.lang.ClassCircularityError: > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher > at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2261) > at com.google.common.cache.LocalCache.get(LocalCache.java:4000) > at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) > at > com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:841) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) > at > org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:825) > at > org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:822) > at > org.apache.spark.sql.execution.ObjectOperator$.deserializeRowToObject(objects.scala:137) > at > org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:251) > at > org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:250) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) > at > org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > at > org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) > at > org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) > at org.apache.spark.scheduler.Task.run(Task.scala:86) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.ClassCircularityError: > io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at > io.netty.util.internal.JavassistTypeParameterMatcherGenera
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Sigs, license, etc are OK. There are no Blockers for 2.0.2, though here are the 4 issues still open: SPARK-14387 Enable Hive-1.x ORC compatibility with spark.sql.hive.convertMetastoreOrc SPARK-17957 Calling outer join and na.fill(0) and then inner join will miss rows SPARK-17981 Incorrectly Set Nullability to False in FilterExec SPARK-18160 spark.files & spark.jars should not be passed to driver in yarn mode Running with Java 8, -Pyarn -Phive -Phive-thriftserver -Phadoop-2.7 on Ubuntu 16, I am seeing consistent failures in this test below. I think we very recently changed this so it could be legitimate. But does anyone else see something like this? I have seen other failures in this test due to OOM but my MAVEN_OPTS allows 6g of heap, which ought to be plenty. - SPARK-18189: Fix serialization issue in KeyValueGroupedDataset *** FAILED *** isContain was true Interpreter output contained 'Exception': Welcome to __ / __/__ ___ _/ /__ _\ \/ _ \/ _ `/ __/ '_/ /___/ .__/\_,_/_/ /_/\_\ version 2.0.2 /_/ Using Scala version 2.11.8 (OpenJDK 64-Bit Server VM, Java 1.8.0_102) Type in expressions to have them evaluated. Type :help for more information. scala> scala> keyValueGrouped: org.apache.spark.sql.KeyValueGroupedDataset[Int,(Int, Int)] = org.apache.spark.sql.KeyValueGroupedDataset@70c30f72 scala> mapGroups: org.apache.spark.sql.Dataset[(Int, Int)] = [_1: int, _2: int] scala> broadcasted: org.apache.spark.broadcast.Broadcast[Int] = Broadcast(0) scala> scala> scala> dataset: org.apache.spark.sql.Dataset[Int] = [value: int] scala> org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): com.google.common.util.concurrent.ExecutionError: java.lang.ClassCircularityError: io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2261) at com.google.common.cache.LocalCache.get(LocalCache.java:4000) at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:4004) at com.google.common.cache.LocalCache$LocalLoadingCache.get(LocalCache.java:4874) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator$.compile(CodeGenerator.scala:841) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:188) at org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection$.create(GenerateSafeProjection.scala:36) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:825) at org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator.generate(CodeGenerator.scala:822) at org.apache.spark.sql.execution.ObjectOperator$.deserializeRowToObject(objects.scala:137) at org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:251) at org.apache.spark.sql.execution.AppendColumnsExec$$anonfun$9.apply(objects.scala:250) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:79) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:47) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.lang.ClassCircularityError: io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at io.netty.util.internal.JavassistTypeParameterMatcherGenerator.generate(JavassistTypeParameterMatcherGenerator.java:62) at io.netty.util.internal.JavassistTypeParameterMatcherGenerator.generate(JavassistTypeParameterMatcherGenerator.java:54) at io.netty.util.internal.TypeParameterMatcher.get(TypeParameterMatcher.java:42) at io.netty.util.internal.TypeParameterMatcher.find(TypeParameterMatcher.java:78) at io.netty.handler.codec.MessageToMessageEncoder.(MessageToMessageEncoder.java:60) at org.apache.spark.network.protocol.MessageEncoder.(MessageEnco
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Sure - given the nature of the bug, it looked like it may have gone under the radar in prior 2.0 releases (test cases pass) so thought to bring attention to this for some evaluation of the criticality this issue. Will take further discussion to the ticket. -- View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-Release-Apache-Spark-2-0-2-RC2-tp19683p19687.html Sent from the Apache Spark Developers List mailing list archive at Nabble.com. - To unsubscribe e-mail: dev-unsubscr...@spark.apache.org
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Vinayak, Thanks for the email. This is really not the thread meant for reporting existing regressions. It's best just commenting on the jira ticket and even better submit a fix for it. On Tuesday, November 1, 2016, vijoshi wrote: > > Hi, > > Have encountered an issue with History Server in 2.0 - and updated > https://issues.apache.org/jira/browse/SPARK-16808 with a comment detailing > the problem. This is a regression in 2.0 from 1.6, so this issue exists > since 2.0.1. Encountered this very recently when we evaluated moving to 2.0 > from 1.6. But the issue is bad enough to virtually make it not possible to > adopt 2.0.x where spark history server runs behind a proxy. > > Regards, > Vinayak > > > > -- > View this message in context: http://apache-spark- > developers-list.1001551.n3.nabble.com/VOTE-Release-Apache-Spark-2-0-2-RC2- > tp19683p19685.html > Sent from the Apache Spark Developers List mailing list archive at > Nabble.com. > > - > To unsubscribe e-mail: dev-unsubscr...@spark.apache.org > >
Re: [VOTE] Release Apache Spark 2.0.2 (RC2)
Hi, Have encountered an issue with History Server in 2.0 - and updated https://issues.apache.org/jira/browse/SPARK-16808 with a comment detailing the problem. This is a regression in 2.0 from 1.6, so this issue exists since 2.0.1. Encountered this very recently when we evaluated moving to 2.0 from 1.6. But the issue is bad enough to virtually make it not possible to adopt 2.0.x where spark history server runs behind a proxy. Regards, Vinayak -- View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-Release-Apache-Spark-2-0-2-RC2-tp19683p19685.html Sent from the Apache Spark Developers List mailing list archive at Nabble.com. - To unsubscribe e-mail: dev-unsubscr...@spark.apache.org