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 488F6200BDA for ; Tue, 13 Dec 2016 16:20:04 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 4721B160B23; Tue, 13 Dec 2016 15:20:04 +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 EDA85160B15 for ; Tue, 13 Dec 2016 16:20:02 +0100 (CET) Received: (qmail 10402 invoked by uid 500); 13 Dec 2016 15:20:02 -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 10392 invoked by uid 99); 13 Dec 2016 15:20:02 -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; Tue, 13 Dec 2016 15:20:02 +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 AC039180254 for ; Tue, 13 Dec 2016 15:20:01 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -7.019 X-Spam-Level: X-Spam-Status: No, score=-7.019 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] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id aLo9SY37M7BW for ; Tue, 13 Dec 2016 15:20:00 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id 60C2A5F2F1 for ; Tue, 13 Dec 2016 15:19:59 +0000 (UTC) Received: (qmail 9934 invoked by uid 99); 13 Dec 2016 15:19:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 13 Dec 2016 15:19:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 6C6712C0087 for ; Tue, 13 Dec 2016 15:19:58 +0000 (UTC) Date: Tue, 13 Dec 2016 15:19:58 +0000 (UTC) From: "Aviem Zur (JIRA)" To: commits@beam.incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (BEAM-1145) Remove classifier from shaded spark runner artifact MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 13 Dec 2016 15:20:04 -0000 Aviem Zur created BEAM-1145: ------------------------------- Summary: Remove classifier from shaded spark runner artifact Key: BEAM-1145 URL: https://issues.apache.org/jira/browse/BEAM-1145 Project: Beam Issue Type: Improvement Components: runner-spark Reporter: Aviem Zur Assignee: Amit Sela Shade plugin configured in spark runner's pom adds a classifier to spark runner shaded jar {code:xml} true spark-app {code} This means, that in order for a user application that is dependent on spark-runner to work in cluster mode, they have to add the classifier in their dependency declaration, like so: {code:xml} org.apache.beam beam-runners-spark 0.4.0-incubating-SNAPSHOT spark-app {code} Otherwise, if they do not specify classifier, the jar they get is unshaded, which in cluster mode, causes collisions between different guava versions. Example exception in cluster mode when adding the dependency without classifier: {code} 16/12/12 06:58:56 WARN TaskSetManager: Lost task 4.0 in stage 8.0 (TID 153, lvsriskng02.lvs.paypal.com): java.lang.NoSuchMethodError: com.google.common.base.Stopwatch.createStarted()Lcom/google/common/base/Stopwatch; at org.apache.beam.runners.spark.stateful.StateSpecFunctions$1.apply(StateSpecFunctions.java:137) at org.apache.beam.runners.spark.stateful.StateSpecFunctions$1.apply(StateSpecFunctions.java:98) at org.apache.spark.streaming.StateSpec$$anonfun$1.apply(StateSpec.scala:180) at org.apache.spark.streaming.StateSpec$$anonfun$1.apply(StateSpec.scala:179) at org.apache.spark.streaming.rdd.MapWithStateRDDRecord$$anonfun$updateRecordWithData$1.apply(MapWithStateRDD.scala:57) at org.apache.spark.streaming.rdd.MapWithStateRDDRecord$$anonfun$updateRecordWithData$1.apply(MapWithStateRDD.scala:55) at scala.collection.Iterator$class.foreach(Iterator.scala:727) at org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28) at org.apache.spark.streaming.rdd.MapWithStateRDDRecord$.updateRecordWithData(MapWithStateRDD.scala:55) at org.apache.spark.streaming.rdd.MapWithStateRDD.compute(MapWithStateRDD.scala:155) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:313) at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:69) at org.apache.spark.rdd.RDD.iterator(RDD.scala:275) at org.apache.spark.streaming.rdd.MapWithStateRDD.compute(MapWithStateRDD.scala:149) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:313) at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:69) at org.apache.spark.rdd.RDD.iterator(RDD.scala:275) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:313) at org.apache.spark.rdd.RDD.iterator(RDD.scala:277) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:313) at org.apache.spark.CacheManager.getOrCompute(CacheManager.scala:69) at org.apache.spark.rdd.RDD.iterator(RDD.scala:275) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:227) 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} I would suggest that the classifier be removed from the shaded jar, to avoid confusion among users, and have a better user experience. P.S. Looks like Dataflow runner does not add a classifier to its shaded jar. -- This message was sent by Atlassian JIRA (v6.3.4#6332)