You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@spark.apache.org by Muthu Jayakumar <ba...@gmail.com> on 2016/10/20 01:07:00 UTC

Dataframe schema...

Hello there,

I am trying to understand how and when does DataFrame (or Dataset) sets
nullable = true vs false on a schema.

Here is my observation from a sample code I tried...


scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
lit("bla")).printSchema()
root
 |-- col1: integer (nullable = false)
 |-- col2: string (nullable = true)
 |-- col3: double (nullable = false)
 |-- col4: string (nullable = false)


scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
lit("bla")).write.parquet("/tmp/sample.parquet")

scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
root
 |-- col1: integer (nullable = true)
 |-- col2: string (nullable = true)
 |-- col3: double (nullable = true)
 |-- col4: string (nullable = true)


The place where this seem to get me into trouble is when I try to union one
data-structure from in-memory (notice that in the below schema the
highlighted element is represented as 'false' for in-memory created schema)
and one from file that starts out with a schema like below...

 |-- some_histogram: struct (nullable = true)
 |    |-- values: array (nullable = true)
 |    |    |-- element: double (containsNull = true)
 |    |-- freq: array (nullable = true)
 |    |    |-- element: long (containsNull = true)

Is there a way to convert this attribute from true to false without running
any mapping / udf on that column?

Please advice,
Muthu

Re: Dataframe schema...

Posted by Michael Armbrust <mi...@databricks.com>.
On Fri, Oct 21, 2016 at 8:40 PM, Koert Kuipers <ko...@tresata.com> wrote:

> This rather innocent looking optimization flag nullable has caused a lot
> of bugs... Makes me wonder if we are better off without it
>

Yes... my most regretted design decision :(

Please give thoughts here: https://issues.apache.org/jira/browse/SPARK-17939

Re: Dataframe schema...

Posted by Koert Kuipers <ko...@tresata.com>.
This rather innocent looking optimization flag nullable has caused a lot of
bugs... Makes me wonder if we are better off without it

On Oct 21, 2016 8:37 PM, "Muthu Jayakumar" <ba...@gmail.com> wrote:

> Thanks Cheng Lian for opening the JIRA. I found this with Spark 2.0.0.
>
> Thanks,
> Muthu
>
> On Fri, Oct 21, 2016 at 3:30 PM, Cheng Lian <li...@databricks.com> wrote:
>
>> Yea, confirmed. While analyzing unions, we treat StructTypes with
>> different field nullabilities as incompatible types and throws this error.
>>
>> Opened https://issues.apache.org/jira/browse/SPARK-18058 to track this
>> issue. Thanks for reporting!
>>
>> Cheng
>>
>> On 10/21/16 3:15 PM, Cheng Lian wrote:
>>
>> Hi Muthu,
>>
>> What is the version of Spark are you using? This seems to be a bug in the
>> analysis phase.
>>
>> Cheng
>>
>> On 10/21/16 12:50 PM, Muthu Jayakumar wrote:
>>
>> Sorry for the late response. Here is what I am seeing...
>>
>>
>> Schema from parquet file.
>>
>> d1.printSchema()
>>
>> root
>>  |-- task_id: string (nullable = true)
>>  |-- task_name: string (nullable = true)
>>  |-- some_histogram: struct (nullable = true)
>>  |    |-- values: array (nullable = true)
>>  |    |    |-- element: double (containsNull = true)
>>  |    |-- freq: array (nullable = true)
>>  |    |    |-- element: long (containsNull = true)
>>
>> d2.printSchema() //Data created using dataframe and/or processed before writing to parquet file.
>>
>> root
>>  |-- task_id: string (nullable = true)
>>  |-- task_name: string (nullable = true)
>>  |-- some_histogram: struct (nullable = true)
>>  |    |-- values: array (nullable = true)
>>  |    |    |-- element: double (containsNull = false)
>>  |    |-- freq: array (nullable = true)
>>  |    |    |-- element: long (containsNull = false)
>>
>> d1.union(d2).printSchema()
>>
>> Exception in thread "main" org.apache.spark.sql.AnalysisException:
>> unresolved operator 'Union;
>> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.f
>> ailAnalysis(CheckAnalysis.scala:40)
>> at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis
>> (Analyzer.scala:58)
>> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfu
>> n$checkAnalysis$1.apply(CheckAnalysis.scala:361)
>> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfu
>> n$checkAnalysis$1.apply(CheckAnalysis.scala:67)
>> at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeN
>> ode.scala:126)
>> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.c
>> heckAnalysis(CheckAnalysis.scala:67)
>> at org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysi
>> s(Analyzer.scala:58)
>> at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed
>> (QueryExecution.scala:49)
>> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:161)
>> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
>> at org.apache.spark.sql.Dataset$.apply(Dataset.scala:59)
>> at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:2594)
>> at org.apache.spark.sql.Dataset.union(Dataset.scala:1459)
>>
>> Please advice,
>> Muthu
>>
>> On Thu, Oct 20, 2016 at 1:46 AM, Michael Armbrust <michael@databricks.com
>> > wrote:
>>
>>> What is the issue you see when unioning?
>>>
>>> On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar <ba...@gmail.com>
>>> wrote:
>>>
>>>> Hello Michael,
>>>>
>>>> Thank you for looking into this query. In my case there seem to be an
>>>> issue when I union a parquet file read from disk versus another dataframe
>>>> that I construct in-memory. The only difference I see is the containsNull =
>>>> true. In fact, I do not see any errors with union on the simple schema of
>>>> "col1 thru col4" above. But the problem seem to exist only on that
>>>> "some_histogram" column which contains the mixed containsNull = true/false.
>>>> Let me know if this helps.
>>>>
>>>> Thanks,
>>>> Muthu
>>>>
>>>>
>>>>
>>>> On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <
>>>> michael@databricks.com> wrote:
>>>>
>>>>> Nullable is just a hint to the optimizer that its impossible for there
>>>>> to be a null value in this column, so that it can avoid generating code for
>>>>> null-checks.  When in doubt, we set nullable=true since it is always safer
>>>>> to check.
>>>>>
>>>>> Why in particular are you trying to change the nullability of the
>>>>> column?
>>>>>
>>>>> On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hello there,
>>>>>>
>>>>>> I am trying to understand how and when does DataFrame (or Dataset)
>>>>>> sets nullable = true vs false on a schema.
>>>>>>
>>>>>> Here is my observation from a sample code I tried...
>>>>>>
>>>>>>
>>>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>>>> lit("bla")).printSchema()
>>>>>> root
>>>>>>  |-- col1: integer (nullable = false)
>>>>>>  |-- col2: string (nullable = true)
>>>>>>  |-- col3: double (nullable = false)
>>>>>>  |-- col4: string (nullable = false)
>>>>>>
>>>>>>
>>>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>>>> lit("bla")).write.parquet("/tmp/sample.parquet")
>>>>>>
>>>>>> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
>>>>>> root
>>>>>>  |-- col1: integer (nullable = true)
>>>>>>  |-- col2: string (nullable = true)
>>>>>>  |-- col3: double (nullable = true)
>>>>>>  |-- col4: string (nullable = true)
>>>>>>
>>>>>>
>>>>>> The place where this seem to get me into trouble is when I try to
>>>>>> union one data-structure from in-memory (notice that in the below schema
>>>>>> the highlighted element is represented as 'false' for in-memory created
>>>>>> schema) and one from file that starts out with a schema like below...
>>>>>>
>>>>>>  |-- some_histogram: struct (nullable = true)
>>>>>>  |    |-- values: array (nullable = true)
>>>>>>  |    |    |-- element: double (containsNull = true)
>>>>>>  |    |-- freq: array (nullable = true)
>>>>>>  |    |    |-- element: long (containsNull = true)
>>>>>>
>>>>>> Is there a way to convert this attribute from true to false without
>>>>>> running any mapping / udf on that column?
>>>>>>
>>>>>> Please advice,
>>>>>> Muthu
>>>>>>
>>>>>
>>>>>
>>>>
>>>
>>
>>
>>
>

Re: Dataframe schema...

Posted by Muthu Jayakumar <ba...@gmail.com>.
Thanks Cheng Lian for opening the JIRA. I found this with Spark 2.0.0.

Thanks,
Muthu

On Fri, Oct 21, 2016 at 3:30 PM, Cheng Lian <li...@databricks.com> wrote:

> Yea, confirmed. While analyzing unions, we treat StructTypes with
> different field nullabilities as incompatible types and throws this error.
>
> Opened https://issues.apache.org/jira/browse/SPARK-18058 to track this
> issue. Thanks for reporting!
>
> Cheng
>
> On 10/21/16 3:15 PM, Cheng Lian wrote:
>
> Hi Muthu,
>
> What is the version of Spark are you using? This seems to be a bug in the
> analysis phase.
>
> Cheng
>
> On 10/21/16 12:50 PM, Muthu Jayakumar wrote:
>
> Sorry for the late response. Here is what I am seeing...
>
>
> Schema from parquet file.
>
> d1.printSchema()
>
> root
>  |-- task_id: string (nullable = true)
>  |-- task_name: string (nullable = true)
>  |-- some_histogram: struct (nullable = true)
>  |    |-- values: array (nullable = true)
>  |    |    |-- element: double (containsNull = true)
>  |    |-- freq: array (nullable = true)
>  |    |    |-- element: long (containsNull = true)
>
> d2.printSchema() //Data created using dataframe and/or processed before writing to parquet file.
>
> root
>  |-- task_id: string (nullable = true)
>  |-- task_name: string (nullable = true)
>  |-- some_histogram: struct (nullable = true)
>  |    |-- values: array (nullable = true)
>  |    |    |-- element: double (containsNull = false)
>  |    |-- freq: array (nullable = true)
>  |    |    |-- element: long (containsNull = false)
>
> d1.union(d2).printSchema()
>
> Exception in thread "main" org.apache.spark.sql.AnalysisException:
> unresolved operator 'Union;
> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.
> failAnalysis(CheckAnalysis.scala:40)
> at org.apache.spark.sql.catalyst.analysis.Analyzer.
> failAnalysis(Analyzer.scala:58)
> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$
> anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:361)
> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$
> anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:67)
> at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(
> TreeNode.scala:126)
> at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.
> checkAnalysis(CheckAnalysis.scala:67)
> at org.apache.spark.sql.catalyst.analysis.Analyzer.
> checkAnalysis(Analyzer.scala:58)
> at org.apache.spark.sql.execution.QueryExecution.
> assertAnalyzed(QueryExecution.scala:49)
> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:161)
> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
> at org.apache.spark.sql.Dataset$.apply(Dataset.scala:59)
> at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:2594)
> at org.apache.spark.sql.Dataset.union(Dataset.scala:1459)
>
> Please advice,
> Muthu
>
> On Thu, Oct 20, 2016 at 1:46 AM, Michael Armbrust <mi...@databricks.com>
> wrote:
>
>> What is the issue you see when unioning?
>>
>> On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar <ba...@gmail.com>
>> wrote:
>>
>>> Hello Michael,
>>>
>>> Thank you for looking into this query. In my case there seem to be an
>>> issue when I union a parquet file read from disk versus another dataframe
>>> that I construct in-memory. The only difference I see is the containsNull =
>>> true. In fact, I do not see any errors with union on the simple schema of
>>> "col1 thru col4" above. But the problem seem to exist only on that
>>> "some_histogram" column which contains the mixed containsNull = true/false.
>>> Let me know if this helps.
>>>
>>> Thanks,
>>> Muthu
>>>
>>>
>>>
>>> On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <
>>> michael@databricks.com> wrote:
>>>
>>>> Nullable is just a hint to the optimizer that its impossible for there
>>>> to be a null value in this column, so that it can avoid generating code for
>>>> null-checks.  When in doubt, we set nullable=true since it is always safer
>>>> to check.
>>>>
>>>> Why in particular are you trying to change the nullability of the
>>>> column?
>>>>
>>>> On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hello there,
>>>>>
>>>>> I am trying to understand how and when does DataFrame (or Dataset)
>>>>> sets nullable = true vs false on a schema.
>>>>>
>>>>> Here is my observation from a sample code I tried...
>>>>>
>>>>>
>>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>>> lit("bla")).printSchema()
>>>>> root
>>>>>  |-- col1: integer (nullable = false)
>>>>>  |-- col2: string (nullable = true)
>>>>>  |-- col3: double (nullable = false)
>>>>>  |-- col4: string (nullable = false)
>>>>>
>>>>>
>>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>>> lit("bla")).write.parquet("/tmp/sample.parquet")
>>>>>
>>>>> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
>>>>> root
>>>>>  |-- col1: integer (nullable = true)
>>>>>  |-- col2: string (nullable = true)
>>>>>  |-- col3: double (nullable = true)
>>>>>  |-- col4: string (nullable = true)
>>>>>
>>>>>
>>>>> The place where this seem to get me into trouble is when I try to
>>>>> union one data-structure from in-memory (notice that in the below schema
>>>>> the highlighted element is represented as 'false' for in-memory created
>>>>> schema) and one from file that starts out with a schema like below...
>>>>>
>>>>>  |-- some_histogram: struct (nullable = true)
>>>>>  |    |-- values: array (nullable = true)
>>>>>  |    |    |-- element: double (containsNull = true)
>>>>>  |    |-- freq: array (nullable = true)
>>>>>  |    |    |-- element: long (containsNull = true)
>>>>>
>>>>> Is there a way to convert this attribute from true to false without
>>>>> running any mapping / udf on that column?
>>>>>
>>>>> Please advice,
>>>>> Muthu
>>>>>
>>>>
>>>>
>>>
>>
>
>
>

Re: Dataframe schema...

Posted by Cheng Lian <li...@databricks.com>.
Yea, confirmed. While analyzing unions, we treat StructTypes with 
different field nullabilities as incompatible types and throws this error.

Opened https://issues.apache.org/jira/browse/SPARK-18058 to track this 
issue. Thanks for reporting!

Cheng


On 10/21/16 3:15 PM, Cheng Lian wrote:
>
> Hi Muthu,
>
> What is the version of Spark are you using? This seems to be a bug in 
> the analysis phase.
>
> Cheng
>
>
> On 10/21/16 12:50 PM, Muthu Jayakumar wrote:
>> Sorry for the late response. Here is what I am seeing...
>>
>>
>> Schema from parquet file.
>> d1.printSchema()
>> root
>>  |-- task_id: string (nullable = true)
>>  |-- task_name: string (nullable = true)
>>  |-- some_histogram: struct (nullable = true)
>>  |    |-- values: array (nullable = true)
>>  |    |    |-- element: double (containsNull = true)
>>  |    |-- freq: array (nullable = true)
>>  |    |    |-- element: long (containsNull = true)
>>
>> d2.printSchema() //Data created using dataframe and/or processed before writing to 
>> parquet file.
>> root
>>  |-- task_id: string (nullable = true)
>>  |-- task_name: string (nullable = true)
>>  |-- some_histogram: struct (nullable = true)
>>  |    |-- values: array (nullable = true)
>>  |    |    |-- element: double (containsNull = false)
>>  |    |-- freq: array (nullable = true)
>>  |    |    |-- element: long (containsNull = false)
>>
>> d1.union(d2).printSchema()
>> Exception in thread "main" org.apache.spark.sql.AnalysisException: 
>> unresolved operator 'Union;
>> at 
>> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
>> at 
>> org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:58)
>> at 
>> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:361)
>> at 
>> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:67)
>> at 
>> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:126)
>> at 
>> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:67)
>> at 
>> org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:58)
>> at 
>> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:49)
>> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:161)
>> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
>> at org.apache.spark.sql.Dataset$.apply(Dataset.scala:59)
>> at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:2594)
>> at org.apache.spark.sql.Dataset.union(Dataset.scala:1459)
>>
>> Please advice,
>> Muthu
>>
>> On Thu, Oct 20, 2016 at 1:46 AM, Michael Armbrust 
>> <michael@databricks.com <ma...@databricks.com>> wrote:
>>
>>     What is the issue you see when unioning?
>>
>>     On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar
>>     <babloo80@gmail.com <ma...@gmail.com>> wrote:
>>
>>         Hello Michael,
>>
>>         Thank you for looking into this query. In my case there seem
>>         to be an issue when I union a parquet file read from disk
>>         versus another dataframe that I construct in-memory. The only
>>         difference I see is the containsNull = true. In fact, I do
>>         not see any errors with union on the simple schema of "col1
>>         thru col4" above. But the problem seem to exist only on that
>>         "some_histogram" column which contains the mixed containsNull
>>         = true/false.
>>         Let me know if this helps.
>>
>>         Thanks,
>>         Muthu
>>
>>
>>
>>         On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust
>>         <michael@databricks.com <ma...@databricks.com>> wrote:
>>
>>             Nullable is just a hint to the optimizer that its
>>             impossible for there to be a null value in this column,
>>             so that it can avoid generating code for null-checks. 
>>             When in doubt, we set nullable=true since it is always
>>             safer to check.
>>
>>             Why in particular are you trying to change the
>>             nullability of the column?
>>
>>             On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar
>>             <babloo80@gmail.com <ma...@gmail.com>> wrote:
>>
>>                 Hello there,
>>
>>                 I am trying to understand how and when does DataFrame
>>                 (or Dataset) sets nullable = true vs false on a schema.
>>
>>                 Here is my observation from a sample code I tried...
>>
>>
>>                 scala> spark.createDataset(Seq((1, "a", 2.0d), (2,
>>                 "b", 2.0d), (3, "c", 2.0d))).toDF("col1", "col2",
>>                 "col3").withColumn("col4", lit("bla")).printSchema()
>>                 root
>>                  |-- col1: integer (nullable = false)
>>                  |-- col2: string (nullable = true)
>>                  |-- col3: double (nullable = false)
>>                  |-- col4: string (nullable = false)
>>
>>
>>                 scala> spark.createDataset(Seq((1, "a", 2.0d), (2,
>>                 "b", 2.0d), (3, "c", 2.0d))).toDF("col1", "col2",
>>                 "col3").withColumn("col4",
>>                 lit("bla")).write.parquet("/tmp/sample.parquet")
>>
>>                 scala>
>>                 spark.read.parquet("/tmp/sample.parquet").printSchema()
>>                 root
>>                  |-- col1: integer (nullable = true)
>>                  |-- col2: string (nullable = true)
>>                  |-- col3: double (nullable = true)
>>                  |-- col4: string (nullable = true)
>>
>>
>>                 The place where this seem to get me into trouble is
>>                 when I try to union one data-structure from in-memory
>>                 (notice that in the below schema the highlighted
>>                 element is represented as 'false' for in-memory
>>                 created schema) and one from file that starts out
>>                 with a schema like below...
>>
>>                  |-- some_histogram: struct (nullable = true)
>>                  |  |-- values: array (nullable = true)
>>                  |    |    |-- element: double (containsNull = true)
>>                  |  |-- freq: array (nullable = true)
>>                  |    |    |-- element: long (containsNull = true)
>>
>>                 Is there a way to convert this attribute from true to
>>                 false without running any mapping / udf on that column?
>>
>>                 Please advice,
>>                 Muthu
>>
>>
>>
>>
>>
>


Re: Dataframe schema...

Posted by Cheng Lian <li...@databricks.com>.
Hi Muthu,

What is the version of Spark are you using? This seems to be a bug in 
the analysis phase.

Cheng


On 10/21/16 12:50 PM, Muthu Jayakumar wrote:
> Sorry for the late response. Here is what I am seeing...
>
>
> Schema from parquet file.
> d1.printSchema()
> root
>  |-- task_id: string (nullable = true)
>  |-- task_name: string (nullable = true)
>  |-- some_histogram: struct (nullable = true)
>  |    |-- values: array (nullable = true)
>  |    |    |-- element: double (containsNull = true)
>  |    |-- freq: array (nullable = true)
>  |    |    |-- element: long (containsNull = true)
>
> d2.printSchema() //Data created using dataframe and/or processed before writing to 
> parquet file.
> root
>  |-- task_id: string (nullable = true)
>  |-- task_name: string (nullable = true)
>  |-- some_histogram: struct (nullable = true)
>  |    |-- values: array (nullable = true)
>  |    |    |-- element: double (containsNull = false)
>  |    |-- freq: array (nullable = true)
>  |    |    |-- element: long (containsNull = false)
>
> d1.union(d2).printSchema()
> Exception in thread "main" org.apache.spark.sql.AnalysisException: 
> unresolved operator 'Union;
> at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
> at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:58)
> at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:361)
> at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:67)
> at 
> org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:126)
> at 
> org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:67)
> at 
> org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:58)
> at 
> org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:49)
> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:161)
> at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
> at org.apache.spark.sql.Dataset$.apply(Dataset.scala:59)
> at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:2594)
> at org.apache.spark.sql.Dataset.union(Dataset.scala:1459)
>
> Please advice,
> Muthu
>
> On Thu, Oct 20, 2016 at 1:46 AM, Michael Armbrust 
> <michael@databricks.com <ma...@databricks.com>> wrote:
>
>     What is the issue you see when unioning?
>
>     On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar
>     <babloo80@gmail.com <ma...@gmail.com>> wrote:
>
>         Hello Michael,
>
>         Thank you for looking into this query. In my case there seem
>         to be an issue when I union a parquet file read from disk
>         versus another dataframe that I construct in-memory. The only
>         difference I see is the containsNull = true. In fact, I do not
>         see any errors with union on the simple schema of "col1 thru
>         col4" above. But the problem seem to exist only on that
>         "some_histogram" column which contains the mixed containsNull
>         = true/false.
>         Let me know if this helps.
>
>         Thanks,
>         Muthu
>
>
>
>         On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust
>         <michael@databricks.com <ma...@databricks.com>> wrote:
>
>             Nullable is just a hint to the optimizer that its
>             impossible for there to be a null value in this column, so
>             that it can avoid generating code for null-checks. When in
>             doubt, we set nullable=true since it is always safer to
>             check.
>
>             Why in particular are you trying to change the nullability
>             of the column?
>
>             On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar
>             <babloo80@gmail.com <ma...@gmail.com>> wrote:
>
>                 Hello there,
>
>                 I am trying to understand how and when does DataFrame
>                 (or Dataset) sets nullable = true vs false on a schema.
>
>                 Here is my observation from a sample code I tried...
>
>
>                 scala> spark.createDataset(Seq((1, "a", 2.0d), (2,
>                 "b", 2.0d), (3, "c", 2.0d))).toDF("col1", "col2",
>                 "col3").withColumn("col4", lit("bla")).printSchema()
>                 root
>                  |-- col1: integer (nullable = false)
>                  |-- col2: string (nullable = true)
>                  |-- col3: double (nullable = false)
>                  |-- col4: string (nullable = false)
>
>
>                 scala> spark.createDataset(Seq((1, "a", 2.0d), (2,
>                 "b", 2.0d), (3, "c", 2.0d))).toDF("col1", "col2",
>                 "col3").withColumn("col4",
>                 lit("bla")).write.parquet("/tmp/sample.parquet")
>
>                 scala>
>                 spark.read.parquet("/tmp/sample.parquet").printSchema()
>                 root
>                  |-- col1: integer (nullable = true)
>                  |-- col2: string (nullable = true)
>                  |-- col3: double (nullable = true)
>                  |-- col4: string (nullable = true)
>
>
>                 The place where this seem to get me into trouble is
>                 when I try to union one data-structure from in-memory
>                 (notice that in the below schema the highlighted
>                 element is represented as 'false' for in-memory
>                 created schema) and one from file that starts out with
>                 a schema like below...
>
>                  |-- some_histogram: struct (nullable = true)
>                  |    |-- values: array (nullable = true)
>                  |    |  |-- element: double (containsNull = true)
>                  |    |-- freq: array (nullable = true)
>                  |    |  |-- element: long (containsNull = true)
>
>                 Is there a way to convert this attribute from true to
>                 false without running any mapping / udf on that column?
>
>                 Please advice,
>                 Muthu
>
>
>
>
>


Re: Dataframe schema...

Posted by Muthu Jayakumar <ba...@gmail.com>.
Sorry for the late response. Here is what I am seeing...


Schema from parquet file.

d1.printSchema()

root
 |-- task_id: string (nullable = true)
 |-- task_name: string (nullable = true)
 |-- some_histogram: struct (nullable = true)
 |    |-- values: array (nullable = true)
 |    |    |-- element: double (containsNull = true)
 |    |-- freq: array (nullable = true)
 |    |    |-- element: long (containsNull = true)

d2.printSchema() //Data created using dataframe and/or processed
before writing to parquet file.

root
 |-- task_id: string (nullable = true)
 |-- task_name: string (nullable = true)
 |-- some_histogram: struct (nullable = true)
 |    |-- values: array (nullable = true)
 |    |    |-- element: double (containsNull = false)
 |    |-- freq: array (nullable = true)
 |    |    |-- element: long (containsNull = false)

d1.union(d2).printSchema()

Exception in thread "main" org.apache.spark.sql.AnalysisException:
unresolved operator 'Union;
at
org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40)
at
org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:58)
at
org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:361)
at
org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1.apply(CheckAnalysis.scala:67)
at
org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:126)
at
org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:67)
at
org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:58)
at
org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:49)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:161)
at org.apache.spark.sql.Dataset.<init>(Dataset.scala:167)
at org.apache.spark.sql.Dataset$.apply(Dataset.scala:59)
at org.apache.spark.sql.Dataset.withTypedPlan(Dataset.scala:2594)
at org.apache.spark.sql.Dataset.union(Dataset.scala:1459)

Please advice,
Muthu

On Thu, Oct 20, 2016 at 1:46 AM, Michael Armbrust <mi...@databricks.com>
wrote:

> What is the issue you see when unioning?
>
> On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar <ba...@gmail.com>
> wrote:
>
>> Hello Michael,
>>
>> Thank you for looking into this query. In my case there seem to be an
>> issue when I union a parquet file read from disk versus another dataframe
>> that I construct in-memory. The only difference I see is the containsNull =
>> true. In fact, I do not see any errors with union on the simple schema of
>> "col1 thru col4" above. But the problem seem to exist only on that
>> "some_histogram" column which contains the mixed containsNull = true/false.
>> Let me know if this helps.
>>
>> Thanks,
>> Muthu
>>
>>
>>
>> On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <michael@databricks.com
>> > wrote:
>>
>>> Nullable is just a hint to the optimizer that its impossible for there
>>> to be a null value in this column, so that it can avoid generating code for
>>> null-checks.  When in doubt, we set nullable=true since it is always safer
>>> to check.
>>>
>>> Why in particular are you trying to change the nullability of the column?
>>>
>>> On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com>
>>> wrote:
>>>
>>>> Hello there,
>>>>
>>>> I am trying to understand how and when does DataFrame (or Dataset) sets
>>>> nullable = true vs false on a schema.
>>>>
>>>> Here is my observation from a sample code I tried...
>>>>
>>>>
>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>> lit("bla")).printSchema()
>>>> root
>>>>  |-- col1: integer (nullable = false)
>>>>  |-- col2: string (nullable = true)
>>>>  |-- col3: double (nullable = false)
>>>>  |-- col4: string (nullable = false)
>>>>
>>>>
>>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3,
>>>> "c", 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>>> lit("bla")).write.parquet("/tmp/sample.parquet")
>>>>
>>>> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
>>>> root
>>>>  |-- col1: integer (nullable = true)
>>>>  |-- col2: string (nullable = true)
>>>>  |-- col3: double (nullable = true)
>>>>  |-- col4: string (nullable = true)
>>>>
>>>>
>>>> The place where this seem to get me into trouble is when I try to union
>>>> one data-structure from in-memory (notice that in the below schema the
>>>> highlighted element is represented as 'false' for in-memory created schema)
>>>> and one from file that starts out with a schema like below...
>>>>
>>>>  |-- some_histogram: struct (nullable = true)
>>>>  |    |-- values: array (nullable = true)
>>>>  |    |    |-- element: double (containsNull = true)
>>>>  |    |-- freq: array (nullable = true)
>>>>  |    |    |-- element: long (containsNull = true)
>>>>
>>>> Is there a way to convert this attribute from true to false without
>>>> running any mapping / udf on that column?
>>>>
>>>> Please advice,
>>>> Muthu
>>>>
>>>
>>>
>>
>

Re: Dataframe schema...

Posted by Michael Armbrust <mi...@databricks.com>.
What is the issue you see when unioning?

On Wed, Oct 19, 2016 at 6:39 PM, Muthu Jayakumar <ba...@gmail.com> wrote:

> Hello Michael,
>
> Thank you for looking into this query. In my case there seem to be an
> issue when I union a parquet file read from disk versus another dataframe
> that I construct in-memory. The only difference I see is the containsNull =
> true. In fact, I do not see any errors with union on the simple schema of
> "col1 thru col4" above. But the problem seem to exist only on that
> "some_histogram" column which contains the mixed containsNull = true/false.
> Let me know if this helps.
>
> Thanks,
> Muthu
>
>
>
> On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <mi...@databricks.com>
> wrote:
>
>> Nullable is just a hint to the optimizer that its impossible for there to
>> be a null value in this column, so that it can avoid generating code for
>> null-checks.  When in doubt, we set nullable=true since it is always safer
>> to check.
>>
>> Why in particular are you trying to change the nullability of the column?
>>
>> On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com>
>> wrote:
>>
>>> Hello there,
>>>
>>> I am trying to understand how and when does DataFrame (or Dataset) sets
>>> nullable = true vs false on a schema.
>>>
>>> Here is my observation from a sample code I tried...
>>>
>>>
>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
>>> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>> lit("bla")).printSchema()
>>> root
>>>  |-- col1: integer (nullable = false)
>>>  |-- col2: string (nullable = true)
>>>  |-- col3: double (nullable = false)
>>>  |-- col4: string (nullable = false)
>>>
>>>
>>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
>>> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>>> lit("bla")).write.parquet("/tmp/sample.parquet")
>>>
>>> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
>>> root
>>>  |-- col1: integer (nullable = true)
>>>  |-- col2: string (nullable = true)
>>>  |-- col3: double (nullable = true)
>>>  |-- col4: string (nullable = true)
>>>
>>>
>>> The place where this seem to get me into trouble is when I try to union
>>> one data-structure from in-memory (notice that in the below schema the
>>> highlighted element is represented as 'false' for in-memory created schema)
>>> and one from file that starts out with a schema like below...
>>>
>>>  |-- some_histogram: struct (nullable = true)
>>>  |    |-- values: array (nullable = true)
>>>  |    |    |-- element: double (containsNull = true)
>>>  |    |-- freq: array (nullable = true)
>>>  |    |    |-- element: long (containsNull = true)
>>>
>>> Is there a way to convert this attribute from true to false without
>>> running any mapping / udf on that column?
>>>
>>> Please advice,
>>> Muthu
>>>
>>
>>
>

Re: Dataframe schema...

Posted by Muthu Jayakumar <ba...@gmail.com>.
Hello Michael,

Thank you for looking into this query. In my case there seem to be an issue
when I union a parquet file read from disk versus another dataframe that I
construct in-memory. The only difference I see is the containsNull = true.
In fact, I do not see any errors with union on the simple schema of "col1
thru col4" above. But the problem seem to exist only on that
"some_histogram" column which contains the mixed containsNull = true/false.
Let me know if this helps.

Thanks,
Muthu



On Wed, Oct 19, 2016 at 6:21 PM, Michael Armbrust <mi...@databricks.com>
wrote:

> Nullable is just a hint to the optimizer that its impossible for there to
> be a null value in this column, so that it can avoid generating code for
> null-checks.  When in doubt, we set nullable=true since it is always safer
> to check.
>
> Why in particular are you trying to change the nullability of the column?
>
> On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com>
> wrote:
>
>> Hello there,
>>
>> I am trying to understand how and when does DataFrame (or Dataset) sets
>> nullable = true vs false on a schema.
>>
>> Here is my observation from a sample code I tried...
>>
>>
>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
>> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>> lit("bla")).printSchema()
>> root
>>  |-- col1: integer (nullable = false)
>>  |-- col2: string (nullable = true)
>>  |-- col3: double (nullable = false)
>>  |-- col4: string (nullable = false)
>>
>>
>> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
>> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
>> lit("bla")).write.parquet("/tmp/sample.parquet")
>>
>> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
>> root
>>  |-- col1: integer (nullable = true)
>>  |-- col2: string (nullable = true)
>>  |-- col3: double (nullable = true)
>>  |-- col4: string (nullable = true)
>>
>>
>> The place where this seem to get me into trouble is when I try to union
>> one data-structure from in-memory (notice that in the below schema the
>> highlighted element is represented as 'false' for in-memory created schema)
>> and one from file that starts out with a schema like below...
>>
>>  |-- some_histogram: struct (nullable = true)
>>  |    |-- values: array (nullable = true)
>>  |    |    |-- element: double (containsNull = true)
>>  |    |-- freq: array (nullable = true)
>>  |    |    |-- element: long (containsNull = true)
>>
>> Is there a way to convert this attribute from true to false without
>> running any mapping / udf on that column?
>>
>> Please advice,
>> Muthu
>>
>
>

Re: Dataframe schema...

Posted by Michael Armbrust <mi...@databricks.com>.
Nullable is just a hint to the optimizer that its impossible for there to
be a null value in this column, so that it can avoid generating code for
null-checks.  When in doubt, we set nullable=true since it is always safer
to check.

Why in particular are you trying to change the nullability of the column?

On Wed, Oct 19, 2016 at 6:07 PM, Muthu Jayakumar <ba...@gmail.com> wrote:

> Hello there,
>
> I am trying to understand how and when does DataFrame (or Dataset) sets
> nullable = true vs false on a schema.
>
> Here is my observation from a sample code I tried...
>
>
> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
> lit("bla")).printSchema()
> root
>  |-- col1: integer (nullable = false)
>  |-- col2: string (nullable = true)
>  |-- col3: double (nullable = false)
>  |-- col4: string (nullable = false)
>
>
> scala> spark.createDataset(Seq((1, "a", 2.0d), (2, "b", 2.0d), (3, "c",
> 2.0d))).toDF("col1", "col2", "col3").withColumn("col4",
> lit("bla")).write.parquet("/tmp/sample.parquet")
>
> scala> spark.read.parquet("/tmp/sample.parquet").printSchema()
> root
>  |-- col1: integer (nullable = true)
>  |-- col2: string (nullable = true)
>  |-- col3: double (nullable = true)
>  |-- col4: string (nullable = true)
>
>
> The place where this seem to get me into trouble is when I try to union
> one data-structure from in-memory (notice that in the below schema the
> highlighted element is represented as 'false' for in-memory created schema)
> and one from file that starts out with a schema like below...
>
>  |-- some_histogram: struct (nullable = true)
>  |    |-- values: array (nullable = true)
>  |    |    |-- element: double (containsNull = true)
>  |    |-- freq: array (nullable = true)
>  |    |    |-- element: long (containsNull = true)
>
> Is there a way to convert this attribute from true to false without
> running any mapping / udf on that column?
>
> Please advice,
> Muthu
>