You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by DaveDeCaprio <gi...@git.apache.org> on 2018/11/28 18:04:27 UTC

[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

GitHub user DaveDeCaprio opened a pull request:

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

    [SPARK-26103][SQL] Limit the length of debug strings for query plans

    ## What changes were proposed in this pull request?
    
    The PR puts in a limit on the size of a debug string generated for a tree node.  Helps to fix out of memory errors when large plans have huge debug strings.   In addition to SPARK-26103, this should also address SPARK-23904 and SPARK-25380.  AN alternative solution was proposed in #23076, but that solution doesn't address all the cases that can cause a large query.  This limit is only on calls treeString that don't pass a Writer, which makes it play nicely with #22429, #23018 and #23039.  Full plans can be written to files, but truncated plans will be used when strings are held in memory, such as for the UI.
    
    - A new configuration parameter called spark.sql.debug.maxPlanLength was added to control the length of the plans.
    - When plans are truncated, "..." is printed to indicate that it isn't a full plan
    - A warning is printed out the first time a truncated plan is displayed. The warning explains what happened and how to adjust the limit.
    
    ## How was this patch tested?
    
    Unit tests were created for the new SizeLimitedWriter.  Also a unit test for TreeNode was created that checks that a long plan is correctly truncated.

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

    $ git pull https://github.com/DaveDeCaprio/spark text-plan-size

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

    https://github.com/apache/spark/pull/23169.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 #23169
    
----
commit 22bd4bddcf4f80d521a27643840f4a3536dac0f3
Author: David DeCaprio <da...@...>
Date:   2018-11-27T19:18:11Z

    Merge pull request #1 from apache/master
    
    merge in spark

commit b7f964d119b5d0ea40896bb86b0110688d8330a8
Author: Dave DeCaprio <da...@...>
Date:   2018-11-28T17:46:14Z

    Added a configurable limit on the maximum length of a plan debug string.

----


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237305318
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -1610,6 +1610,12 @@ object SQLConf {
           """ "... N more fields" placeholder.""")
         .intConf
         .createWithDefault(25)
    +
    +  val MAX_PLAN_STRING_LENGTH = buildConf("spark.sql.debug.maxPlanLength")
    --- End diff --
    
    I'm not sure `debug` is right. You know this patch should help UI to reduce memory usage which is not a debug. If we specify `debug` here, end users would interpret as there's a debug mode. Same as description.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99414 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99414/testReport)** for PR 23169 at commit [`3171cf3`](https://github.com/apache/spark/commit/3171cf31ae8001e5c37b31668154bbf2b6b4411f).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237344026
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala ---
    @@ -595,4 +596,14 @@ class TreeNodeSuite extends SparkFunSuite {
         val expected = Coalesce(Stream(Literal(1), Literal(3)))
         assert(result === expected)
       }
    +
    +  test("toString() tree depth") {
    --- End diff --
    
    fixed


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    @DaveDeCaprio 
    
    You might miss to roll back change in test.
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99632/testReport/org.apache.spark.sql.catalyst.trees/TreeNodeSuite/treeString_limits_plan_length/
    
    I also think you need to add a new test with setting configuration to some value and see whether it works properly.


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237304214
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/SizeLimitedWriter.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * 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.util
    +
    +import java.io.Writer
    +
    +class WriterSizeException(val attemptedSize: Long, val charLimit: Long) extends Exception(
    +  s"Attempted to write $attemptedSize characters to a writer that is limited to $charLimit")
    +
    +/**
    + * This class is used to control the size of generated writers.  Guarantees that the total number
    + * of characters written will be less than the specified size.
    + *
    + * Checks size before writing and throws a WriterSizeException if the total size would count the
    + * limit.
    + */
    +class SizeLimitedWriter(underlying: Writer, charLimit: Long) extends Writer {
    +
    +  var charsWritten: Long = 0
    +
    +  override def write(cbuf: Array[Char], off: Int, len: Int): Unit = {
    --- End diff --
    
    I'd rather make this writing the content as much as possible (with `...`), or let WriterSizeException contains relevant information to help caller be able to call this again with smaller length. 
    
    In worst case, if first physical plan is huge we end up only showing `...`.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99686 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99686/testReport)** for PR 23169 at commit [`22fe117`](https://github.com/apache/spark/commit/22fe117656ea004757efaffd847f81dc01df8433).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    Yes, I'm trying to figure out how to test with a changed SQLConf.  Once i do that I'll fix.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99683 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99683/testReport)** for PR 23169 at commit [`1b692a0`](https://github.com/apache/spark/commit/1b692a0444a1c0f1fc24a08241f24dd35e4c428b).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class TreeNodeSuite extends SparkFunSuite with SQLHelper `


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99683 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99683/testReport)** for PR 23169 at commit [`1b692a0`](https://github.com/apache/spark/commit/1b692a0444a1c0f1fc24a08241f24dd35e4c428b).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237309191
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala ---
    @@ -202,6 +202,26 @@ package object util extends Logging {
       /** Shorthand for calling truncatedString() without start or end strings. */
       def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "")
     
    +  /** Whether we have warned about plan string truncation yet. */
    +  private val planSizeWarningPrinted = new AtomicBoolean(false)
    +
    +  def withSizeLimitedWriter[T](writer: Writer)(f: (Writer) => T): Option[T] = {
    +    try {
    +      val limited = new SizeLimitedWriter(writer, SQLConf.get.maxPlanStringLength)
    +      Some(f(limited))
    +    }
    +    catch {
    +      case e: WriterSizeException =>
    +        writer.write("...")
    --- End diff --
    
    nit: Would we want to restrict string's length to maxPlanStringLength including `...`? I think exceeding defined length by max 3 chars is not a big deal but just to double check.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99426/testReport)** for PR 23169 at commit [`45a60fc`](https://github.com/apache/spark/commit/45a60fc7f9f4a0c04eae5ae10be68c5aba3dc3e1).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    @HeartSaVioR I've updated the description in SQLConf.scala.  Is there some other documentation that should be updated?


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99427/testReport)** for PR 23169 at commit [`9678799`](https://github.com/apache/spark/commit/9678799f3b203d667c2f4b27dcedd3591606a8cf).
     * This patch **fails Scala style 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    If you have an idea of what those use cases are I could take a look and see if there is an impact.  If not, we could turn it off by default (set the max length to Long.Max).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99418 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99418/testReport)** for PR 23169 at commit [`3ffdc6a`](https://github.com/apache/spark/commit/3ffdc6a13370be2f7cd03bea0e48f8e5ef62ccca).
     * 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99878 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99878/testReport)** for PR 23169 at commit [`855f540`](https://github.com/apache/spark/commit/855f5404c53eba51ed373fa9e7be4eaafd60bb30).
     * 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99876 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99876/testReport)** for PR 23169 at commit [`f6d0efc`](https://github.com/apache/spark/commit/f6d0efc7c0e1d461e5854c6e04f3347f174bf13a).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class GBTClassifierParams(GBTParams, HasVarianceImpurity):`
      * `class GBTClassifier(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,`
      * `class HasDistanceMeasure(Params):`
      * `class HasValidationIndicatorCol(Params):`
      * `class HasVarianceImpurity(Params):`
      * `class TreeRegressorParams(HasVarianceImpurity):`
      * `class GBTParams(TreeEnsembleParams, HasMaxIter, HasStepSize, HasValidationIndicatorCol):`
      * `class GBTRegressorParams(GBTParams, TreeRegressorParams):`
      * `class GBTRegressor(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,`
      * `class ArrowCollectSerializer(Serializer):`
      * `class CSVInferSchema(val options: CSVOptions) extends Serializable `
      * `class InterpretedSafeProjection(expressions: Seq[Expression]) extends Projection `
      * `sealed trait DateTimeFormatter `
      * `class Iso8601DateTimeFormatter(`
      * `class LegacyDateTimeFormatter(`
      * `class LegacyFallbackDateTimeFormatter(`
      * `sealed trait DateFormatter `
      * `class Iso8601DateFormatter(`
      * `class LegacyDateFormatter(`
      * `class LegacyFallbackDateFormatter(`


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99878 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99878/testReport)** for PR 23169 at commit [`855f540`](https://github.com/apache/spark/commit/855f5404c53eba51ed373fa9e7be4eaafd60bb30).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237344154
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/SizeLimitedWriter.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * 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.util
    +
    +import java.io.Writer
    +
    +class WriterSizeException(val attemptedSize: Long, val charLimit: Long) extends Exception(
    +  s"Attempted to write $attemptedSize characters to a writer that is limited to $charLimit")
    +
    +/**
    + * This class is used to control the size of generated writers.  Guarantees that the total number
    + * of characters written will be less than the specified size.
    + *
    + * Checks size before writing and throws a WriterSizeException if the total size would count the
    + * limit.
    + */
    +class SizeLimitedWriter(underlying: Writer, charLimit: Long) extends Writer {
    +
    +  var charsWritten: Long = 0
    --- End diff --
    
    fixed


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99410 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99410/testReport)** for PR 23169 at commit [`5528ca1`](https://github.com/apache/spark/commit/5528ca12ed625ade3a4dc4f043da838b1005e826).
     * This patch **fails Scala style 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99430 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99430/testReport)** for PR 23169 at commit [`a5af842`](https://github.com/apache/spark/commit/a5af8426779d22ba620e188e4411630db6d2e52f).
     * 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    Build finished. Test FAILed.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99410 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99410/testReport)** for PR 23169 at commit [`5528ca1`](https://github.com/apache/spark/commit/5528ca12ed625ade3a4dc4f043da838b1005e826).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99686 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99686/testReport)** for PR 23169 at commit [`22fe117`](https://github.com/apache/spark/commit/22fe117656ea004757efaffd847f81dc01df8433).
     * 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99418 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99418/testReport)** for PR 23169 at commit [`3ffdc6a`](https://github.com/apache/spark/commit/3ffdc6a13370be2f7cd03bea0e48f8e5ef62ccca).


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237301700
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/SizeLimitedWriter.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * 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.util
    +
    +import java.io.Writer
    +
    +class WriterSizeException(val attemptedSize: Long, val charLimit: Long) extends Exception(
    +  s"Attempted to write $attemptedSize characters to a writer that is limited to $charLimit")
    +
    +/**
    + * This class is used to control the size of generated writers.  Guarantees that the total number
    + * of characters written will be less than the specified size.
    + *
    + * Checks size before writing and throws a WriterSizeException if the total size would count the
    + * limit.
    + */
    +class SizeLimitedWriter(underlying: Writer, charLimit: Long) extends Writer {
    +
    +  var charsWritten: Long = 0
    --- End diff --
    
    Looks like it should not be exposed outside of class. Let's guard it as `private`. 


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    Ok, I've updated this PR so that the default behavior does not change - full plan strings are always printed. 
     This should be fully backwards compatible.  Plan strings will only be truncated if you specifically configure them to be.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99426 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99426/testReport)** for PR 23169 at commit [`45a60fc`](https://github.com/apache/spark/commit/45a60fc7f9f4a0c04eae5ae10be68c5aba3dc3e1).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class WriterSizeException(val extraChars: Long, val charLimit: Long) extends Exception(`


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/99686/
    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 #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237346220
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -1610,6 +1610,12 @@ object SQLConf {
           """ "... N more fields" placeholder.""")
         .intConf
         .createWithDefault(25)
    +
    +  val MAX_PLAN_STRING_LENGTH = buildConf("spark.sql.debug.maxPlanLength")
    --- End diff --
    
    Removed debug from the config name and description


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237343993
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala ---
    @@ -202,6 +202,26 @@ package object util extends Logging {
       /** Shorthand for calling truncatedString() without start or end strings. */
       def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "")
     
    +  /** Whether we have warned about plan string truncation yet. */
    +  private val planSizeWarningPrinted = new AtomicBoolean(false)
    +
    +  def withSizeLimitedWriter[T](writer: Writer)(f: (Writer) => T): Option[T] = {
    +    try {
    +      val limited = new SizeLimitedWriter(writer, SQLConf.get.maxPlanStringLength)
    +      Some(f(limited))
    +    }
    +    catch {
    +      case e: WriterSizeException =>
    +        writer.write("...")
    --- End diff --
    
    It was easy enough to do this by dropping the limit by 3 characters.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99631 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99631/testReport)** for PR 23169 at commit [`a4be985`](https://github.com/apache/spark/commit/a4be985b5e7eeed43409ba0b0cda2a45c44e5110).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    **[Test build #99414 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99414/testReport)** for PR 23169 at commit [`3171cf3`](https://github.com/apache/spark/commit/3171cf31ae8001e5c37b31668154bbf2b6b4411f).
     * This patch **fails to build**.
     * 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 #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark pull request #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237344141
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/SizeLimitedWriter.scala ---
    @@ -0,0 +1,48 @@
    +/*
    + * 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.util
    +
    +import java.io.Writer
    +
    +class WriterSizeException(val attemptedSize: Long, val charLimit: Long) extends Exception(
    +  s"Attempted to write $attemptedSize characters to a writer that is limited to $charLimit")
    +
    +/**
    + * This class is used to control the size of generated writers.  Guarantees that the total number
    + * of characters written will be less than the specified size.
    + *
    + * Checks size before writing and throws a WriterSizeException if the total size would count the
    + * limit.
    + */
    +class SizeLimitedWriter(underlying: Writer, charLimit: Long) extends Writer {
    +
    +  var charsWritten: Long = 0
    +
    +  override def write(cbuf: Array[Char], off: Int, len: Int): Unit = {
    --- End diff --
    
    I changed to writing out to the full limit


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    ok to test


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    I added changes to QueryExecution in the latest commit to address the UI issue.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    Thanks for addressong review comments. It looks great overall.
    
    We may want to document the new config so that we can guide setting the value to lower when end users suffer from memory pressure due to long physical plans in UI pages.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    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 #23169: [SPARK-26103][SQL] Limit the length of debug stri...

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

    https://github.com/apache/spark/pull/23169#discussion_r237307829
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala ---
    @@ -595,4 +596,14 @@ class TreeNodeSuite extends SparkFunSuite {
         val expected = Coalesce(Stream(Literal(1), Literal(3)))
         assert(result === expected)
       }
    +
    +  test("toString() tree depth") {
    --- End diff --
    
    `treeString` sounds right to me rather than `toString`.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    @HeartSaVioR  I added tests for the default case and for a truncated plan.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

    https://github.com/apache/spark/pull/23169
  
    @MaxGekk and @hvanhovell, this is an alternative solution for #23076.  It limits overall plan length when generating the full string in memory, but not if a specific writer is passed in.


---

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


[GitHub] spark issue #23169: [SPARK-26103][SQL] Limit the length of debug strings for...

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

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


---

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