You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by HyukjinKwon <gi...@git.apache.org> on 2018/10/19 09:05:15 UTC

[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

GitHub user HyukjinKwon opened a pull request:

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

    [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's input json as literal only

    ## What changes were proposed in this pull request?
    
    The main purpose of `schema_of_json` is the usage of combination with `from_json` (to make up the leak of schema inference) which takes its schema only as literal; however, currently `schema_of_json` allows JSON input as non-literal expressions (e.g, column).
    
    This was mistakenly allowed - we don't have to take other usages rather then the main purpose into account for now.
    
    This PR makes a followup to only allow literals for `schema_of_json`'s JSON input. We can allow non literal expressions later when it's needed or there are some usecase for it.
    
    ## How was this patch tested?
    
    Unit tests were added.

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

    $ git pull https://github.com/HyukjinKwon/spark SPARK-25447-followup

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

    https://github.com/apache/spark/pull/22775.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 #22775
    
----
commit 2705aa8739ecfc1d2ea8aea40082734084f36514
Author: hyukjinkwon <gu...@...>
Date:   2018-10-19T06:40:58Z

    Make schema_of_json's input json as liternal only

----


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4463/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4522/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    if we are ok with this direction, this LGTM except a few minor comments. Thanks!


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r227243187
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    so `schem_of_json` must be used with `from_json`? I don't have a strong feeling to enforce it. It's also weird that users are willing to write verbose json literal in `from_json(..., schema = schema_of_json(...))` instead of DDL string.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97661 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97661/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r227237973
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    is it possible that users want to run `schema_of_json` on a string column of table?


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4141/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Ah.. let me rebase and sync the tests


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Actually this is not that hard. The conflict comes from the fact that in 2.4 `schema_of_json` doesn't take `option` parameter.
    
    I've fixed the conflict and pushed to 2.4. You can take a look at the commit and see if there is something wrong. I ran the touched tests locally to verify it.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97905 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97905/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228389378
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -2375,20 +2375,22 @@ def schema_of_json(col, options={}):
         .. versionchanged:: 3.0
            It accepts `options` parameter to control schema inferring.
     
    -    >>> from pyspark.sql.types import *
    -    >>> data = [(1, '{"a": 1}')]
    -    >>> df = spark.createDataFrame(data, ("key", "value"))
    -    >>> df.select(schema_of_json(df.value).alias("json")).collect()
    -    [Row(json=u'struct<a:bigint>')]
    +    >>> df = spark.range(1)
         >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
    -    >>> schema = schema_of_json(lit('{a: 1}'), {'allowUnquotedFieldNames':'true'})
    +    >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
         >>> df.select(schema.alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
         """
    +    if isinstance(col, basestring):
    --- End diff --
    
    shall we do the same for scala APIs? i.e. create `def schema_of_json(json: String)`


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4149/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97605 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97605/testReport)** for PR 22775 at commit [`2705aa8`](https://github.com/apache/spark/commit/2705aa8739ecfc1d2ea8aea40082734084f36514).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97910 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97910/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4398/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4520/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Yup, will fix.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97649 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97649/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).
     * This patch **fails PySpark pip packaging tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    The latest Python failure looks relevant to this PR.
    ```scala
    AnalysisException: u"cannot resolve 'schemaofjson(`value`)' due to data type mismatch: The input json should be a string literal and not null; however, got `value`.;;\n'Project [schemaofjson(value#2191) AS json#2194]\n+- LogicalRDD [key#2190L, value#2191], false\n"
    ```


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r227239707
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    Also, basically we expect structured data format in SQL path, and different DDL type strings for each record wouldn't be quite useful anyway.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228389510
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -752,14 +752,20 @@ case class StructsToJson(
     case class SchemaOfJson(
         child: Expression,
         options: Map[String, String])
    -  extends UnaryExpression with String2StringExpression with CodegenFallback {
    +  extends UnaryExpression with ExpectsInputTypes with CodegenFallback {
     
       def this(child: Expression) = this(child, Map.empty[String, String])
     
       def this(child: Expression, options: Expression) = this(
           child = child,
           options = ExprUtils.convertToMapData(options))
     
    +  override def dataType: DataType = StringType
    +
    +  override def inputTypes: Seq[DataType] = Seq(StringType)
    --- End diff --
    
    why do we need it since we already override `checkInputDataTypes`?


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228520891
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -2365,30 +2365,32 @@ def to_json(col, options={}):
     
     @ignore_unicode_prefix
     @since(2.4)
    -def schema_of_json(col, options={}):
    +def schema_of_json(json, options={}):
         """
    -    Parses a column containing a JSON string and infers its schema in DDL format.
    +    Parses a JSON string and infers its schema in DDL format.
     
    -    :param col: string column in json format
    +    :param json: a JSON string or a string literal containing a JSON string.
         :param options: options to control parsing. accepts the same options as the JSON datasource
     
         .. versionchanged:: 3.0
            It accepts `options` parameter to control schema inferring.
     
    -    >>> from pyspark.sql.types import *
    -    >>> data = [(1, '{"a": 1}')]
    -    >>> df = spark.createDataFrame(data, ("key", "value"))
    -    >>> df.select(schema_of_json(df.value).alias("json")).collect()
    -    [Row(json=u'struct<a:bigint>')]
    +    >>> df = spark.range(1)
         >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
    -    >>> schema = schema_of_json(lit('{a: 1}'), {'allowUnquotedFieldNames':'true'})
    +    >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
         >>> df.select(schema.alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
         """
    +    if isinstance(json, basestring):
    --- End diff --
    
    Actually, Wenchen, I think that's going to make it a bit complicated after few more thoughts .. I think it's okay to go ahead. It's kind of a mistake that we had to do in 2.4.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97610 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97610/testReport)** for PR 22775 at commit [`2705aa8`](https://github.com/apache/spark/commit/2705aa8739ecfc1d2ea8aea40082734084f36514).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97905 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97905/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    @cloud-fan, mind taking a look please?


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97605 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97605/testReport)** for PR 22775 at commit [`2705aa8`](https://github.com/apache/spark/commit/2705aa8739ecfc1d2ea8aea40082734084f36514).


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98008 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98008/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Maybe I am too much careful about it but I am kind of nervous about this column case. I don't intend to disallow it entirely but only for Spark 2.4. We might have to find a way to use column column with `from_json` but I want to defer to 3.0.


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228504453
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -2365,30 +2365,32 @@ def to_json(col, options={}):
     
     @ignore_unicode_prefix
     @since(2.4)
    -def schema_of_json(col, options={}):
    +def schema_of_json(json, options={}):
         """
    -    Parses a column containing a JSON string and infers its schema in DDL format.
    +    Parses a JSON string and infers its schema in DDL format.
     
    -    :param col: string column in json format
    +    :param json: a JSON string or a string literal containing a JSON string.
         :param options: options to control parsing. accepts the same options as the JSON datasource
     
         .. versionchanged:: 3.0
            It accepts `options` parameter to control schema inferring.
     
    -    >>> from pyspark.sql.types import *
    -    >>> data = [(1, '{"a": 1}')]
    -    >>> df = spark.createDataFrame(data, ("key", "value"))
    -    >>> df.select(schema_of_json(df.value).alias("json")).collect()
    -    [Row(json=u'struct<a:bigint>')]
    +    >>> df = spark.range(1)
         >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
    -    >>> schema = schema_of_json(lit('{a: 1}'), {'allowUnquotedFieldNames':'true'})
    +    >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
         >>> df.select(schema.alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
         """
    +    if isinstance(json, basestring):
    --- End diff --
    
    +1


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    I agree it should be a literal value.



---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228499222
  
    --- Diff: python/pyspark/sql/functions.py ---
    @@ -2365,30 +2365,32 @@ def to_json(col, options={}):
     
     @ignore_unicode_prefix
     @since(2.4)
    -def schema_of_json(col, options={}):
    +def schema_of_json(json, options={}):
         """
    -    Parses a column containing a JSON string and infers its schema in DDL format.
    +    Parses a JSON string and infers its schema in DDL format.
     
    -    :param col: string column in json format
    +    :param json: a JSON string or a string literal containing a JSON string.
         :param options: options to control parsing. accepts the same options as the JSON datasource
     
         .. versionchanged:: 3.0
            It accepts `options` parameter to control schema inferring.
     
    -    >>> from pyspark.sql.types import *
    -    >>> data = [(1, '{"a": 1}')]
    -    >>> df = spark.createDataFrame(data, ("key", "value"))
    -    >>> df.select(schema_of_json(df.value).alias("json")).collect()
    -    [Row(json=u'struct<a:bigint>')]
    +    >>> df = spark.range(1)
         >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
    -    >>> schema = schema_of_json(lit('{a: 1}'), {'allowUnquotedFieldNames':'true'})
    +    >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'})
         >>> df.select(schema.alias("json")).collect()
         [Row(json=u'struct<a:bigint>')]
         """
    +    if isinstance(json, basestring):
    --- End diff --
    
    after more thoughts, maybe we should not add new features to 2.4? We can accept strings directly in 3.0.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228389610
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    +
    +  override def checkInputDataTypes(): TypeCheckResult = child match {
    +    case Literal(s, StringType) if s != null => super.checkInputDataTypes()
    +    case _ => TypeCheckResult.TypeCheckFailure(
    +      s"The input json should be a string literal and not null; however, got ${child.sql}.")
    +  }
    +
    +  override def eval(v: InternalRow = EmptyRow): Any = {
    --- End diff --
    
    when implementing `eval`, we usually don't put the default value. Shall we follow this code style?


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97910 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97910/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98067 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98067/testReport)** for PR 22775 at commit [`03f34d9`](https://github.com/apache/spark/commit/03f34d9d86a8087c3de4d5580e2ddf9fba8a8407).


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r228372331
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    It's not weird that users want to use schema_of_json at all.
    
    Imagine it's a very large json with very complicated string. It's pretty difficult to actually write the ddl string.



---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98078 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98078/testReport)** for PR 22775 at commit [`8e6b97a`](https://github.com/apache/spark/commit/8e6b97a35f0d7c384c390f0a463e47f9160f0e32).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4128/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98001 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98001/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r227239024
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    It is possible and that's one case I am trying to disallow here. `from_json` only accepts `schema_of_json` when the input is literals. I can't imagine a proper usecase by DDL formatted strings from a JSON column.
    
    The main purpose of this expression was `from_json(..., schema = schema_of_json(...))`. I was trying to allow what I need only.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Oh you mean the conflict fixing is not that hard. Thanks for doing this @cloud-fan. I planned to do this today .. :-).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    I think I'm not qualified to make the decision here, as I don't fully understand the use case.
    
    It looks to me that one use case would be to run `schema_of_json` on a column and manually figure out what's the final schema, and then use this schema in `from_json`. If `schema_of_string` only accepts literal, I'm not how users would use it.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4169/
    Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4469/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4155/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98066 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98066/testReport)** for PR 22775 at commit [`e2ca651`](https://github.com/apache/spark/commit/e2ca6517098adc093f957a6158ed760fb0826f4d).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    thanks, merging to master! can you send a new PR for 2.4? it conflicts


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97661 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97661/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Yea, but I meant a bit complicated but I'm okay in that way @cloud-fan. Thanks for doing that. I planed to do it today (now) :).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97631 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97631/testReport)** for PR 22775 at commit [`9cb0b94`](https://github.com/apache/spark/commit/9cb0b946f95f881ba9203f785cc6446f93244157).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97640 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97640/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4395/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4527/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json...

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

    https://github.com/apache/spark/pull/22775#discussion_r227251515
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala ---
    @@ -770,8 +776,17 @@ case class SchemaOfJson(
         factory
       }
     
    -  override def convert(v: UTF8String): UTF8String = {
    -    val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser =>
    +  @transient
    +  private lazy val json = child.eval().asInstanceOf[UTF8String]
    --- End diff --
    
    Yea, that was my thought, and yea I don't strongly feel we should enforce it too in the future but was thinking we better enforce for 2.4.x (if the RC fails).
    
    I was thinking about other possibilities to use this expression with `from_json` (for instance, let's say .. as an aggregation expression or .. automatically somehow collect few examples from a column and return a schema). Actually Max proposed an aggregation expression first and that got rejected by Reynold, and this way was suggested by him.
    
    > It's also weird that users are willing to write verbose json literal in from_json(..., schema = schema_of_json(...)) instead of DDL string.
    
    Yea, it is, but that's what current codes do (https://github.com/apache/spark/blob/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala#L803).
    
    Nothing serious but I was just thinking about allowing what we need for now given there have been some discussion about it before.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    This should be targeted to 2.4 .. otherwise we should describe the behaviour change at migration note.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98067 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98067/testReport)** for PR 22775 at commit [`03f34d9`](https://github.com/apache/spark/commit/03f34d9d86a8087c3de4d5580e2ddf9fba8a8407).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97646 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97646/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97610 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97610/testReport)** for PR 22775 at commit [`2705aa8`](https://github.com/apache/spark/commit/2705aa8739ecfc1d2ea8aea40082734084f36514).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Yup, yup .. I should sync the tests


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97646 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97646/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).
     * This patch **fails PySpark pip packaging tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Actually, that usecase can more easily accomplished by simply inferring schema by JSON datasource. Yea, I indeed suggested that as workaround for this issue before. Let's say, `spark.read.json(df.select("json").as[String]).schema`.
    
    I know it's not super clear about the usecase of `schema_of_json` and actually that's also partly why I want to allow what we need for this expression now.
    
    @rxin, WDYT? This PR tries to allow what we only need for now. Let's say disallow:
    
    ```scala
    schema_of_json(column)
    ```
    
    and only allow 
    
    ```scala
    schema_of_json(literal)
    ```
    
    because the main usecase is:
    
    ```scala
    from_json(schema_of_json(literal))
    ```
    
    and
    
    ```scala
    from_json(schema_of_json(column))
    ```
    
    is already not being supported. My judgement was `schema_of_json(column)` doesn't have to be exposed for now and actually want to have a talk with @MaxGekk about this when he comes back from his vacation.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97631 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97631/testReport)** for PR 22775 at commit [`9cb0b94`](https://github.com/apache/spark/commit/9cb0b946f95f881ba9203f785cc6446f93244157).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98001 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98001/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97649 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97649/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #97640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97640/testReport)** for PR 22775 at commit [`c132407`](https://github.com/apache/spark/commit/c1324073ee339d551a5716ec92169abb6b54c7c8).


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4125/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/4158/
    Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98066 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98066/testReport)** for PR 22775 at commit [`e2ca651`](https://github.com/apache/spark/commit/e2ca6517098adc093f957a6158ed760fb0826f4d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    @cloud-fan, looks we are going to start another RC. Would you mind if I ask to take a quick look before the new RC?


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98008 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98008/testReport)** for PR 22775 at commit [`dbf9cab`](https://github.com/apache/spark/commit/dbf9cabfb80c157cf0e52014b07c4abeb1aa2ff6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

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


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    Sure!


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    seems like a real test failure 


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/22775
  
    retest this please


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    **[Test build #98078 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98078/testReport)** for PR 22775 at commit [`8e6b97a`](https://github.com/apache/spark/commit/8e6b97a35f0d7c384c390f0a463e47f9160f0e32).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22775: [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's inpu...

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

    https://github.com/apache/spark/pull/22775
  
    Merged build finished. Test PASSed.


---

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