Skip to content

Conversation

@yhuai
Copy link
Contributor

@yhuai yhuai commented Feb 28, 2015

This PR contains the following changes:

  1. Add a new method, DataType.equalsIgnoreCompatibleNullability, which is the middle ground between DataType's equality check and DataType.equalsIgnoreNullability. For two data types from and to, it does equalsIgnoreNullability as well as if the nullability of from is compatible with that of to. For example, the nullability of ArrayType(IntegerType, containsNull = false) is compatible with that of ArrayType(IntegerType, containsNull = true) (for an array without null values, we can always say it may contain null values). However, the nullability of ArrayType(IntegerType, containsNull = true) is incompatible with that of ArrayType(IntegerType, containsNull = false) (for an array that may have null values, we cannot say it does not have null values).
  2. For the resolved field of InsertIntoTable, use equalsIgnoreCompatibleNullability to replace the equality check of the data types.
  3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
  4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to false. This change makes the whole data pipeline more robust.
  5. Update the equality check of JSON relation. Since JSON does not really cares nullability, equalsIgnoreNullability seems a better choice to compare schemata from to JSON tables.

JIRA: https://issues.apache.org/jira/browse/SPARK-5950

Thanks @viirya for the initial work in #4729.

cc @marmbrus @liancheng

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28113 has started for PR 4826 at commit e4f397c.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28113 has finished for PR 4826 at commit e4f397c.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28113/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28119 has started for PR 4826 at commit 0eb5578.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28119 has finished for PR 4826 at commit 0eb5578.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28119/
Test FAILed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@mengxr Is it OK to ignore the nullability of ArrayType (containsNull field) and MapType (valueContainsNull field) in this check?

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28132 has started for PR 4826 at commit 8360817.

  • This patch merges cleanly.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@liancheng @marmbrus I am also changing the nullability for our old parquet write path to make the behavior consistent with our new write path. Let me know if there is any potential compatibility issue and we should revert this change.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we also make data types of ParquetRelation.output always nullable?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Never mind, since we always write nullable data, it should be OK to leave ParquetRelation.output untouched.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Verified that when merging schemas, official Parquet implementation will handle nullability (repetition level) properly. So our change should be safe for interoperation with other systems that support Parquet schema evolving.

@SparkQA
Copy link

SparkQA commented Feb 28, 2015

Test build #28132 has finished for PR 4826 at commit 8360817.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28132/
Test PASSed.

@SparkQA
Copy link

SparkQA commented Mar 1, 2015

Test build #28137 has started for PR 4826 at commit d3747d1.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 1, 2015

Test build #28137 has finished for PR 4826 at commit d3747d1.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28137/
Test PASSed.

@liancheng
Copy link
Contributor

A typo in the PR description:

However, the nullability of ArrayType(IntegerType, containsNull = true) is compatible with that of ArrayType(IntegerType, containsNull = false)

Here "compatible" should be "incompatible".

@marmbrus
Copy link
Contributor

marmbrus commented Mar 2, 2015

@liancheng I said basically the same thing to @yhuai when we were discussing it last week. It does seem like nullability has been only a burden. That said, it could allow us to do quite a few optimizations in the future if we keep it, so I'm kind of torn. We should discuss and maybe benchmark.

Stylistically, I think we should be adding these methods to datatypes themselves. I'd suggest that DataTypes have a method called sameType that returns true when types are the same modulo nullability, instead of having a static method in DataType. Pretty much every place where we now uses == should be using this function instead. I'd also have method asNullable that returns a DataType with nullability flipped on. This does not have to happen in this PR if its too much work.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can introduce a method to the class of DataType based on this one later (I am not sure what will be a good name. I thought about compatibleWith, but I feel it is not very accurate).

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28187 has started for PR 4826 at commit 0cb7ea2.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28187 has finished for PR 4826 at commit 0cb7ea2.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28187/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28193 has started for PR 4826 at commit 587d88b.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28193 has finished for PR 4826 at commit 587d88b.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28193/
Test FAILed.

@SparkQA
Copy link

SparkQA commented Mar 2, 2015

Test build #28199 has started for PR 4826 at commit 80e487e.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28199 has finished for PR 4826 at commit 80e487e.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
    • class MatrixFactorizationModel(JavaModelWrapper, Saveable, JavaLoader):
    • class Saveable(object):
    • class Loader(object):
    • class JavaLoader(Loader):
    • java_class = ".".join([java_package, cls.__name__])
    • logError("User class threw exception: " + cause.getMessage, cause)

@AmplabJenkins
Copy link

Test FAILed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28199/
Test FAILed.

@yhuai
Copy link
Contributor Author

yhuai commented Mar 3, 2015

test this please

@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28204 has started for PR 4826 at commit 80e487e.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28212 has started for PR 4826 at commit 3b61a04.

  • This patch merges cleanly.

@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28204 has finished for PR 4826 at commit 80e487e.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28204/
Test PASSed.

asfgit pushed a commit that referenced this pull request Mar 3, 2015
… should work when using datasource api

This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However,  the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability,  `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.

JIRA: https://issues.apache.org/jira/browse/SPARK-5950

Thanks viirya for the initial work in #4729.

cc marmbrus liancheng

Author: Yin Huai <[email protected]>

Closes #4826 from yhuai/insertNullabilityCheck and squashes the following commits:

3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.

(cherry picked from commit 1259994)
Signed-off-by: Michael Armbrust <[email protected]>
@asfgit asfgit closed this in 1259994 Mar 3, 2015
@SparkQA
Copy link

SparkQA commented Mar 3, 2015

Test build #28212 has finished for PR 4826 at commit 3b61a04.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@AmplabJenkins
Copy link

Test PASSed.
Refer to this link for build results (access rights to CI server needed):
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/28212/
Test PASSed.

liancheng added a commit to liancheng/spark that referenced this pull request May 11, 2015
- Only data columns should be passed
- Data schema should be converted to nullable version (see SPARK-5950 and PR apache#4826)
liancheng added a commit to liancheng/spark that referenced this pull request May 11, 2015
- Only data columns should be passed
- Data schema should be converted to nullable version (see SPARK-5950 and PR apache#4826)
liancheng added a commit to liancheng/spark that referenced this pull request May 12, 2015
- Only data columns should be passed
- Data schema should be converted to nullable version (see SPARK-5950 and PR apache#4826)
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants