[ https://issues.apache.org/jira/browse/SPARK-24924?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16571908#comment-16571908 ]
Thomas Graves commented on SPARK-24924: --------------------------------------- so originally when I started on this I didn't know about the side affects of the hive table here. So this isn't as straight forward as I originally thought. I still personally don't like remapping this because users get something other then what they explicitly asked for, but if we want to keep this compatibility we either have to do that or actually have a com.databricks.avro class that would just map into our internal avro. That would give the benefit that they could eclipse it with their own jar if they wanted to keep using their customer version, I assume we could theoretically also support the spark.read.avro format as well. Or I guess the third option is to just break compatibility and require the users to change the table property, but then they can't read it with older versions of spark. It also seems bad to me that we aren't supporting spark.read.avro, so its an api compatibility issue. We magically help them with compatibility with their tables by mapping them but we don't support the old api and they have to update your code. This feels like an inconsistent story to me and not sure how that fits with our versioning policy since its a 3rd party thing. Not sure I like any of these options. Seems like these are the options: 1)I wonder if we actually add the class com.databricks.avro into the spark source that does the remap and support spark.read/write.avro for a couple releases for compatibility, then remove it and tell people to change the table property or provide an api to do that. 2) make the mapping of com.databricks.avro => internal avro configurable, that would allow them to continue use their version of com.databricks.avro until they can update api. 3) do nothing, leave this as is with this jira and user has to deal with losing spark.read.avro api and possible confusion and breaking if they are using modified version of com.databricks.avro thoughts from others? > Add mapping for built-in Avro data source > ----------------------------------------- > > Key: SPARK-24924 > URL: https://issues.apache.org/jira/browse/SPARK-24924 > Project: Spark > Issue Type: Sub-task > Components: SQL > Affects Versions: 2.4.0 > Reporter: Dongjoon Hyun > Assignee: Dongjoon Hyun > Priority: Minor > Fix For: 2.4.0 > > > This issue aims to the followings. > # Like `com.databricks.spark.csv` mapping, we had better map > `com.databricks.spark.avro` to built-in Avro data source. > # Remove incorrect error message, `Please find an Avro package at ...`. -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org