You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/02/29 05:52:43 UTC

[GitHub] [spark] beliefer opened a new pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

beliefer opened a new pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744
 
 
   ### What changes were proposed in this pull request?
   I found a lot scattered config in `Streaming`.I think should arrange these config in unified position.
   
   
   ### Why are the changes needed?
   Arrange scattered config
   
   
   ### Does this PR introduce any user-facing change?
   No
   
   
   ### How was this patch tested?
   Exists UT
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386220515
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   OK, then could we remove `DELTA_CHAIN_LENGTH_THRESHOLD`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302873
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593348601
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119164/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595512215
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302882
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23906/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595266287
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595518790
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594519602
 
 
   I've just checked the documentation and `spark.streaming.receiver.maxRate` says in the doc that default number is `not set` but in the code it's ` Long.MaxValue.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592931452
 
 
   **[Test build #119120 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119120/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592931541
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119120/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595518796
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119431/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593348601
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119164/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595274274
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24135/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595518768
 
 
   **[Test build #119431 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119431/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).
    * This patch **fails to generate documentation**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386204408
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala
 ##########
 @@ -0,0 +1,161 @@
+/*
+ * 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.streaming
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+object StreamingConf {
+
+  private[streaming] val BACKPRESSURE_ENABLED =
+    ConfigBuilder("spark.streaming.backpressure.enabled")
+      .booleanConf
+      .createWithDefault(false)
+
+  private[streaming] val RECEIVER_MAX_RATE =
+    ConfigBuilder("spark.streaming.receiver.maxRate")
+      .longConf
+      .createWithDefault(Long.MaxValue)
+
+  private[streaming] val BACKPRESSURE_INITIAL_RATE =
+    ConfigBuilder("spark.streaming.backpressure.initialRate")
+      .longConf
+      .createOptional
+
+  val BLOCK_INTERVAL =
 
 Review comment:
   Is there specific reason this is exposed as public?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592931541
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119120/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595513551
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594606056
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119313/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896204
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302320
 
 
   **[Test build #119164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119164/testReport)** for PR 27744 at commit [`37b2e93`](https://github.com/apache/spark/commit/37b2e93abf3e7f377d36973b1f893eafad4367b1).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594982235
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595588111
 
 
   **[Test build #119441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119441/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595721553
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119459/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595721553
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119459/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386219669
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   Only here `DELTA_CHAIN_LENGTH_THRESHOLD`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592917834
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925270
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386219669
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   I'm sorry, I read it wrong.`DELTA_CHAIN_LENGTH_THRESHOLD` used in two place.
   https://github.com/apache/spark/blob/5c29613428bffad5b0aae7cbcec85123a1ef4819/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala#L87
   https://github.com/apache/spark/blob/5c29613428bffad5b0aae7cbcec85123a1ef4819/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala#L102

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387653446
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala
 ##########
 @@ -0,0 +1,161 @@
+/*
+ * 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.streaming
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+object StreamingConf {
+
+  private[streaming] val BACKPRESSURE_ENABLED =
+    ConfigBuilder("spark.streaming.backpressure.enabled")
+      .booleanConf
+      .createWithDefault(false)
+
+  private[streaming] val RECEIVER_MAX_RATE =
+    ConfigBuilder("spark.streaming.receiver.maxRate")
+      .longConf
+      .createWithDefault(Long.MaxValue)
+
+  private[streaming] val BACKPRESSURE_INITIAL_RATE =
+    ConfigBuilder("spark.streaming.backpressure.initialRate")
+      .longConf
+      .createOptional
 
 Review comment:
   maybe `fallbackConf(RECEIVER_MAX_RATE)`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595512969
 
 
   **[Test build #119431 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119431/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595599911
 
 
   **[Test build #119441 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119441/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).
    * This patch **fails PySpark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925868
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-596976757
 
 
   Merged to master and branch-3.0. Thanks, @beliefer, @HeartSaVioR, @gaborgsomogyi.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594982239
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119345/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896111
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387732530
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   I'm not sure about this, I hope someone will be more clear about this.
   Maybe we need move `DELTA_CHAIN_LENGTH_THRESHOLD`  form `StateMap.scala` to `StreamingConf.scala`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595600000
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595586845
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595274269
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302320
 
 
   **[Test build #119164 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119164/testReport)** for PR 27744 at commit [`37b2e93`](https://github.com/apache/spark/commit/37b2e93abf3e7f377d36973b1f893eafad4367b1).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387653984
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
 ##########
 @@ -100,7 +101,7 @@ private[streaming] class BlockGenerator(
   }
   import GeneratorState._
 
-  private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms")
+  private val blockIntervalMs = conf.get(BLOCK_INTERVAL)
   require(blockIntervalMs > 0, s"'spark.streaming.blockInterval' should be a positive value")
 
 Review comment:
   `BLOCK_INTERVAL.key`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925728
 
 
   **[Test build #119120 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119120/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913523
 
 
   **[Test build #119118 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119118/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594156
 
 
   **[Test build #119313 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119313/testReport)** for PR 27744 at commit [`dd3bb5b`](https://github.com/apache/spark/commit/dd3bb5b1b92c0176c8ae9a0bbfb20a6718beea5f).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594987
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24052/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302882
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23906/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi edited a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi edited a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594519602
 
 
   I've just checked the documentation and `spark.streaming.receiver.maxRate` says in the doc that default number is `not set` but in the code it's `Long.MaxValue`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595274269
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r388251882
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   I've had a deeper look and `DELTA_CHAIN_LENGTH_THRESHOLD` is `OpenHashMapBasedStateMap` specific default value. I don't think we have to move it but I think the newly extracted configuration must use the constant.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896118
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23859/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593035750
 
 
   cc @HeartSaVioR and @zsxwing 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896194
 
 
   **[Test build #119117 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119117/testReport)** for PR 27744 at commit [`e982f5c`](https://github.com/apache/spark/commit/e982f5c308dfd48715d6e72a215d8ddef9ac35f3).
    * This patch **fails Scala style tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595266298
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119397/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386264614
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595684819
 
 
   **[Test build #119459 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119459/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594987
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24052/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896210
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119117/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595287950
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119398/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386218251
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala
 ##########
 @@ -0,0 +1,161 @@
+/*
+ * 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.streaming
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+object StreamingConf {
+
+  private[streaming] val BACKPRESSURE_ENABLED =
+    ConfigBuilder("spark.streaming.backpressure.enabled")
+      .booleanConf
+      .createWithDefault(false)
+
+  private[streaming] val RECEIVER_MAX_RATE =
+    ConfigBuilder("spark.streaming.receiver.maxRate")
+      .longConf
+      .createWithDefault(Long.MaxValue)
+
+  private[streaming] val BACKPRESSURE_INITIAL_RATE =
+    ConfigBuilder("spark.streaming.backpressure.initialRate")
+      .longConf
+      .createOptional
+
+  val BLOCK_INTERVAL =
 
 Review comment:
   Sorry, I forgot it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595261212
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24134/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595518796
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119431/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594508783
 
 
   @gaborgsomogyi Thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595681378
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593348590
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913523
 
 
   **[Test build #119118 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119118/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595600000
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592895678
 
 
   **[Test build #119117 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119117/testReport)** for PR 27744 at commit [`e982f5c`](https://github.com/apache/spark/commit/e982f5c308dfd48715d6e72a215d8ddef9ac35f3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594156
 
 
   **[Test build #119313 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119313/testReport)** for PR 27744 at commit [`dd3bb5b`](https://github.com/apache/spark/commit/dd3bb5b1b92c0176c8ae9a0bbfb20a6718beea5f).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594961480
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386264614
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595588111
 
 
   **[Test build #119441 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119441/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592917834
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925868
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387732530
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   I'm not sure about this, I hope someone will be more clear about this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595721546
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896118
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23859/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592931535
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913643
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23860/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595586845
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595512969
 
 
   **[Test build #119431 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119431/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595287931
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595266287
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925870
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23862/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595260623
 
 
   **[Test build #119397 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119397/testReport)** for PR 27744 at commit [`29e2a6c`](https://github.com/apache/spark/commit/29e2a6c801aa5adf75c6e660378ca04bfe4532c9).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913643
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23860/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594606041
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387731113
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala
 ##########
 @@ -100,7 +101,7 @@ private[streaming] class BlockGenerator(
   }
   import GeneratorState._
 
-  private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms")
+  private val blockIntervalMs = conf.get(BLOCK_INTERVAL)
   require(blockIntervalMs > 0, s"'spark.streaming.blockInterval' should be a positive value")
 
 Review comment:
   OK

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593302873
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592931535
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594502460
 
 
   As I see you've created the Jira with Structured Streaming component and then trying to make DStreams modifications. I suggest to change the title to `[SPARK-30992][DSTREAMS] Arrange scattered config of streaming module` in order to avoid misunderstandings like this. Additionally changed the jira component.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595600005
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119441/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595586679
 
 
   retest this please

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595586848
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24177/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913639
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-597015124
 
 
   @HyukjinKwon @HeartSaVioR @gaborgsomogyi Thanks for all your review.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595266260
 
 
   **[Test build #119397 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119397/testReport)** for PR 27744 at commit [`29e2a6c`](https://github.com/apache/spark/commit/29e2a6c801aa5adf75c6e660378ca04bfe4532c9).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r388328477
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   OK.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594961492
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24082/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595266298
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119397/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387729483
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/StreamingConf.scala
 ##########
 @@ -0,0 +1,161 @@
+/*
+ * 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.streaming
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+object StreamingConf {
+
+  private[streaming] val BACKPRESSURE_ENABLED =
+    ConfigBuilder("spark.streaming.backpressure.enabled")
+      .booleanConf
+      .createWithDefault(false)
+
+  private[streaming] val RECEIVER_MAX_RATE =
+    ConfigBuilder("spark.streaming.receiver.maxRate")
+      .longConf
+      .createWithDefault(Long.MaxValue)
+
+  private[streaming] val BACKPRESSURE_INITIAL_RATE =
+    ConfigBuilder("spark.streaming.backpressure.initialRate")
+      .longConf
+      .createOptional
 
 Review comment:
   OK. Thanks!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595274274
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24135/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on a change in pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r387657948
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   Do the newly defined config's default value and the mentioned other places have to keep the same value? I mean if one change the one should the others be changed? (now the value defined in 2 distinct places)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594961
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594605983
 
 
   **[Test build #119313 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119313/testReport)** for PR 27744 at commit [`dd3bb5b`](https://github.com/apache/spark/commit/dd3bb5b1b92c0176c8ae9a0bbfb20a6718beea5f).
    * This patch **fails to build**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594960900
 
 
   **[Test build #119345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119345/testReport)** for PR 27744 at commit [`005211f`](https://github.com/apache/spark/commit/005211f7e104f7935d433a989144641a7f51bb4e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592917835
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119118/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595261203
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon closed pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HyukjinKwon closed pull request #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592917824
 
 
   **[Test build #119118 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119118/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595260623
 
 
   **[Test build #119397 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119397/testReport)** for PR 27744 at commit [`29e2a6c`](https://github.com/apache/spark/commit/29e2a6c801aa5adf75c6e660378ca04bfe4532c9).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594982239
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119345/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595586848
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24177/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595600005
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119441/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594717
 
 
   > I've just checked the documentation and `spark.streaming.receiver.maxRate` says in the doc that default number is `not set` but in the code it's `Long.MaxValue`.
   
   I don't know the reason.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592917835
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119118/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896210
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119117/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593348144
 
 
   **[Test build #119164 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119164/testReport)** for PR 27744 at commit [`37b2e93`](https://github.com/apache/spark/commit/37b2e93abf3e7f377d36973b1f893eafad4367b1).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386219669
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   ~~Only here `DELTA_CHAIN_LENGTH_THRESHOLD`.~~

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925870
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/23862/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594606056
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119313/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595261212
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24134/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595513559
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24168/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595273462
 
 
   **[Test build #119398 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119398/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592913639
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595513559
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24168/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595721546
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594960900
 
 
   **[Test build #119345 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119345/testReport)** for PR 27744 at commit [`005211f`](https://github.com/apache/spark/commit/005211f7e104f7935d433a989144641a7f51bb4e).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592895678
 
 
   **[Test build #119117 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119117/testReport)** for PR 27744 at commit [`e982f5c`](https://github.com/apache/spark/commit/e982f5c308dfd48715d6e72a215d8ddef9ac35f3).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595518790
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595684819
 
 
   **[Test build #119459 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119459/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
gaborgsomogyi commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595196604
 
 
   > > I've just checked the documentation and `spark.streaming.receiver.maxRate` says in the doc that default number is `not set` but in the code it's `Long.MaxValue`.
   > 
   > I don't know the reason.
   
   Had a deeper look and `Long.MaxValue` is defined as unlimited (not set) so we can consider it's fine.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592925728
 
 
   **[Test build #119120 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119120/testReport)** for PR 27744 at commit [`5c29613`](https://github.com/apache/spark/commit/5c29613428bffad5b0aae7cbcec85123a1ef4819).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386203986
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   Is `DELTA_CHAIN_LENGTH_THRESHOLD` used other than this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594961492
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/24082/
   Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896111
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595261203
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594961480
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595287847
 
 
   **[Test build #119398 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119398/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595287950
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/119398/
   Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595721082
 
 
   **[Test build #119459 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119459/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on a change in pull request #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#discussion_r386268065
 
 

 ##########
 File path: streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala
 ##########
 @@ -61,8 +62,7 @@ private[streaming] object StateMap {
   def empty[K, S]: StateMap[K, S] = new EmptyStateMap[K, S]
 
   def create[K: ClassTag, S: ClassTag](conf: SparkConf): StateMap[K, S] = {
-    val deltaChainThreshold = conf.getInt("spark.streaming.sessionByKey.deltaChainThreshold",
-      DELTA_CHAIN_LENGTH_THRESHOLD)
+    val deltaChainThreshold = conf.get(SESSION_BY_KEY_DELTA_CHAIN_THRESHOLD)
 
 Review comment:
   I'm sorry, I read it wrong.`DELTA_CHAIN_LENGTH_THRESHOLD` used in two place.
   https://github.com/apache/spark/blob/5c29613428bffad5b0aae7cbcec85123a1ef4819/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala#L87
   https://github.com/apache/spark/blob/5c29613428bffad5b0aae7cbcec85123a1ef4819/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala#L102

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
beliefer commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595258550
 
 
   @gaborgsomogyi Thanks.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594606041
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595513551
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595287931
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-595273462
 
 
   **[Test build #119398 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119398/testReport)** for PR 27744 at commit [`00fde50`](https://github.com/apache/spark/commit/00fde50a7d3144927cbc133709b270dee9397ced).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-592896204
 
 
   Merged build finished. Test FAILed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594982003
 
 
   **[Test build #119345 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/119345/testReport)** for PR 27744 at commit [`005211f`](https://github.com/apache/spark/commit/005211f7e104f7935d433a989144641a7f51bb4e).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594594961
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][STREAMING] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-593348590
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #27744: [SPARK-30992][DSTREAMS] Arrange scattered config of streaming module
URL: https://github.com/apache/spark/pull/27744#issuecomment-594982235
 
 
   Merged build finished. Test PASSed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

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