Github user tejasapatil commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18843#discussion_r132546198
  
    --- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -844,24 +844,39 @@ object SQLConf {
           .stringConf
           .createWithDefaultFunction(() => TimeZone.getDefault.getID)
     
    +  val WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD =
    +    buildConf("spark.sql.windowExec.buffer.in.memory.threshold")
    +      .internal()
    +      .doc("Threshold for number of rows guaranteed to be held in memory 
by the window operator")
    +      .intConf
    +      .createWithDefault(4096)
    +
       val WINDOW_EXEC_BUFFER_SPILL_THRESHOLD =
         buildConf("spark.sql.windowExec.buffer.spill.threshold")
           .internal()
    -      .doc("Threshold for number of rows buffered in window operator")
    +      .doc("Threshold for number of rows to be spilled by window operator")
           .intConf
    -      .createWithDefault(4096)
    +      
.createWithDefault(UnsafeExternalSorter.DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD.toInt)
    +
    +  val SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD =
    --- End diff --
    
    I am fine with that. We can even go a step further and just have two 
configs : in-mem threshold and spill threshold at the 
`ExternalAppendOnlyUnsafeRowArray` for all its clients (currently SMJ, 
cartesian product, Window). That way we have consistency across all clients and 
both knobs. One downside is backward compatibility : spill threshold was 
already defined per operator level and people might be using it in prod.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

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

Reply via email to