From issues-return-198337-archive-asf-public=cust-asf.ponee.io@spark.apache.org Wed Aug 8 00:15:06 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id 49E2D180657 for ; Wed, 8 Aug 2018 00:15:06 +0200 (CEST) Received: (qmail 67943 invoked by uid 500); 7 Aug 2018 22:15:05 -0000 Mailing-List: contact issues-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list issues@spark.apache.org Received: (qmail 67934 invoked by uid 99); 7 Aug 2018 22:15:05 -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; Tue, 07 Aug 2018 22:15:05 +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 AC917C026F for ; Tue, 7 Aug 2018 22:15:04 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.501 X-Spam-Level: X-Spam-Status: No, score=-109.501 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, 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 X1mfw1-jT7MW for ; Tue, 7 Aug 2018 22:15:03 +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 0F4AB5F433 for ; Tue, 7 Aug 2018 22:15:01 +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 EC274E12FE for ; Tue, 7 Aug 2018 22:15:00 +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 85D0D23F9C for ; Tue, 7 Aug 2018 22:15:00 +0000 (UTC) Date: Tue, 7 Aug 2018 22:15:00 +0000 (UTC) From: "Sean Owen (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (SPARK-25047) Can't assign SerializedLambda to scala.Function1 in deserialization of BucketedRandomProjectionLSHModel MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/SPARK-25047?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D165= 72410#comment-16572410 ]=20 Sean Owen commented on SPARK-25047: ----------------------------------- More notes.These two SO answers shed a little light: [https://stackoverflow.com/a/28367602/64174] [https://stackoverflow.com/questions/28079307/unable-to-deserialize-lambda/= 28084460#28084460] It suggests the problem is that the SerializedLambda instance that is deser= ialized should provide a readResolve() method to, I assume, resolve it back= into a scala.Function1. And that should actually be=C2=A0implemented by a = {{$deserializeLambda$(SerializedLambda)}} function in the capturing class. = It seems like something isn't turning it back from a SerializedLambda to so= mething else. The method is in the byte code of=C2=A0BucketedRandomProjectionLSH and deco= mpiles as {code:java} private static /* synthetic */ Object $deserializeLambda$(SerializedLambda = serializedLambda) { =C2=A0 =C2=A0 return LambdaDeserialize.bootstrap(new MethodHandle[]{$anonfu= n$hashDistance$1$adapted(scala.Tuple2 ), $anonfun$hashFunction$2$adapted(or= g.apache.spark.ml.feature.BucketedRandomProjectionLSHModel org.apache.spark= .ml.linalg.Vector org.apache.spark.ml.linalg.Vector ), $anonfun$hashFunctio= n$3$adapted(java.lang.Object ), $anonfun$hashFunction$1(org.apache.spark.ml= .feature.BucketedRandomProjectionLSHModel org.apache.spark.ml.linalg.Vector= )}, serializedLambda); }{code} While I traced through this for a while, I couldn't make sense of it. Howev= er, nothing actually failed around here. The ultimate error was a bit later= , and as in the StackOverflow post above. It goes without saying that there are plenty of fields of type scala.Functi= on1 in Spark and this is the only problem one, and I can't see why. Is it b= ecause it involves an array type? grepping suggests that could be unique. H= owever I tried to create a repro in a simple class file and all worked as e= xpected too. Something is odd about this case, and I don't know if it is in fact trigger= ing some odd corner case issue in scala or Java 8, or whether the Spark cod= e could be tweaked to dodge it. =C2=A0 > Can't assign SerializedLambda to scala.Function1 in deserialization of Bu= cketedRandomProjectionLSHModel > -------------------------------------------------------------------------= ------------------------------ > > Key: SPARK-25047 > URL: https://issues.apache.org/jira/browse/SPARK-25047 > Project: Spark > Issue Type: Sub-task > Components: ML > Affects Versions: 2.4.0 > Reporter: Sean Owen > Priority: Major > > Another distinct test failure: > {code:java} > - BucketedRandomProjectionLSH: streaming transform *** FAILED *** > =C2=A0 org.apache.spark.sql.streaming.StreamingQueryException: Query [id = =3D 7f34fb07-a718-4488-b644-d27cfd29ff6c, runId =3D 0bbc0ba2-2952-4504-85d6= -8aba877ba01b] terminated with exception: Job aborted due to stage failure:= Task 0 in stage 16.0 failed 1 times, most recent failure: Lost task 0.0 in= stage 16.0 (TID 16, localhost, executor driver): java.lang.ClassCastExcept= ion: cannot assign instance of java.lang.invoke.SerializedLambda to field o= rg.apache.spark.ml.feature.BucketedRandomProjectionLSHModel.hashFunction of= type scala.Function1 in instance of org.apache.spark.ml.feature.BucketedRa= ndomProjectionLSHModel > ... > =C2=A0 Cause: java.lang.ClassCastException: cannot assign instance of jav= a.lang.invoke.SerializedLambda to field org.apache.spark.ml.feature.Buckete= dRandomProjectionLSHModel.hashFunction of type scala.Function1 in instance = of org.apache.spark.ml.feature.BucketedRandomProjectionLSHModel > =C2=A0 at java.io.ObjectStreamClass$FieldReflector.setObjFieldValues(Obje= ctStreamClass.java:2233) > =C2=A0 at java.io.ObjectStreamClass.setObjFieldValues(ObjectStreamClass.j= ava:1405) > =C2=A0 at java.io.ObjectInputStream.defaultReadFields(ObjectInputStream.j= ava:2284) > ...{code} > Here the different nature of a Java 8 LMF closure trips of Java serializa= tion/deserialization. I think this can be patched by manually implementing = the Java serialization here, and don't see other instances (yet). > Also wondering if this "val" can be a "def". -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org