-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-5931][CORE] Use consistent naming for time properties #5236
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
Changes from 1 commit
59bf9e1
404f8c3
7db6d2a
4933fda
c9f5cad
21ef3dd
064ebd6
7320c87
272c215
3352d34
3f1cfc8
6d1518e
2fcc91c
5181597
c6a0095
cde9bff
42477aa
9a29d8d
34f87c2
8f741e1
9e2547c
499bdf0
5232a36
3a12dd8
68f4e93
70ac213
647b5ac
1c0c07c
8613631
bac9edf
1858197
3b126e1
39164f9
b2fc965
dd0a680
bf779b0
76cfa27
5193d5f
6387772
19c31af
ff40bfe
28187bf
1465390
cbf41db
d4efd26
4e48679
1a1122c
cbd2ca6
6f651a8
7d19cdd
dc7bd08
69fedcc
8927e66
642a06d
25d3f52
bc04e05
951ca2d
f5fafcd
de3bff9
4526c81
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory | |
| import java.net._ | ||
| import java.nio.ByteBuffer | ||
| import java.util.{Properties, Locale, Random, UUID} | ||
| import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} | ||
| import java.util.concurrent._ | ||
| import javax.net.ssl.HttpsURLConnection | ||
|
|
||
| import scala.collection.JavaConversions._ | ||
|
|
@@ -1010,63 +1010,64 @@ private[spark] object Utils extends Logging { | |
| ) | ||
| } | ||
|
|
||
| /** Check whether a time-suffix was provided for the time string. */ | ||
| private def hasTimeSuffix(str: String) : Boolean = { | ||
| val lower = str.toLowerCase.trim() | ||
| lower.endsWith("ms") || lower.endsWith("us") || lower.endsWith("s") | ||
| } | ||
|
|
||
| val timeError = "Time must be specified as seconds (s), " + | ||
| "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." | ||
|
|
||
| val timeSuffixes = Map ( | ||
| "us" -> TimeUnit.MICROSECONDS, | ||
| "ms" -> TimeUnit.MILLISECONDS, | ||
| "s" -> TimeUnit.SECONDS, | ||
| "min" -> TimeUnit.MINUTES, | ||
| "h" -> TimeUnit.HOURS, | ||
| "d" -> TimeUnit.DAYS | ||
| ) | ||
| /** | ||
|
Contributor
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. nit: add empty line before comment start |
||
| * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a microsecond count for | ||
| * internal use. If no suffix is provided a direct conversion is attempted. | ||
| */ | ||
| @throws(classOf[NumberFormatException]) | ||
|
Contributor
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. This is a |
||
| private def timeStringToUs(str: String) : Long = { | ||
| private def parseTimeString(str: String) : (Option[TimeUnit], Long) = { | ||
| val timeError = "Time must be specified as seconds (s), " + | ||
|
Contributor
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. nit: indentation is wrong |
||
| "milliseconds (ms), or microseconds (us) e.g. 50s, 100ms, or 250us." | ||
|
|
||
| try { | ||
| val lower = str.toLowerCase.trim() | ||
| if (lower.endsWith("ms")) { | ||
| lower.substring(0, lower.length - 2).toLong * 1000 | ||
| } else if (lower.endsWith("us")) { | ||
| lower.substring(0, lower.length - 2).toLong | ||
| } else if (lower.endsWith("s")) { | ||
| lower.substring(0, lower.length - 1).toLong * 1000 * 1000 | ||
| } else { | ||
| // Invalid suffix, force correct formatting | ||
| lower.toLong | ||
| } | ||
| var suffix: String = "" | ||
| timeSuffixes.foreach(s => { | ||
| if(lower.endsWith(s._1)) | ||
| suffix = s._1 | ||
| }) | ||
|
|
||
| (timeSuffixes.get(suffix), str.substring(0, str.length - suffix.length).toLong) | ||
| } catch { | ||
| case e: NumberFormatException => throw new NumberFormatException(timeError) | ||
|
|
||
|
||
| case e: NumberFormatException => throw new NumberFormatException(timeError + "\n" + | ||
| e.toString) | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If | ||
| * no suffix is provided, the passed number is assumed to be in us. | ||
| */ | ||
| @throws(classOf[NumberFormatException]) | ||
| def timeStringAsUs(str: String): Long = { | ||
|
||
| timeStringToUs(str) | ||
| val parsed = parseTimeString(str) | ||
| parsed._1.getOrElse(TimeUnit.MICROSECONDS).toMicros(parsed._2) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If | ||
| * no suffix is provided, the passed number is assumed to be in ms. | ||
| */ | ||
| @throws(classOf[NumberFormatException]) | ||
| def timeStringAsMs(str : String) : Long = { | ||
|
||
| timeStringToUs(str)/1000 | ||
| val parsed = parseTimeString(str) | ||
| parsed._1.getOrElse(TimeUnit.MILLISECONDS).toMicros(parsed._2) | ||
| } | ||
|
|
||
| /** | ||
| * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If | ||
| * no suffix is provided, the passed number is assumed to be in seconds. | ||
| */ | ||
| @throws(classOf[NumberFormatException]) | ||
| def timeStringAsS(str : String) : Long = { | ||
| timeStringToUs(str)/1000/1000 | ||
| val parsed = parseTimeString(str) | ||
| parsed._1.getOrElse(TimeUnit.SECONDS).toMicros(parsed._2) | ||
| } | ||
|
|
||
| /** | ||
|
|
||
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 this be private? also, no space after
Map.