You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by 0x0FFF <gi...@git.apache.org> on 2015/09/01 15:37:12 UTC

[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

GitHub user 0x0FFF opened a pull request:

    https://github.com/apache/spark/pull/8556

    [SPARK-10392] [SQL] Pyspark - Wrong DateType support on JDBC connection

    This PR addresses issue [SPARK-10392](https://issues.apache.org/jira/browse/SPARK-10392)
    The problem is that for "start of epoch" date (01 Jan 1970) PySpark class DateType returns 0 instead of the `datetime.date` due to implementation of its return statement
    
    Issue reproduction on master:
    ```
    >>> from pyspark.sql.types import *
    >>> a = DateType()
    >>> a.fromInternal(0)
    0
    >>> a.fromInternal(1)
    datetime.date(1970, 1, 2)
    ```

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/0x0FFF/spark SPARK-10392

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/8556.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #8556
    
----
commit 7985de776958ddec7335a3399f2a6d772dbb18e1
Author: 0x0FFF <pr...@gmail.com>
Date:   2015-09-01T13:30:22Z

    [SPARK-10392] [SQL] Pyspark - Wrong DateType support on JDBC connection

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136845665
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136854495
  
      [Test build #41889 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41889/consoleFull) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136806465
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41875/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136825789
  
    @0x0FFF Thanks for working on this, could you add a regression test for it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38466414
  
    --- Diff: python/pyspark/sql/tests.py ---
    @@ -1164,6 +1164,9 @@ def test_window_functions_without_partitionBy(self):
             for r, ex in zip(rs, expected):
                 self.assertEqual(tuple(r), ex[:len(r)])
     
    +    def test_datetype_equal_zero(self):
    --- End diff --
    
    This test case don't need sqlCtx, it's better to be inside DataTypeTests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136837411
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136724447
  
    Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136863236
  
      [Test build #1708 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1708/consoleFull) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/8556


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136893568
  
      [Test build #1709 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1709/console) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class DCT(JavaTransformer, HasInputCol, HasOutputCol):`
      * `class SQLTransformer(JavaTransformer):`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136862025
  
    Jenkins, retest this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136837439
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136858263
  
      [Test build #41889 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41889/console) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136871851
  
      [Test build #1708 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1708/console) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class DCT(JavaTransformer, HasInputCol, HasOutputCol):`
      * `class SQLTransformer(JavaTransformer):`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38456406
  
    --- Diff: python/pyspark/sql/types.py ---
    @@ -168,10 +168,12 @@ def needConversion(self):
             return True
     
         def toInternal(self, d):
    -        return d and d.toordinal() - self.EPOCH_ORDINAL
    +        if d is not None:
    --- End diff --
    
    I think this change is not needed, `d` will always be true if it's not None


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136876574
  
    merged into master, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38460209
  
    --- Diff: python/pyspark/sql/types.py ---
    @@ -168,10 +168,12 @@ def needConversion(self):
             return True
     
         def toInternal(self, d):
    -        return d and d.toordinal() - self.EPOCH_ORDINAL
    +        if d is not None:
    --- End diff --
    
    It will be false for None, empty string, empty list, 0. This way it will be returned instead of the date:
    ```
    >>> [] and 'foo'
    []
    >>> 0 and 'foo'
    0
    >>> 1 and 'foo'
    'foo'
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136836673
  
    Added regression test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136862005
  
    Failed mllib test for python2.6, I didn't change anything that might have affected it. Same test passes locally on my machine


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136800148
  
      [Test build #41875 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41875/consoleFull) for   PR 8556 at commit [`7985de7`](https://github.com/apache/spark/commit/7985de776958ddec7335a3399f2a6d772dbb18e1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136845669
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41886/
    Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136853046
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136806464
  
    Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136863299
  
      [Test build #1709 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1709/consoleFull) for   PR 8556 at commit [`a7fd681`](https://github.com/apache/spark/commit/a7fd68156764960ebf773912d1dfcebb331cf55a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136845521
  
      [Test build #41886 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41886/console) for   PR 8556 at commit [`f41b125`](https://github.com/apache/spark/commit/f41b125c7e0443828ff69cb5a9bffeeba268b680).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `public class JavaTrainValidationSplitExample `
      * `class DCT(JavaTransformer, HasInputCol, HasOutputCol):`
      * `class SQLTransformer(JavaTransformer):`
      * `case class LimitNode(limit: Int, child: LocalNode) extends UnaryLocalNode `
      * `case class UnionNode(children: Seq[LocalNode]) extends LocalNode `



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136798336
  
    Jenkins, this is ok to test.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136855093
  
    LGTM, will merge once it pass tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136806331
  
      [Test build #41875 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41875/console) for   PR 8556 at commit [`7985de7`](https://github.com/apache/spark/commit/7985de776958ddec7335a3399f2a6d772dbb18e1).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136798504
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by andrewor14 <gi...@git.apache.org>.
Github user andrewor14 commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136816836
  
    @davies


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38460758
  
    --- Diff: python/pyspark/sql/types.py ---
    @@ -168,10 +168,12 @@ def needConversion(self):
             return True
     
         def toInternal(self, d):
    -        return d and d.toordinal() - self.EPOCH_ORDINAL
    +        if d is not None:
    --- End diff --
    
    `d` could not be list/int/string, it can only be Date or None.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136862977
  
    @0x0FFF The JVM died during tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136837583
  
      [Test build #41886 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41886/consoleFull) for   PR 8556 at commit [`f41b125`](https://github.com/apache/spark/commit/f41b125c7e0443828ff69cb5a9bffeeba268b680).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38466353
  
    --- Diff: python/pyspark/sql/types.py ---
    @@ -168,10 +168,12 @@ def needConversion(self):
             return True
     
         def toInternal(self, d):
    -        return d and d.toordinal() - self.EPOCH_ORDINAL
    +        if d is not None:
    --- End diff --
    
    Either is good to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136858398
  
    Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136853088
  
    Merged build started.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136858404
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/41889/
    Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136852809
  
    Moved regression test to DataTypeTests class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by 0x0FFF <gi...@git.apache.org>.
Github user 0x0FFF commented on a diff in the pull request:

    https://github.com/apache/spark/pull/8556#discussion_r38462716
  
    --- Diff: python/pyspark/sql/types.py ---
    @@ -168,10 +168,12 @@ def needConversion(self):
             return True
     
         def toInternal(self, d):
    -        return d and d.toordinal() - self.EPOCH_ORDINAL
    +        if d is not None:
    --- End diff --
    
    Sorry, I see. Should I revert this change? It is mostly for both methods to follow the same logic, as `return x and y` is not a very readable code in my opinion. Also the class `TimestampType` is implemented in a similar way - both methods are starting with input parameter check for None


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request: [SPARK-10392] [SQL] Pyspark - Wrong DateType s...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the pull request:

    https://github.com/apache/spark/pull/8556#issuecomment-136798468
  
     Merged build triggered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org