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 0F5A3200B88 for ; Wed, 7 Sep 2016 20:27:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 0E675160AC1; Wed, 7 Sep 2016 18:27:22 +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 551F7160ABF for ; Wed, 7 Sep 2016 20:27:21 +0200 (CEST) Received: (qmail 5627 invoked by uid 500); 7 Sep 2016 18:27:20 -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 5609 invoked by uid 99); 7 Sep 2016 18:27:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 07 Sep 2016 18:27:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 72D482C1B77 for ; Wed, 7 Sep 2016 18:27:20 +0000 (UTC) Date: Wed, 7 Sep 2016 18:27:20 +0000 (UTC) From: "Jakob Odersky (JIRA)" To: issues@spark.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (SPARK-17368) Scala value classes create encoder problems and break at runtime MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Wed, 07 Sep 2016 18:27:22 -0000 [ https://issues.apache.org/jira/browse/SPARK-17368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15471428#comment-15471428 ] Jakob Odersky commented on SPARK-17368: --------------------------------------- Hmm, you're right my assumption was of using only value classes in the beginning and at the end was too naive. [~srowen], how likely do you think it is that we can include a meta-encoder in Spark? It could be included in the form of an optional import. Since the existing encoders/ScalaReflection framework already use runtime-reflection, my guess is that adding compile-time reflection will not be too difficult. > Scala value classes create encoder problems and break at runtime > ---------------------------------------------------------------- > > Key: SPARK-17368 > URL: https://issues.apache.org/jira/browse/SPARK-17368 > Project: Spark > Issue Type: Bug > Components: Spark Core, SQL > Affects Versions: 1.6.2, 2.0.0 > Environment: JDK 8 on MacOS > Scala 2.11.8 > Spark 2.0.0 > Reporter: Aris Vlasakakis > > Using Scala value classes as the inner type for Datasets breaks in Spark 2.0 and 1.6.X. > This simple Spark 2 application demonstrates that the code will compile, but will break at runtime with the error. The value class is of course *FeatureId*, as it extends AnyVal. > {noformat} > Exception in thread "main" java.lang.RuntimeException: Error while encoding: java.lang.RuntimeException: Couldn't find v on int > assertnotnull(input[0, int, true], top level non-flat input object).v AS v#0 > +- assertnotnull(input[0, int, true], top level non-flat input object).v > +- assertnotnull(input[0, int, true], top level non-flat input object) > +- input[0, int, true]". > at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.toRow(ExpressionEncoder.scala:279) > at org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421) > at org.apache.spark.sql.SparkSession$$anonfun$3.apply(SparkSession.scala:421) > {noformat} > Test code for Spark 2.0.0: > {noformat} > import org.apache.spark.sql.{Dataset, SparkSession} > object BreakSpark { > case class FeatureId(v: Int) extends AnyVal > def main(args: Array[String]): Unit = { > val seq = Seq(FeatureId(1), FeatureId(2), FeatureId(3)) > val spark = SparkSession.builder.getOrCreate() > import spark.implicits._ > spark.sparkContext.setLogLevel("warn") > val ds: Dataset[FeatureId] = spark.createDataset(seq) > println(s"BREAK HERE: ${ds.count}") > } > } > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org For additional commands, e-mail: issues-help@spark.apache.org