You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by MaxGekk <gi...@git.apache.org> on 2018/06/24 14:02:31 UTC

[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

GitHub user MaxGekk opened a pull request:

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

    [SPARK-24642][SQL] New function infers schema for JSON column

    ## What changes were proposed in this pull request?
    
    In the PR, I propose new aggregate function - *infer_schema()*. The function infers schema for an expression contains JSON strings. *infer_schema()* returns schema in DDL format.
    
    One of use cases is using of *infer_schema()* in combination with *from_json()* in SQL:
    
    ```sql
    select from_json(json_col, infer_schema(json_col))
    from json_table;
    ```
    
    ## How was this patch tested?
    
    I added tests to `json-functions.sql` to check schema inferring for array and struct types.

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

    $ git pull https://github.com/MaxGekk/spark-1 json_infer_schema

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

    https://github.com/apache/spark/pull/21626.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 #21626
    
----
commit f98aea2b59025f4a41e28fac8e2b2b689ddf4d27
Author: Maxim Gekk <ma...@...>
Date:   2018-06-23T17:16:22Z

    Initial implementation of the infer_schema function

commit a0c9a1137c5444890f048bd480d63496e31ec599
Author: Maxim Gekk <ma...@...>
Date:   2018-06-23T17:29:24Z

    Move typeMerger out of the merge function

commit 17a1f98448194af984de43d4dedad99271e25189
Author: Maxim Gekk <ma...@...>
Date:   2018-06-23T20:14:38Z

    SQL test for the infer_schema function

commit 45fc2e419dda2e53f5ff7e7ecbbee64d2bf23cf7
Author: Maxim Gekk <ma...@...>
Date:   2018-06-24T10:31:44Z

    Pretty name is changed to infer_schema

commit 4db679927e35cc41e2160b91d2435dc653f368a9
Author: Maxim Gekk <ma...@...>
Date:   2018-06-24T10:40:01Z

    Refactoring

commit 7e5ad618b6fba583db85dd1bdb251cc824c80bc8
Author: Maxim Gekk <ma...@...>
Date:   2018-06-24T11:32:32Z

    bug fix

commit 96e5cd33fbc4711302f9f0cf47e851df66fda524
Author: Maxim Gekk <ma...@...>
Date:   2018-06-24T12:05:12Z

    Added description for InferSchema

commit 333139da49951df1aee39aeabc286a162dd92ad9
Author: Maxim Gekk <ma...@...>
Date:   2018-06-24T12:05:46Z

    Drop views

----


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

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


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    Can one of the admins verify this patch?


---

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


[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

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

    https://github.com/apache/spark/pull/21626#discussion_r197662496
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/InferSchema.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.expressions.aggregate
    +
    +import scala.util.Try
    +
    +import com.fasterxml.jackson.core.JsonFactory
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExpressionDescription, JsonExprUtils}
    +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JsonInferSchema, JSONOptions}
    +import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleRootType
    +import org.apache.spark.sql.catalyst.util.DropMalformedMode
    +import org.apache.spark.sql.types.{DataType, StringType, StructType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +@ExpressionDescription(
    +  usage = """_FUNC_(expr,[options]) - Infers schema for JSON `expr` by using JSON `options`.""")
    +case class InferSchema(
    --- End diff --
    
    We probably shouldn't just call it `InferSchema` in case we add a similar function, for example, for CSV.


---

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


[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

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

    https://github.com/apache/spark/pull/21626#discussion_r198147668
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/InferSchema.scala ---
    @@ -0,0 +1,169 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.expressions.aggregate
    +
    +import scala.util.Try
    +
    +import com.fasterxml.jackson.core.JsonFactory
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExpressionDescription, JsonExprUtils}
    +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JsonInferSchema, JSONOptions}
    +import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleRootType
    +import org.apache.spark.sql.catalyst.util.DropMalformedMode
    +import org.apache.spark.sql.types.{DataType, StringType, StructType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +@ExpressionDescription(
    +  usage = """_FUNC_(expr, [options]) - Infers schema for JSON `expr` by using JSON `options`.""",
    +  examples = """
    +    Examples:
    +      > CREATE TEMPORARY VIEW json_table(json) AS SELECT * FROM VALUES ('{"a":1}'), ('{"a": 3}');
    +      > SELECT _FUNC_(json) FROM json_table;
    --- End diff --
    
    There's inline table expression: `SELECT _FUNC_(json) FROM VALUES ('{"a":1}'), ('{"a": 3}') json_table(json)` FYI.


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    Oops, I am sorry. I didn't know it's happening in the JIRA.


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    BTW I think you can do all the discussion on the public lists.


---

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


[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

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

    https://github.com/apache/spark/pull/21626#discussion_r198152922
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala ---
    @@ -25,12 +25,36 @@ import org.apache.spark.SparkException
     import org.apache.spark.rdd.RDD
     import org.apache.spark.sql.catalyst.analysis.TypeCoercion
     import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil
    -import org.apache.spark.sql.catalyst.json.JSONOptions
     import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, ParseMode, PermissiveMode}
     import org.apache.spark.sql.types._
     import org.apache.spark.util.Utils
     
    -private[sql] object JsonInferSchema {
    +object JsonInferSchema {
    +  def inferForRow[T](
    --- End diff --
    
    maybe inferFromRow?


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

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


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    @rxin thinks it is complicated solution. I am closing the PR. 


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    **[Test build #92338 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92338/testReport)** for PR 21626 at commit [`0e3d1de`](https://github.com/apache/spark/commit/0e3d1de37bfc73cf1cc5aa33b2f5d80aa154a06b).
     * 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 #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    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 #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    FWIW, I believe schema inference feature should be added (see http://apache-spark-developers-list.1001551.n3.nabble.com/Spark-SQL-JSON-Column-Support-td19132.html#a19154 too when from_json was first added) and I imagined that's relatively complicated. This way was one way I was thinking, actually. If there's an easier way, that would be great.


---

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


[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

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

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


---

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


[GitHub] spark pull request #21626: [SPARK-24642][SQL] New function infers schema for...

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

    https://github.com/apache/spark/pull/21626#discussion_r197662799
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/InferSchema.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.expressions.aggregate
    +
    +import scala.util.Try
    +
    +import com.fasterxml.jackson.core.JsonFactory
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExpressionDescription, JsonExprUtils}
    +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JsonInferSchema, JSONOptions}
    +import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleRootType
    +import org.apache.spark.sql.catalyst.util.DropMalformedMode
    +import org.apache.spark.sql.types.{DataType, StringType, StructType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +@ExpressionDescription(
    +  usage = """_FUNC_(expr,[options]) - Infers schema for JSON `expr` by using JSON `options`.""")
    --- End diff --
    
    nit: `,[` -> `, [`.
    
    shall we add `examples` and `since` too?


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    **[Test build #92267 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92267/testReport)** for PR 21626 at commit [`333139d`](https://github.com/apache/spark/commit/333139da49951df1aee39aeabc286a162dd92ad9).


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    **[Test build #92267 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92267/testReport)** for PR 21626 at commit [`333139d`](https://github.com/apache/spark/commit/333139da49951df1aee39aeabc286a162dd92ad9).
     * 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 #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    **[Test build #92338 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92338/testReport)** for PR 21626 at commit [`0e3d1de`](https://github.com/apache/spark/commit/0e3d1de37bfc73cf1cc5aa33b2f5d80aa154a06b).


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    Do you prefer add Python / Scala one here or separately?


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    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 #21626: [SPARK-24642][SQL] New function infers schema for...

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

    https://github.com/apache/spark/pull/21626#discussion_r198151731
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/InferSchema.scala ---
    @@ -0,0 +1,169 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.catalyst.expressions.aggregate
    +
    +import scala.util.Try
    +
    +import com.fasterxml.jackson.core.JsonFactory
    +
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, ExpressionDescription, JsonExprUtils}
    +import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JsonInferSchema, JSONOptions}
    +import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleRootType
    +import org.apache.spark.sql.catalyst.util.DropMalformedMode
    +import org.apache.spark.sql.types.{DataType, StringType, StructType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +@ExpressionDescription(
    +  usage = """_FUNC_(expr, [options]) - Infers schema for JSON `expr` by using JSON `options`.""",
    +  examples = """
    +    Examples:
    +      > CREATE TEMPORARY VIEW json_table(json) AS SELECT * FROM VALUES ('{"a":1}'), ('{"a": 3}');
    +      > SELECT _FUNC_(json) FROM json_table;
    +       struct<a:bigint>
    +  """,
    +  since = "2.4.0")
    +case class InferSchema(
    +  child: Expression,
    +  inputFormat: String,
    +  options: Map[String, String],
    +  override val mutableAggBufferOffset: Int,
    +  override val inputAggBufferOffset: Int) extends ImperativeAggregate {
    +
    +  require(inputFormat.toLowerCase == "json", "Only JSON format is supported")
    +
    +  def this(child: Expression) = {
    +    this(
    +      child = child,
    +      inputFormat = "json",
    +      options = Map.empty[String, String],
    +      mutableAggBufferOffset = 0,
    +      inputAggBufferOffset = 0)
    +  }
    +
    +  def this(child: Expression, options: Expression) = {
    --- End diff --
    
    wait do we take options too?


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    Seems fine to me.


---

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


[GitHub] spark issue #21626: [SPARK-24642][SQL] New function infers schema for JSON c...

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

    https://github.com/apache/spark/pull/21626
  
    It is on the public list: https://issues.apache.org/jira/browse/SPARK-24642


---

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