spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Harry Brundage (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (SPARK-11319) PySpark silently Accepts null values in non-nullable DataFrame fields.
Date Thu, 19 Nov 2015 23:14:11 GMT

    [ https://issues.apache.org/jira/browse/SPARK-11319?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15014705#comment-15014705
] 

Harry Brundage commented on SPARK-11319:
----------------------------------------

bq. First of all, lets talk about nullability in the context of Spark SQL. It is a hint to
the optimizer that we can ignore null checks. When in doubt, you should always set it to true
since then you are asking Spark to perform null checks. All interfaces should default to setting
this true when it is unspecified.

This makes sense and I understand, however, that's not all the schema's field represents to
users, or at least to me. While it is used as a hint to the optimizer, it also serves as a
powerful description of the actual shape of the data useful for other things! Our parquet
drops for hive are built off of that same schema, so the optimizations there are either enjoyed
or ignored depending on what we write out with Spark. We have business logic for example that
asserts some fields we know should never be null are not null in the output Dataframe passed
back into our framework. We have schema search and explorer tools that report on our datasets
in the system that I would like to accurately describe assumptions developers can make about
data. As I am sure you have battled with Michael other systems like Parquet have other optimizations
that are made depending on nullability. I know for example nulls are run-length encoded outside
the actual data pages in Parquet, but giving it or any other format writer accurate information
about what it can expect before it starts writing all the data is definitely ideal.

> Probably the best solution is to ignore you and set nullable to true no matter what you
say.

For the reasons above if you do this please please don't actually store that change in the
user-visible schema.

> There is a tension between performance and checking everything so that we can provide
better errors. In many cases, we've had similarly effusive requests from users to change behavior
where we were being overly cautious and it was hurting performance. (i.e. "Just Trust Us!")

I understand this tension and I do not think I should just get my way, but I think here you've
fallen on the wrong side. Let me explain.

bq. I do not believe that this is true. Both datasources know that its possible to have corrupt
lines and thus the schema information they produce says that all columns are nullable.

Sure, but again, if we know that the column is not nullable, we are forced to go around the
schema detection logic and provide a schema ourselves. We do this with pig schemas describing
JSON files very often. I know JSON sucks but sadly large parts of the world converged on it
and I think Spark must commit to providing good JSON support. 

My problem is that yes, inconsistent or nulls are indeed possible, but I would like to not
have to detect it myself. Spark can't make assumptions about the data, sure, so when asked
to infer what the schema is for a file it describes every field as non-nullable. When the
developer tells Spark what the schema is, Spark must then decide to trust them or not. Trusting
the developer means a performance gain from no validation step and thus sexy benchmarks for
the marketing materials, but if you ask me, corruption is still a serious threat. Developers
screw up! Kevin hit this problem in the first few weeks of our experiments with Dataframes
and he's only one dude! It's rather unclear to me and my team of Spark users that passing
nullability information to Spark doesn't actually validate it but instead is a seemingly simple
optimization to do the exact opposite. We know that our schema expectations are sometimes
violated in unforeseen circumstances despite our best efforts, and so we want this from you,
but... aren't yours too? Don't you think some Avro or Parquet bug will come along sometime
that causes this same problem? 

Similarly to Kevin, I also think that this contract around nullability has a lot more power
that would be kneecapped without vaildation in place. Do you really not anticipate any other
uses of the nullability information arising? Tungsten isn't going to optimize layout knowing
there will always be data? Maybe select a different compression algorithm for the column?
Maybe prune tasks completely knowing they're operating on all nulls or no nulls? It just seems
bonkers to me for Spark to embrace and support obviously incorrect invocations like the one
above. 

bq. Every database you have ever used was based on the assumption that you are going to do
an expensive ETL and then query that data many times. Spark SQL is trying to optimize for
the case where people are querying data in-situ and so I don't think a direct comparison here
is really fair. A traditional RDBMS has integrity constraints because they control the data
that they are querying. We can't do this because someone can always just drop another file
that violates these into HDFS or S3, etc. Thus in all cases where we know we don't have control
we set nullable = true internally. In the advanced interfaces where we allow users to specific
schema manually we expect them to do the same.

I do think it is fair anecdotally, my entire team of 2+ year spark veterans was absolutely
baffled by this behaviour. Granted we are building ETLs using Spark, but I mean your very
own product [claims|https://databricks.com/product/databricks] production pipelines as a use
case for the thing. I understand what you are saying and I empathize. You must be governed
by the lowest common denominator, you do not have much information to work with and often
lack control hence the nullables everywhere.

However, this is one place where you actually do have control and I think it really wouldn't
cost that much to validate that data matches the schema's description on the way through.
We use this advanced API to try to help you. Anyone trying to teach Spark more about what
they know could lean on your sane primitives, but instead, you're asking us to build them
ourselves and ensure we apply it all the time. We promote many RDD[T]'s to Dataframes as we
transition our system away from being built on freeform RDDs to Dataframes, we seed Spark
with richer type information because JSON sucks and we haven't moved to something better yet,
and I am sure many other users do equally fucked up stuff all the time. 

If you say you resolve your tension to the don't validate side because you want the performance
benefit, I say want your system to have my back. You are smarter than me, you can build it
better and in the right place and right language. I say it won't cost that much. I don't want
to have to wrap the hammer in bubble wrap. 

> PySpark silently Accepts null values in non-nullable DataFrame fields.
> ----------------------------------------------------------------------
>
>                 Key: SPARK-11319
>                 URL: https://issues.apache.org/jira/browse/SPARK-11319
>             Project: Spark
>          Issue Type: Bug
>          Components: PySpark, SQL
>            Reporter: Kevin Cox
>
> Running the following code with a null value in a non-nullable column silently works.
This makes the code incredibly hard to trust.
> {code}
> In [2]: from pyspark.sql.types import *
> In [3]: sqlContext.createDataFrame([(None,)], StructType([StructField("a", TimestampType(),
False)])).collect()
> Out[3]: [Row(a=None)]
> {code}



--
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


Mime
View raw message