Skip to content

Commit 747393a

Browse files
author
Ilya Ganelin
committed
[SPARK-5932] Added unit tests for ByteString conversion
1 parent 09ea450 commit 747393a

File tree

3 files changed

+108
-9
lines changed

3 files changed

+108
-9
lines changed

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 40 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1036,6 +1036,46 @@ private[spark] object Utils extends Logging {
10361036
JavaUtils.timeStringAsSec(str)
10371037
}
10381038

1039+
/**
1040+
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to bytes for
1041+
* internal use.
1042+
*
1043+
* If no suffix is provided, the passed number is assumed to be in bytes.
1044+
*/
1045+
def byteStringAsBytes(str: String): Long = {
1046+
JavaUtils.byteStringAsBytes(str)
1047+
}
1048+
1049+
/**
1050+
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to kilobytes for
1051+
* internal use.
1052+
*
1053+
* If no suffix is provided, the passed number is assumed to be in kilobytes.
1054+
*/
1055+
def byteStringAsKB(str: String): Long = {
1056+
JavaUtils.byteStringAsKB(str)
1057+
}
1058+
1059+
/**
1060+
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
1061+
* internal use.
1062+
*
1063+
* If no suffix is provided, the passed number is assumed to be in megabytes.
1064+
*/
1065+
def byteStringAsMB(str: String): Long = {
1066+
JavaUtils.byteStringAsMB(str)
1067+
}
1068+
1069+
/**
1070+
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb, 500gb) to gigabytes for
1071+
* internal use.
1072+
*
1073+
* If no suffix is provided, the passed number is assumed to be in gigabytes.
1074+
*/
1075+
def byteStringAsGB(str: String): Long = {
1076+
JavaUtils.byteStringAsGB(str)
1077+
}
1078+
10391079
/**
10401080
* Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes.
10411081
*/

core/src/test/scala/org/apache/spark/util/UtilsSuite.scala

Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ import org.scalatest.FunSuite
3333
import org.apache.hadoop.conf.Configuration
3434
import org.apache.hadoop.fs.Path
3535

36+
import org.apache.spark.network.util.ByteUnit
3637
import org.apache.spark.SparkConf
3738

3839
class UtilsSuite extends FunSuite with ResetSystemProperties {
@@ -63,6 +64,10 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
6364
assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1))
6465

6566
// Test invalid strings
67+
intercept[NumberFormatException] {
68+
Utils.timeStringAsMs("600l")
69+
}
70+
6671
intercept[NumberFormatException] {
6772
Utils.timeStringAsMs("This breaks 600s")
6873
}
@@ -79,6 +84,64 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
7984
Utils.timeStringAsMs("This 123s breaks")
8085
}
8186
}
87+
88+
test("Test byteString conversion") {
89+
// Test zero
90+
assert(Utils.byteStringAsBytes("0") === 0)
91+
92+
assert(Utils.byteStringAsGB("1") === 1)
93+
assert(Utils.byteStringAsGB("1g") === 1)
94+
assert(Utils.byteStringAsGB("1000m") === 1)
95+
assert(Utils.byteStringAsGB("1000000k") === 1)
96+
assert(Utils.byteStringAsGB("1k") === ByteUnit.KB.toGB(1))
97+
assert(Utils.byteStringAsGB("1m") === ByteUnit.MB.toGB(1))
98+
assert(Utils.byteStringAsGB("1t") === ByteUnit.TB.toGB(1))
99+
assert(Utils.byteStringAsGB("1p") === ByteUnit.PB.toGB(1))
100+
101+
assert(Utils.byteStringAsMB("1") === 1)
102+
assert(Utils.byteStringAsMB("1m") === 1)
103+
assert(Utils.byteStringAsMB("1000k") === 1)
104+
assert(Utils.byteStringAsMB("1k") === ByteUnit.KB.toMB(1))
105+
assert(Utils.byteStringAsMB("1g") === ByteUnit.GB.toMB(1))
106+
assert(Utils.byteStringAsMB("1t") === ByteUnit.TB.toMB(1))
107+
assert(Utils.byteStringAsMB("1p") === ByteUnit.PB.toMB(1))
108+
109+
assert(Utils.byteStringAsKB("1") === 1)
110+
assert(Utils.byteStringAsKB("1k") === 1)
111+
assert(Utils.byteStringAsKB("1m") === ByteUnit.MB.toKB(1))
112+
assert(Utils.byteStringAsKB("1g") === ByteUnit.GB.toKB(1))
113+
assert(Utils.byteStringAsKB("1t") === ByteUnit.TB.toKB(1))
114+
assert(Utils.byteStringAsKB("1p") === ByteUnit.PB.toKB(1))
115+
116+
assert(Utils.byteStringAsBytes("1") === 1)
117+
assert(Utils.byteStringAsBytes("1k") === ByteUnit.KB.toBytes(1))
118+
assert(Utils.byteStringAsBytes("1m") === ByteUnit.MB.toBytes(1))
119+
assert(Utils.byteStringAsBytes("1g") === ByteUnit.GB.toBytes(1))
120+
assert(Utils.byteStringAsBytes("1t") === ByteUnit.TB.toBytes(1))
121+
assert(Utils.byteStringAsBytes("1p") === ByteUnit.PB.toBytes(1))
122+
123+
// Test invalid strings
124+
intercept[NumberFormatException] {
125+
Utils.byteStringAsBytes("500ub")
126+
}
127+
128+
// Test invalid strings
129+
intercept[NumberFormatException] {
130+
Utils.byteStringAsBytes("This breaks 600b")
131+
}
132+
133+
intercept[NumberFormatException] {
134+
Utils.byteStringAsBytes("This breaks 600")
135+
}
136+
137+
intercept[NumberFormatException] {
138+
Utils.byteStringAsBytes("600gb This breaks")
139+
}
140+
141+
intercept[NumberFormatException] {
142+
Utils.byteStringAsBytes("This 123mb breaks")
143+
}
144+
}
82145

83146
test("bytesToString") {
84147
assert(Utils.bytesToString(10) === "10.0 B")

network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java

Lines changed: 5 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -22,14 +22,12 @@
2222
import java.io.IOException;
2323
import java.nio.ByteBuffer;
2424
import java.util.concurrent.TimeUnit;
25-
2625
import java.util.regex.Matcher;
2726
import java.util.regex.Pattern;
2827

2928
import com.google.common.base.Charsets;
3029
import com.google.common.base.Preconditions;
3130
import com.google.common.collect.ImmutableMap;
32-
import com.sun.javafx.css.SizeUnits;
3331
import io.netty.buffer.Unpooled;
3432
import org.slf4j.Logger;
3533
import org.slf4j.LoggerFactory;
@@ -142,11 +140,11 @@ private static boolean isSymlink(File file) throws IOException {
142140
private static ImmutableMap<String, ByteUnit> byteSuffixes =
143141
ImmutableMap.<String, ByteUnit>builder()
144142
.put("b", ByteUnit.BYTE)
145-
.put("kb", ByteUnit.KB)
146-
.put("mb", ByteUnit.MB)
147-
.put("gb", ByteUnit.GB)
148-
.put("tb", ByteUnit.TB)
149-
.put("pb", ByteUnit.PB)
143+
.put("k", ByteUnit.KB)
144+
.put("m", ByteUnit.MB)
145+
.put("g", ByteUnit.GB)
146+
.put("t", ByteUnit.TB)
147+
.put("p", ByteUnit.PB)
150148
.build();
151149

152150
/**
@@ -273,6 +271,4 @@ public static long byteStringAsMB(String str) {
273271
public static long byteStringAsGB(String str) {
274272
return parseByteString(str, ByteUnit.GB);
275273
}
276-
277-
278274
}

0 commit comments

Comments
 (0)