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 BFB90200BBD for ; Tue, 8 Nov 2016 19:49:04 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id BE6C8160B0A; Tue, 8 Nov 2016 18:49: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 DD755160AD0 for ; Tue, 8 Nov 2016 19:49:03 +0100 (CET) Received: (qmail 38702 invoked by uid 500); 8 Nov 2016 18:49:03 -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 38693 invoked by uid 99); 8 Nov 2016 18:49:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 08 Nov 2016 18:49:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id A55A0CA3F3 for ; Tue, 8 Nov 2016 18:49:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -6.218 X-Spam-Level: X-Spam-Status: No, score=-6.218 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, 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-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 2lMPElmWl1rZ for ; Tue, 8 Nov 2016 18:49: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 AA3195F1B8 for ; Tue, 8 Nov 2016 18:48:59 +0000 (UTC) Received: (qmail 37882 invoked by uid 99); 8 Nov 2016 18:48:58 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 08 Nov 2016 18:48:58 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 964092C2A67 for ; Tue, 8 Nov 2016 18:48:58 +0000 (UTC) Date: Tue, 8 Nov 2016 18:48:58 +0000 (UTC) From: "Amit Sela (JIRA)" To: commits@beam.incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Comment Edited] (BEAM-944) Spark runner causes an exception when creating pipeline options MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 08 Nov 2016 18:49:04 -0000 [ https://issues.apache.org/jira/browse/BEAM-944?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15648409#comment-15648409 ] Amit Sela edited comment on BEAM-944 at 11/8/16 6:48 PM: --------------------------------------------------------- Wait, there is a {{runtime}} dependency in examples pom.. Both spark-core and spark-streaming are scoped {{runtime}}. I that's the issue, we'll have the same problem and have to remove the provided context. As for listeners, they're nice, useful for testing (to the best of my knowledge so far), but removing the provided context option will prevent the runner from running with the Spark Job Server, and intents to running Scio+SparkRunner in REPL. And maybe Zeppelin. All of which are examples of features users asked about.. was (Author: amitsela): Wait, there is a {{runtime}} dependency in examples pom.. Both spark-core and spark-streaming are scoped {{runtime}}. I that's the issue, we'll have the same problem and have to remove the provided context. As for listeners, they're nice, useful for testing (to the best of my knowledge so far), but the removing provided context option will prevent the runner from running with the Spark Job Server, and intents to running Scio+SparkRunner in REPL. And maybe Zeppelin. All of which are examples of features users asked about.. > Spark runner causes an exception when creating pipeline options > --------------------------------------------------------------- > > Key: BEAM-944 > URL: https://issues.apache.org/jira/browse/BEAM-944 > Project: Beam > Issue Type: Bug > Components: runner-spark > Affects Versions: 0.3.0-incubating > Reporter: Davor Bonaci > Assignee: Amit Sela > Fix For: 0.4.0-incubating > > > Running any example with *any* runner throws an exception as soon as it tries to construct PipelineOptions as long as SparkRunner is on the class path: > {code} > mvn compile exec:java -Dexec.mainClass=org.apache.beam.examples.complete.game.LeaderBoard -Dexec.args="--runner=DirectRunner" > [INFO] Scanning for projects... > [INFO] > [INFO] ------------------------------------------------------------------------ > [INFO] Building Tutorial 0.0.1-SNAPSHOT > [INFO] ------------------------------------------------------------------------ > [INFO] > [INFO] --- maven-resources-plugin:2.6:resources (default-resources) @ Tutorial --- > [WARNING] Using platform encoding (UTF-8 actually) to copy filtered resources, i.e. build is platform dependent! > [INFO] skip non existing resourceDirectory /Users/fjp/dev/mobile/src/main/resources > [INFO] > [INFO] --- maven-compiler-plugin:3.3:compile (default-compile) @ Tutorial --- > [INFO] No sources to compile > [INFO] > [INFO] --- exec-maven-plugin:1.5.0:java (default-cli) @ Tutorial --- > SLF4J: Class path contains multiple SLF4J bindings. > SLF4J: Found binding in [jar:file:/Users/fjp/.m2/repository/org/slf4j/slf4j-log4j12/1.7.10/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class] > SLF4J: Found binding in [jar:file:/Users/fjp/.m2/repository/org/slf4j/slf4j-jdk14/1.7.5/slf4j-jdk14-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class] > SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an explanation. > SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] > [WARNING] > java.lang.reflect.InvocationTargetException > 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:497) > at org.codehaus.mojo.exec.ExecJavaMojo$1.run(ExecJavaMojo.java:294) > at java.lang.Thread.run(Thread.java:745) > Caused by: java.lang.ExceptionInInitializerError > at org.apache.beam.examples.complete.game.LeaderBoard.main(LeaderBoard.java:179) > ... 6 more > Caused by: java.lang.TypeNotPresentException: Type org.apache.spark.streaming.api.java.JavaStreamingListener not present > at sun.reflect.generics.factory.CoreReflectionFactory.makeNamedType(CoreReflectionFactory.java:117) > at sun.reflect.generics.visitor.Reifier.visitClassTypeSignature(Reifier.java:125) > at sun.reflect.generics.tree.ClassTypeSignature.accept(ClassTypeSignature.java:49) > at sun.reflect.generics.visitor.Reifier.reifyTypeArguments(Reifier.java:68) > at sun.reflect.generics.visitor.Reifier.visitClassTypeSignature(Reifier.java:138) > at sun.reflect.generics.tree.ClassTypeSignature.accept(ClassTypeSignature.java:49) > at sun.reflect.generics.repository.ConstructorRepository.getParameterTypes(ConstructorRepository.java:94) > at java.lang.reflect.Executable.getGenericParameterTypes(Executable.java:284) > at java.lang.reflect.Method.getGenericParameterTypes(Method.java:282) > at java.beans.FeatureDescriptor.getParameterTypes(FeatureDescriptor.java:387) > at java.beans.PropertyDescriptor.findPropertyType(PropertyDescriptor.java:653) > at java.beans.PropertyDescriptor.setWriteMethod(PropertyDescriptor.java:326) > at java.beans.PropertyDescriptor.(PropertyDescriptor.java:138) > at org.apache.beam.sdk.options.PipelineOptionsFactory.getPropertyDescriptors(PipelineOptionsFactory.java:896) > at org.apache.beam.sdk.options.PipelineOptionsFactory.validateClass(PipelineOptionsFactory.java:992) > at org.apache.beam.sdk.options.PipelineOptionsFactory.validateWellFormed(PipelineOptionsFactory.java:627) > at org.apache.beam.sdk.options.PipelineOptionsFactory.register(PipelineOptionsFactory.java:561) > at org.apache.beam.sdk.options.PipelineOptionsFactory.initializeRegistry(PipelineOptionsFactory.java:587) > at org.apache.beam.sdk.options.PipelineOptionsFactory.(PipelineOptionsFactory.java:536) > ... 7 more > Caused by: java.lang.ClassNotFoundException: org.apache.spark.streaming.api.java.JavaStreamingListener > at java.net.URLClassLoader.findClass(URLClassLoader.java:381) > at java.lang.ClassLoader.loadClass(ClassLoader.java:424) > at java.lang.ClassLoader.loadClass(ClassLoader.java:357) > at java.lang.Class.forName0(Native Method) > at java.lang.Class.forName(Class.java:348) > at sun.reflect.generics.factory.CoreReflectionFactory.makeNamedType(CoreReflectionFactory.java:114) > ... 25 more > {code} > It is introduced by https://github.com/apache/incubator-beam/pull/1072 that was added in mid-October, so it reproduces starting with 0.3.0-incubating, but not earlier. > A possible workaround for users is to add this explicit dependency, which is marked as provided in the Spark runner: > {code} > > org.apache.spark > spark-streaming_2.10 > ${spark.version} > > {code} > On a quick glance, JavaStreamingListener is not a real pipeline option. Perhaps it can be just removed from pipeline options, and passed around differently. > Separately, we should perhaps re-think how this works with respect to dependencies in general. > Marking blocking for 0.4.0-incubating since it affects Spark quickstart. > [~amitsela], what do you think? -- This message was sent by Atlassian JIRA (v6.3.4#6332)