-
Notifications
You must be signed in to change notification settings - Fork 28.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-26491][CORE][TEST] Use ConfigEntry for hardcoded configs for test categories #23413
Changes from 4 commits
73618ce
ba41657
b54a07b
a0ae515
10d1a97
10e0237
0df2dfe
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} | |
import org.apache.spark._ | ||
import org.apache.spark.annotation.DeveloperApi | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config.Tests.IS_TESTING | ||
import org.apache.spark.scheduler.AccumulableInfo | ||
import org.apache.spark.storage.{BlockId, BlockStatus} | ||
import org.apache.spark.util._ | ||
|
@@ -202,7 +203,7 @@ class TaskMetrics private[spark] () extends Serializable { | |
} | ||
|
||
// Only used for test | ||
private[spark] val testAccum = sys.props.get("spark.testing").map(_ => new LongAccumulator) | ||
private[spark] val testAccum = sys.props.get(IS_TESTING.key).map(_ => new LongAccumulator) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Same. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I agree it is doable but the code would become an if which seems to me less clean than the current code |
||
|
||
|
||
import InternalAccumulator._ | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
/* | ||
* 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.internal.config | ||
|
||
private[spark] object Tests { | ||
val TEST_MEMORY = ConfigBuilder("spark.testing.memory") | ||
.longConf | ||
.createWithDefault(Runtime.getRuntime.maxMemory) | ||
|
||
val TEST_SCHEDULE_INTERVAL = | ||
ConfigBuilder("spark.testing.dynamicAllocation.scheduleInterval") | ||
.longConf | ||
.createWithDefault(100) | ||
|
||
val IS_TESTING = ConfigBuilder("spark.testing") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems to me that if you use
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. there are cases when it is removed (see the HistoryServer one) and when it is set to false (in the Kubernetes one). So I don't think it is doable. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I actually tried out the k8s tests without that line and they work fine, which tells me it's not needed. Same for a lot of other explicit checks and setting of that property. I think eventually we should fix all this code to not mess with that property, especially since it's so inconsistent (SparkConf vs. system properties vs. env variable). The SHS test does need to remove the conf, though; it could be implemented differently, but at that point it's probably better to do a separate change and leave this one as a simple "make everybody use constants for this" change. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. +1 for this. I think we should check case by case if some of them are not needed and removed, but I agree we best do that in dedicated PRs for each of these cases. |
||
.booleanConf | ||
.createOptional | ||
|
||
val TEST_USE_COMPRESSED_OOPS = ConfigBuilder("spark.test.useCompressedOops") | ||
.booleanConf | ||
.createOptional | ||
|
||
val TEST_NO_STAGE_RETRY = ConfigBuilder("spark.test.noStageRetry") | ||
.booleanConf | ||
.createWithDefault(false) | ||
|
||
val TEST_RESERVED_MEMORY = ConfigBuilder("spark.testing.reservedMemory") | ||
.longConf | ||
.createOptional | ||
|
||
val TEST_N_HOSTS = ConfigBuilder("spark.testing.nHosts") | ||
.intConf | ||
.createWithDefault(5) | ||
|
||
val TEST_N_EXECUTORS_HOST = ConfigBuilder("spark.testing.nExecutorsPerHost") | ||
.intConf | ||
.createWithDefault(4) | ||
|
||
val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") | ||
.intConf | ||
.createWithDefault(2) | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -28,6 +28,7 @@ import com.google.common.collect.MapMaker | |
|
||
import org.apache.spark.annotation.DeveloperApi | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.internal.config.Tests.TEST_USE_COMPRESSED_OOPS | ||
import org.apache.spark.util.collection.OpenHashSet | ||
|
||
/** | ||
|
@@ -126,8 +127,8 @@ object SizeEstimator extends Logging { | |
private def getIsCompressedOops: Boolean = { | ||
// This is only used by tests to override the detection of compressed oops. The test | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Given this comment I wonder if changing this one config is needed. (I also don't see any There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. there are cases when this config is set in the tests. It seems it is useless indeed, since we read it only here as a system property. I think we can consider removing the useless set as part of another PR related to that. Otherwise we can try and replace the config with a string There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I didn't mean it's useless (it might be, didn't look), just that it could be kept as a system property instead of a config constant (since it's never set in or read from SparkConf). Using a string constant sounds fine, but given what I think is the goal of these changes (see SPARK-26060), that wouldn't really change much. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the point is: it is set in There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see; then it can probably be just removed from those tests. |
||
// actually uses a system property instead of a SparkConf, so we'll stick with that. | ||
if (System.getProperty("spark.test.useCompressedOops") != null) { | ||
return System.getProperty("spark.test.useCompressedOops").toBoolean | ||
if (System.getProperty(TEST_USE_COMPRESSED_OOPS.key) != null) { | ||
return System.getProperty(TEST_USE_COMPRESSED_OOPS.key).toBoolean | ||
} | ||
|
||
// java.vm.info provides compressed ref info for IBM JDKs | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can probably be
Utils.isTesting
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think so, here we are checking the
SparkConf
, while inUtils.isTesting
we check the system properties. So I don't think it is doable.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well, that's kinda the point.
spark.testing
is set as a system property by the build scripts.SparkConf
just inherits system properties, which is why you can also check it there.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
(I saw your comment about the SHS tests removing the testing conf, let me take a look...)