Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id B30BE200BBB for ; Thu, 10 Nov 2016 14:11:08 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id B1785160B01; Thu, 10 Nov 2016 13:11:08 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id CF745160AF6 for ; Thu, 10 Nov 2016 14:11:07 +0100 (CET) Received: (qmail 80608 invoked by uid 500); 10 Nov 2016 13:11:07 -0000 Mailing-List: contact commits-help@beam.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.incubator.apache.org Delivered-To: mailing list commits@beam.incubator.apache.org Received: (qmail 80598 invoked by uid 99); 10 Nov 2016 13:11:07 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Nov 2016 13:11:07 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 942A1180BCC for ; Thu, 10 Nov 2016 13:11:06 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -7.018 X-Spam-Level: X-Spam-Status: No, score=-7.018 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-2.999, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id fGBZqQ-f96em for ; Thu, 10 Nov 2016 13:11:05 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id 06AA55FB1F for ; Thu, 10 Nov 2016 13:10:59 +0000 (UTC) Received: (qmail 80359 invoked by uid 99); 10 Nov 2016 13:10:59 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Nov 2016 13:10:59 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 311162C0059 for ; Thu, 10 Nov 2016 13:10:59 +0000 (UTC) Date: Thu, 10 Nov 2016 13:10:59 +0000 (UTC) From: "Amit Sela (JIRA)" To: commits@beam.incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (BEAM-891) Flake in Spark metrics library? MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 10 Nov 2016 13:11:08 -0000 [ https://issues.apache.org/jira/browse/BEAM-891?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15654002#comment-15654002 ] Amit Sela commented on BEAM-891: -------------------------------- This seems to be the race: https://github.com/apache/spark/blob/branch-1.6/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala#L711 As the {{StreamingContext}} get's the {{env}}'s metricsSystem, which uses the same {{MetricsSystem}} as we use in {{SparkRuntimeContext}}. In general, Spark's {{ENV}} modules are singletons. > Flake in Spark metrics library? > ------------------------------- > > Key: BEAM-891 > URL: https://issues.apache.org/jira/browse/BEAM-891 > Project: Beam > Issue Type: Bug > Components: runner-spark > Reporter: Daniel Halperin > Assignee: Stas Levin > > [~staslev] I think you implemented this functionality originally? Want to take a look? CC [~amitsela] > Run: https://builds.apache.org/job/beam_PostCommit_RunnableOnService_SparkLocal/org.apache.beam$beam-runners-spark/43/testReport/junit/org.apache.beam.sdk.transforms/FilterTest/testFilterGreaterThan/ > Error: > {code} > java.lang.RuntimeException: java.lang.IndexOutOfBoundsException: 5 > at org.apache.beam.runners.spark.SparkRunner.run(SparkRunner.java:169) > at org.apache.beam.runners.spark.TestSparkRunner.run(TestSparkRunner.java:77) > at org.apache.beam.runners.spark.TestSparkRunner.run(TestSparkRunner.java:53) > at org.apache.beam.sdk.Pipeline.run(Pipeline.java:182) > at org.apache.beam.sdk.testing.TestPipeline.run(TestPipeline.java:112) > at org.apache.beam.sdk.transforms.FilterTest.testFilterGreaterThan(FilterTest.java:122) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:498) > at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:47) > at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) > at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:44) > at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) > at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:271) > at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:70) > at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:50) > at org.junit.runners.ParentRunner$3.run(ParentRunner.java:238) > at org.apache.maven.surefire.junitcore.pc.Scheduler$1.run(Scheduler.java:393) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > 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.IndexOutOfBoundsException: 5 > at scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43) > at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:47) > at scala.collection.IndexedSeqOptimized$class.segmentLength(IndexedSeqOptimized.scala:189) > at scala.collection.mutable.ArrayBuffer.segmentLength(ArrayBuffer.scala:47) > at scala.collection.IndexedSeqOptimized$class.indexWhere(IndexedSeqOptimized.scala:198) > at scala.collection.mutable.ArrayBuffer.indexWhere(ArrayBuffer.scala:47) > at scala.collection.GenSeqLike$class.indexOf(GenSeqLike.scala:144) > at scala.collection.AbstractSeq.indexOf(Seq.scala:40) > at scala.collection.GenSeqLike$class.indexOf(GenSeqLike.scala:128) > at scala.collection.AbstractSeq.indexOf(Seq.scala:40) > at scala.collection.mutable.BufferLike$class.$minus$eq(BufferLike.scala:126) > at scala.collection.mutable.AbstractBuffer.$minus$eq(Buffer.scala:48) > at org.apache.spark.metrics.MetricsSystem.removeSource(MetricsSystem.scala:159) > at org.apache.beam.runners.spark.translation.SparkRuntimeContext.registerMetrics(SparkRuntimeContext.java:94) > at org.apache.beam.runners.spark.translation.SparkRuntimeContext.(SparkRuntimeContext.java:66) > at org.apache.beam.runners.spark.translation.EvaluationContext.(EvaluationContext.java:73) > at org.apache.beam.runners.spark.SparkRunner.run(SparkRunner.java:146) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)