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 18017200C82 for ; Fri, 12 May 2017 18:18:11 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 16809160BA3; Fri, 12 May 2017 16:18:11 +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 38C79160BB8 for ; Fri, 12 May 2017 18:18:10 +0200 (CEST) Received: (qmail 61990 invoked by uid 500); 12 May 2017 16:18:09 -0000 Mailing-List: contact commits-help@beam.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@beam.apache.org Delivered-To: mailing list commits@beam.apache.org Received: (qmail 61981 invoked by uid 99); 12 May 2017 16:18:09 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 May 2017 16:18:09 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id D4A1EC0C07 for ; Fri, 12 May 2017 16:18:08 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.201 X-Spam-Level: X-Spam-Status: No, score=-99.201 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id Fg4hpN7VLd3e for ; Fri, 12 May 2017 16:18:07 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 660DB5FDC0 for ; Fri, 12 May 2017 16:18:06 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 62A33E0D69 for ; Fri, 12 May 2017 16:18:05 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 48BB224309 for ; Fri, 12 May 2017 16:18:04 +0000 (UTC) Date: Fri, 12 May 2017 16:18:04 +0000 (UTC) From: "Davor Bonaci (JIRA)" To: commits@beam.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (BEAM-2095) The hasNext method of the iterator returned by SourceRDD#compute is not idempotent MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 12 May 2017 16:18:11 -0000 [ https://issues.apache.org/jira/browse/BEAM-2095?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Davor Bonaci updated BEAM-2095: ------------------------------- Fix Version/s: 2.0.0 > The hasNext method of the iterator returned by SourceRDD#compute is not idempotent > ---------------------------------------------------------------------------------- > > Key: BEAM-2095 > URL: https://issues.apache.org/jira/browse/BEAM-2095 > Project: Beam > Issue Type: Bug > Components: runner-spark > Affects Versions: 0.6.0 > Reporter: Arvid Heise > Assignee: Stas Levin > Fix For: 2.0.0, 2.1.0 > > > When reading an Avro from HDFS with the new HDFSFileSource, we experience the following exceptions: > {code} > 17/04/27 11:48:38 ERROR executor.Executor: Exception in task 2.0 in stage 1.0 (TID 32) > java.util.NoSuchElementException > at com.gfk.hyperlane.engine.target_group_evaluation.dataset.HDFSFileSource$HDFSFileReader.getCurrent(HDFSFileSource.java:498) > at org.apache.beam.runners.spark.io.SourceRDD$Bounded$1.next(SourceRDD.java:142) > at org.apache.beam.runners.spark.io.SourceRDD$Bounded$1.next(SourceRDD.java:111) > at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:42) > at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43) > at scala.collection.Iterator$$anon$12.next(Iterator.scala:357) > at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43) > at scala.collection.convert.Wrappers$IteratorWrapper.next(Wrappers.scala:30) > at org.apache.beam.runners.spark.translation.SparkProcessContext$ProcCtxtIterator.computeNext(SparkProcessContext.java:165) > at org.apache.beam.spark.repackaged.com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:145) > at org.apache.beam.spark.repackaged.com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:140) > at org.apache.beam.runners.spark.translation.SparkProcessContext$ProcCtxtIterator.computeNext(SparkProcessContext.java:162) > at org.apache.beam.spark.repackaged.com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:145) > at org.apache.beam.spark.repackaged.com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:140) > at org.apache.beam.runners.spark.translation.SparkProcessContext.processPartition(SparkProcessContext.java:64) > at org.apache.beam.runners.spark.translation.DoFnFunction.call(DoFnFunction.java:105) > at org.apache.beam.runners.spark.translation.DoFnFunction.call(DoFnFunction.java:48) > at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$4$1.apply(JavaRDDLike.scala:159) > at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$4$1.apply(JavaRDDLike.scala:159) > at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710) > at org.apache.spark.rdd.RDD$$anonfun$mapPartitions$1$$anonfun$apply$20.apply(RDD.scala:710) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) > at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > at org.apache.spark.rdd.RDD.iterator(RDD.scala:270) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) > at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) > at org.apache.spark.scheduler.Task.run(Task.scala:89) > at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214) > 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) > {code} > The error comes from a call to BoundedReader#getCurrent after it has been closed. > We logged the following call patterns: > (for data) > advance > getCurrent > (when drained) > advance > close > getCurrent > The issue probably comes from the implementation in SourceRDD > https://github.com/apache/beam/blob/3101e69c438d5c42577fc7d3476d623f6e551837/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java#L145 > A repeated call to hasNext will result in repeated calls of advance. This results in a data loss and may return different results. In particular, it may cause the issue as observed. > The usual solution is to use hasNext() to already retrieve and cache the next element if cache empty and return and reset the cache in next(). -- This message was sent by Atlassian JIRA (v6.3.15#6346)