You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by petermaxlee <gi...@git.apache.org> on 2016/06/29 04:13:41 UTC

[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

GitHub user petermaxlee opened a pull request:

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

    [SPARK-16274][SQL] Implement xpath_boolean

    ## What changes were proposed in this pull request?
    This patch implements xpath_boolean expression for Spark SQL, a xpath function that returns true or false. The implementation is modelled after Hive's xpath_boolean, except that how the expression handles null inputs. Hive throws a NullPointerException at runtime if either of the input is null. This implementation returns null if either of the input is null.
    
    ## How was this patch tested?
    Added unit tests for expressions (based on Hive's tests and some I added myself) and an end-to-end test.


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

    $ git pull https://github.com/petermaxlee/spark SPARK-16274

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

    https://github.com/apache/spark/pull/13964.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 #13964
    
----
commit 34cda070f62677b6920174cc976107456172aeab
Author: petermaxlee <pe...@gmail.com>
Date:   2016-06-29T04:09:57Z

    [SPARK-16274][SQL] Implement xpath_boolean

----


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68883895
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    I thought this should be OK since the literal is non foldable and this class is only in the testing package.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68887178
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    +  @transient private var lastPathUtf8: UTF8String = null
    --- End diff --
    
    Great idea. Done!



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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68884017
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    --- End diff --
    
    shall we also optimize when the xml string is literal but the path string is not?


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    LGTM, pending tests, thanks for working on it!


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    looks pretty good, one last comment: https://github.com/apache/spark/pull/13964/files#r68929933


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68887731
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -301,6 +302,7 @@ object FunctionRegistry {
         expression[UnBase64]("unbase64"),
         expression[Unhex]("unhex"),
         expression[Upper]("upper"),
    +    expression[XPathBoolean]("xpath_boolean"),
    --- End diff --
    
    should we also register this function in `org.apache.spark.sql.functions`?


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68883797
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    Ur, it seems to be for a testing purpose. Is it okay?


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884371
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +/**
    + * End-to-end tests for XML expressions.
    + */
    +class XmlFunctionsSuite extends QueryTest with SharedSQLContext {
    +
    +  test("xpath_boolean") {
    +    val input = "<a><b>b</b></a>"
    +    val path = "a/b"
    --- End diff --
    
    will do.



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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884185
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal, NonFoldableLiteral}
    +import org.apache.spark.sql.types.StringType
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +/**
    + * Test suite for various xpath functions.
    + */
    +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
    --- End diff --
    
    I wrote this one based on what's already in the code base for other expressions. Let me know if I should do anything else.



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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3148 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3148/consoleFull)** for PR 13964 at commit [`3e4c176`](https://github.com/apache/spark/commit/3e4c176eb928517edd25a51fff8687e788d108b2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68886279
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    +  @transient private var lastPathUtf8: UTF8String = null
    --- End diff --
    
    how about
    ```
    @transient lazy val pathLiteral: String = path match {
      case Literal(str: String) => str
      case _ => null
    }
    ```


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68887674
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal, NonFoldableLiteral}
    +import org.apache.spark.sql.types.StringType
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +/**
    + * Test suite for various xpath functions.
    + */
    +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
    +
    +  private def testBoolean[T](xml: String, path: String, expected: T): Unit = {
    +    checkEvaluation(
    +      XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)),
    +      expected)
    +  }
    +
    +  test("xpath_boolean") {
    +    testBoolean("<a><b>b</b></a>", "a/b", true)
    +    testBoolean("<a><b>b</b></a>", "a/c", false)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"b\"", true)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"c\"", false)
    +    testBoolean("<a><b>10</b></a>", "a/b < 10", false)
    +    testBoolean("<a><b>10</b></a>", "a/b = 10", true)
    +
    +    // null input
    +    testBoolean(null, null, null)
    +    testBoolean(null, "a", null)
    +    testBoolean("<a><b>10</b></a>", null, null)
    +
    +    // exception handling for invalid input
    +    intercept[Exception] {
    +      testBoolean("<a>/a>", "a", null)
    +    }
    +  }
    +
    +  test("xpath_boolean path cache invalidation") {
    --- End diff --
    
    do we still need to test it? there is no cache anymore


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3138 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3138/consoleFull)** for PR 13964 at commit [`34cda07`](https://github.com/apache/spark/commit/34cda070f62677b6920174cc976107456172aeab).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class XPathBoolean(xml: Expression, path: Expression)`


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884363
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    What do you mean?


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68884274
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +/**
    + * End-to-end tests for XML expressions.
    + */
    +class XmlFunctionsSuite extends QueryTest with SharedSQLContext {
    +
    +  test("xpath_boolean") {
    +    val input = "<a><b>b</b></a>"
    +    val path = "a/b"
    --- End diff --
    
    how about
    ```
    val df = Seq("<a><b>b</b></a>" -> "a/b").toDF("xml", "path")
    checkAnswer(df.select(expr("xpath_boolean(xml, path)")), Row(true))
    ```


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68888030
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -301,6 +302,7 @@ object FunctionRegistry {
         expression[UnBase64]("unbase64"),
         expression[Unhex]("unhex"),
         expression[Upper]("upper"),
    +    expression[XPathBoolean]("xpath_boolean"),
    --- End diff --
    
    hm let's not register the xml ones there yet.



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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68886317
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    +  @transient private var lastPathUtf8: UTF8String = null
    --- End diff --
    
    then it's more obvious that we are tring to optimize when the path is literal.


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3140 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3140/consoleFull)** for PR 13964 at commit [`bdd49aa`](https://github.com/apache/spark/commit/bdd49aad79c6109046195f1f2713283a947d61f3).


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3141 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3141/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3144 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3144/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68884303
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +/**
    + * End-to-end tests for XML expressions.
    + */
    +class XmlFunctionsSuite extends QueryTest with SharedSQLContext {
    +
    +  test("xpath_boolean") {
    +    val input = "<a><b>b</b></a>"
    +    val path = "a/b"
    --- End diff --
    
    for end-to-end test, I think it's better to use attribute as input, not literal.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884058
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    Oh, I agree.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884892
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -301,6 +302,7 @@ object FunctionRegistry {
         expression[UnBase64]("unbase64"),
         expression[Unhex]("unhex"),
         expression[Upper]("upper"),
    +    expression[XPathBoolean]("xpath_boolean"),
    --- End diff --
    
    What about excluding at HiveSessionCatalog, too?
    
    https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala#L230


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    cc @srowen @cloud-fan @vanzin @squito 
    
    If this one works, I can implement the other ones too.



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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3148 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3148/consoleFull)** for PR 13964 at commit [`3e4c176`](https://github.com/apache/spark/commit/3e4c176eb928517edd25a51fff8687e788d108b2).


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68886429
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    Sorry I read the code wrong. You are testing cache invalidation and this must be mutable.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68929933
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    --- End diff --
    
    nit: we should say it more explicitly:
    `If the path is a constant, cache the path string so that we don't need to convert path from UTF8String to String for every row`


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3141 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3141/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression `


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3143 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3143/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

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


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68888009
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal, NonFoldableLiteral}
    +import org.apache.spark.sql.types.StringType
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +/**
    + * Test suite for various xpath functions.
    + */
    +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
    +
    +  private def testBoolean[T](xml: String, path: String, expected: T): Unit = {
    +    checkEvaluation(
    +      XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)),
    +      expected)
    +  }
    +
    +  test("xpath_boolean") {
    +    testBoolean("<a><b>b</b></a>", "a/b", true)
    +    testBoolean("<a><b>b</b></a>", "a/c", false)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"b\"", true)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"c\"", false)
    +    testBoolean("<a><b>10</b></a>", "a/b < 10", false)
    +    testBoolean("<a><b>10</b></a>", "a/b = 10", true)
    +
    +    // null input
    +    testBoolean(null, null, null)
    +    testBoolean(null, "a", null)
    +    testBoolean("<a><b>10</b></a>", null, null)
    +
    +    // exception handling for invalid input
    +    intercept[Exception] {
    +      testBoolean("<a>/a>", "a", null)
    +    }
    +  }
    +
    +  test("xpath_boolean path cache invalidation") {
    --- End diff --
    
    The underlying implementation can still exploit that (e.g. Hive's implementation does it), so I'm thinking it might be useful.


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3140 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3140/consoleFull)** for PR 13964 at commit [`bdd49aa`](https://github.com/apache/spark/commit/bdd49aad79c6109046195f1f2713283a947d61f3).
     * This patch **fails SparkR unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3138 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3138/consoleFull)** for PR 13964 at commit [`34cda07`](https://github.com/apache/spark/commit/34cda070f62677b6920174cc976107456172aeab).


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68889173
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal, NonFoldableLiteral}
    +import org.apache.spark.sql.types.StringType
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +/**
    + * Test suite for various xpath functions.
    + */
    +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
    +
    +  private def testBoolean[T](xml: String, path: String, expected: T): Unit = {
    +    checkEvaluation(
    +      XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)),
    +      expected)
    +  }
    +
    +  test("xpath_boolean") {
    +    testBoolean("<a><b>b</b></a>", "a/b", true)
    +    testBoolean("<a><b>b</b></a>", "a/c", false)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"b\"", true)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"c\"", false)
    +    testBoolean("<a><b>10</b></a>", "a/b < 10", false)
    +    testBoolean("<a><b>10</b></a>", "a/b = 10", true)
    +
    +    // null input
    +    testBoolean(null, null, null)
    +    testBoolean(null, "a", null)
    +    testBoolean("<a><b>10</b></a>", null, null)
    +
    +    // exception handling for invalid input
    +    intercept[Exception] {
    +      testBoolean("<a>/a>", "a", null)
    +    }
    +  }
    +
    +  test("xpath_boolean path cache invalidation") {
    +    // This is a test to ensure the expression is not reusing the path for different strings
    +    val xml = NonFoldableLiteral("<a><b>b</b></a>")
    +    val path = NonFoldableLiteral("a/b")
    +    val expr = XPathBoolean(xml, path)
    +
    +    // Run evaluation once
    +    assert(expr.eval(null) == true)
    +
    +    // Change the input path and make sure we don't screw up caching
    +    path.value = UTF8String.fromString("a/c")
    +    assert(expr.eval(null) == false)
    --- End diff --
    
    updated!



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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    merging to master, thanks!


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68888729
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathExpressionSuite.scala ---
    @@ -0,0 +1,68 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.SparkFunSuite
    +import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, Literal, NonFoldableLiteral}
    +import org.apache.spark.sql.types.StringType
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +/**
    + * Test suite for various xpath functions.
    + */
    +class XPathExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
    +
    +  private def testBoolean[T](xml: String, path: String, expected: T): Unit = {
    +    checkEvaluation(
    +      XPathBoolean(Literal.create(xml, StringType), Literal.create(path, StringType)),
    +      expected)
    +  }
    +
    +  test("xpath_boolean") {
    +    testBoolean("<a><b>b</b></a>", "a/b", true)
    +    testBoolean("<a><b>b</b></a>", "a/c", false)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"b\"", true)
    +    testBoolean("<a><b>b</b></a>", "a/b = \"c\"", false)
    +    testBoolean("<a><b>10</b></a>", "a/b < 10", false)
    +    testBoolean("<a><b>10</b></a>", "a/b = 10", true)
    +
    +    // null input
    +    testBoolean(null, null, null)
    +    testBoolean(null, "a", null)
    +    testBoolean("<a><b>10</b></a>", null, null)
    +
    +    // exception handling for invalid input
    +    intercept[Exception] {
    +      testBoolean("<a>/a>", "a", null)
    +    }
    +  }
    +
    +  test("xpath_boolean path cache invalidation") {
    +    // This is a test to ensure the expression is not reusing the path for different strings
    +    val xml = NonFoldableLiteral("<a><b>b</b></a>")
    +    val path = NonFoldableLiteral("a/b")
    +    val expr = XPathBoolean(xml, path)
    +
    +    // Run evaluation once
    +    assert(expr.eval(null) == true)
    +
    +    // Change the input path and make sure we don't screw up caching
    +    path.value = UTF8String.fromString("a/c")
    +    assert(expr.eval(null) == false)
    --- End diff --
    
    To test the changing of input, I think it's more clear to use `BoundReference`:
    ```
    val expr = XPathBoolean(Literal("<a><b>b</b></a>"), 'path.string.at(1))
    checkEvaluation(expr, true, create_row("a/b"))
    checkEvaluation(expr, false, create_row("a/c"))
    ```


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68884143
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NonFoldableLiteral.scala ---
    @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._
      * A literal value that is not foldable. Used in expression codegen testing to test code path
      * that behave differently based on foldable values.
      */
    -case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression {
    +case class NonFoldableLiteral(var value: Any, dataType: DataType) extends LeafExpression {
    --- End diff --
    
    It's ok if we do save a lot of effort because of it, but seems we don't?


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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/13964#discussion_r68883944
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    --- End diff --
    
    shall we also optimize for literal xml string?


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3143 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3143/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression `


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884156
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/xml/XPathBoolean.scala ---
    @@ -0,0 +1,56 @@
    +/*
    + * 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.xml
    +
    +import org.apache.spark.sql.catalyst.expressions._
    +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
    +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, StringType}
    +import org.apache.spark.unsafe.types.UTF8String
    +
    +
    +@ExpressionDescription(
    +  usage = "_FUNC_(xml, xpath) - Evaluates a boolean xpath expression.",
    +  extended = "> SELECT _FUNC_('<a><b>1</b></a>','a/b');\ntrue")
    +case class XPathBoolean(xml: Expression, path: Expression)
    +  extends BinaryExpression with ExpectsInputTypes with CodegenFallback {
    +
    +  @transient private lazy val xpathUtil = new UDFXPathUtil
    +
    +  // We use these to avoid converting the path from UTF8String to String if it is a constant.
    --- End diff --
    
    I think a literal path is a very common case, but a literal xml is fairly unlikely.


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68884564
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/XmlFunctionsSuite.scala ---
    @@ -0,0 +1,33 @@
    +/*
    + * 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
    +
    +import org.apache.spark.sql.test.SharedSQLContext
    +
    +/**
    + * End-to-end tests for XML expressions.
    + */
    +class XmlFunctionsSuite extends QueryTest with SharedSQLContext {
    +
    +  test("xpath_boolean") {
    +    val input = "<a><b>b</b></a>"
    +    val path = "a/b"
    --- End diff --
    
    I've updated this. PTAL.



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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

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


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    cc @srowen @cloud-fan @vanzin @squito 


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

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


[GitHub] spark issue #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964
  
    **[Test build #3144 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3144/consoleFull)** for PR 13964 at commit [`8d2db6e`](https://github.com/apache/spark/commit/8d2db6eef914c82d3b398e4af5a8b4b6b49a4f28).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class NonFoldableLiteral(value: Any, dataType: DataType) extends LeafExpression `


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

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


[GitHub] spark pull request #13964: [SPARK-16274][SQL] Implement xpath_boolean

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

    https://github.com/apache/spark/pull/13964#discussion_r68887194
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala ---
    @@ -301,6 +302,7 @@ object FunctionRegistry {
         expression[UnBase64]("unbase64"),
         expression[Unhex]("unhex"),
         expression[Upper]("upper"),
    +    expression[XPathBoolean]("xpath_boolean"),
    --- End diff --
    
    done.



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

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