Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
49 commits
Select commit Hold shift + click to select a range
d7a06b8
Updated SparkConf class to add getOrCreate method. Started test suite…
Apr 13, 2015
a99032f
Spacing fix
Apr 14, 2015
e92caf7
[SPARK-6703] Added test to ensure that getOrCreate both allows creati…
Apr 14, 2015
8be2f83
Replaced match with if
Apr 14, 2015
733ec9f
Fixed some bugs in test code
Apr 14, 2015
dfec4da
Changed activeContext to AtomicReference
Apr 14, 2015
0e1567c
Got rid of unecessary option for AtomicReference
Apr 14, 2015
15e8dea
Updated comments and added MiMa Exclude
Apr 14, 2015
270cfe3
[SPARK-6703] Documentation fixes
Apr 14, 2015
cb0c6b7
Doc updates and code cleanup
Apr 14, 2015
8c884fa
Made getOrCreate synchronized
Apr 14, 2015
1dc0444
Added ref equality check
Apr 14, 2015
db9a963
Closing second spark context
Apr 17, 2015
5390fd9
Merge remote-tracking branch 'upstream/master' into SPARK-5932
Apr 18, 2015
09ea450
[SPARK-5932] Added byte string conversion to Jav utils
Apr 18, 2015
747393a
[SPARK-5932] Added unit tests for ByteString conversion
Apr 18, 2015
a9f4fcf
[SPARK-5932] Added unit tests for unit conversion
Apr 18, 2015
851d691
[SPARK-5932] Updated memoryStringToMb to use new interfaces
Apr 18, 2015
475370a
[SPARK-5932] Simplified ByteUnit code, switched to using longs. Updat…
Apr 18, 2015
0cdff35
[SPARK-5932] Updated to use bibibytes in method names. Updated spark.…
Apr 18, 2015
b809a78
[SPARK-5932] Updated spark.kryoserializer.buffer.max
Apr 18, 2015
eba4de6
[SPARK-5932] Updated spark.shuffle.file.buffer.kb
Apr 18, 2015
1fbd435
[SPARK-5932] Updated spark.broadcast.blockSize
Apr 18, 2015
2d15681
[SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes
Apr 18, 2015
ae7e9f6
[SPARK-5932] Updated spark.io.compression.snappy.block.size
Apr 18, 2015
afc9a38
[SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memo…
Apr 18, 2015
7a6c847
[SPARK-5932] Updated spark.shuffle.file.buffer
Apr 18, 2015
5d29f90
[SPARK-5932] Finished documentation updates
Apr 18, 2015
928469e
[SPARK-5932] Converted some longs to ints
Apr 18, 2015
35a7fa7
Minor formatting
Apr 18, 2015
0f4443e
Merge remote-tracking branch 'upstream/master' into SPARK-5932
Apr 18, 2015
f15f209
Fixed conversion of kryo buffer size
Apr 19, 2015
f32bc01
[SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversio…
Apr 19, 2015
69e2f20
Updates to code
Apr 21, 2015
54b78b4
Simplified byteUnit class
Apr 21, 2015
c7803cd
Empty lines
Apr 21, 2015
fe286b4
Resolved merge conflict
Apr 21, 2015
d3d09b6
[SPARK-5932] Fixing error in KryoSerializer
Apr 21, 2015
84a2581
Added smoother handling of fractional values for size parameters. Thi…
Apr 21, 2015
8b43748
Fixed error in pattern matching for doubles
Apr 21, 2015
e428049
resolving merge conflict
Apr 22, 2015
3dfae96
Fixed some nits. Added automatic conversion of old paramter for kryos…
Apr 22, 2015
22413b1
Made MAX private
Apr 22, 2015
9ee779c
Simplified fraction matches
Apr 22, 2015
852a407
[SPARK-5932] Added much improved overflow handling. Can now handle si…
Apr 23, 2015
fc85733
Got rid of floating point math
Apr 24, 2015
2ab886b
Scala style
Apr 24, 2015
49a8720
Whitespace fix
Apr 24, 2015
11f6999
Nit fixes
Apr 24, 2015
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-5932] Updated to use bibibytes in method names. Updated spark.…
…kryoserializer.buffer.mb and spark.reducer.maxMbInFlight
  • Loading branch information
Ilya Ganelin committed Apr 18, 2015
commit 0cdff352881a2a6c0927c38c52b9574c96b14ce7
83 changes: 80 additions & 3 deletions core/src/main/scala/org/apache/spark/SparkConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -211,7 +211,74 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging {
Utils.timeStringAsMs(get(key, defaultValue))
}

/**
* Get a size parameter as bytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then bytes are assumed.
* @throws NoSuchElementException
*/
def getSizeAsBytes(key: String): Long = {
Utils.byteStringAsBytes(get(key))
}

/**``
* Get a size parameter as bytes, falling back to a default if not set. If no
* suffix is provided then bytes are assumed.
*/
def getSizeAsBytes(key: String, defaultValue: String): Long = {
Utils.byteStringAsBytes(get(key, defaultValue))
}

/**
* Get a size parameter as Kibibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Kibibytes are assumed.
* @throws NoSuchElementException
*/
def getSizeAsKb(key: String): Long = {
Utils.byteStringAsMb(get(key))
}

/**
* Get a size parameter as Kibibytes, falling back to a default if not set. If no
* suffix is provided then Kibibytes are assumed.
*/
def getSizeAsKb(key: String, defaultValue: String): Long = {
Utils.byteStringAsMb(get(key, defaultValue))
}

/**
* Get a size parameter as Mebibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Mebibytes are assumed.
* @throws NoSuchElementException
*/
def getSizeAsMb(key: String): Long = {
Utils.byteStringAsMb(get(key))
}

/**
* Get a size parameter as Mebibytes, falling back to a default if not set. If no
* suffix is provided then Mebibytes are assumed.
*/
def getSizeAsMb(key: String, defaultValue: String): Long = {
Utils.byteStringAsMb(get(key, defaultValue))
}

/**
* Get a size parameter as Gibibytes; throws a NoSuchElementException if it's not set. If no
* suffix is provided then Gibibytes are assumed.
* @throws NoSuchElementException
*/
def getSizeAsGb(key: String): Long = {
Utils.byteStringAsGb(get(key))
}

/**
* Get a size parameter as Gibibytes, falling back to a default if not set. If no
* suffix is provided then Gibibytes are assumed.
*/
def getSizeAsGb(key: String, defaultValue: String): Long = {
Utils.byteStringAsGb(get(key, defaultValue))
}

/** Get a parameter as an Option */
def getOption(key: String): Option[String] = {
Option(settings.get(key)).orElse(getDeprecatedConfig(key, this))
Expand Down Expand Up @@ -407,7 +474,13 @@ private[spark] object SparkConf extends Logging {
"The spark.cache.class property is no longer being used! Specify storage levels using " +
"the RDD.persist() method instead."),
DeprecatedConfig("spark.yarn.user.classpath.first", "1.3",
"Please use spark.{driver,executor}.userClassPathFirst instead."))
"Please use spark.{driver,executor}.userClassPathFirst instead."),
DeprecatedConfig("spark.reducer.maxMbInFlight", "1.4",
"Please use spark.reducer.maxSizeInFlight instead."),
DeprecatedConfig("spark.kryoserializer.buffer.mb", "1.4",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could have a translation function for this, no? Like spark.yarn.applicationMaster.waitTries below.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Automatic translation would fail with a NumberFormatException though if they try to pass in a correctly formatted new value for the old config (e.g. 64k) and there's nowhere to catch that exception in this code.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since you added the alternate config, you can remove this entry.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wanted to still provide this message to clarify that fractional values are no longer allowed.

"Please use spark.kryoserializer.buffer instead."))


Map(configs.map { cfg => (cfg.key -> cfg) }:_*)
}

Expand All @@ -431,8 +504,12 @@ private[spark] object SparkConf extends Logging {
"spark.yarn.am.waitTime" -> Seq(
AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3",
// Translate old value to a duration, with 10s wait time per try.
translation = s => s"${s.toLong * 10}s"))
)
translation = s => s"${s.toLong * 10}s")),
"spark.reducer.maxSizeInFlight" -> Seq(
AlternateConfig("spark.reducer.maxMbInFlight", "1.4")),
"spark.kryoserializer.buffer" -> Seq(
AlternateConfig("spark.kryoserializer.buffer.mb", "1.4"))
)

/**
* A view of `configsWithAlternatives` that makes it more efficient to look up deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,9 +49,9 @@ class KryoSerializer(conf: SparkConf)
with Logging
with Serializable {

private val bufferSizeMb = conf.getDouble("spark.kryoserializer.buffer.mb", 0.064)
private val bufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer", "64k")
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note - Intentionally converted to 64k here instead of 0.064m since I assume this was the original intent and 0.064m does NOT equal 64k in general (mibi vs megabytes)

if (bufferSizeMb >= 2048) {
throw new IllegalArgumentException("spark.kryoserializer.buffer.mb must be less than " +
throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " +
s"2048 mb, got: + $bufferSizeMb mb.")
}
private val bufferSize = (bufferSizeMb * 1024 * 1024).toInt
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,8 @@ private[hash] object BlockStoreShuffleFetcher extends Logging {
blockManager,
blocksByAddress,
serializer,
SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024)
// Note: we use getSizeAsMb to assume Mb when no suffix is provided
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • for backward compatibility

SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024)
val itr = blockFetcherItr.flatMap(unpackBlock)

val completionIter = CompletionIterator[T, Iterator[T]](itr, {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ class DistributedSuite extends FunSuite with Matchers with LocalSparkContext {
}

test("groupByKey where map output sizes exceed maxMbInFlight") {
val conf = new SparkConf().set("spark.reducer.maxMbInFlight", "1")
val conf = new SparkConf().set("spark.reducer.maxSizeInFlight", "1m")
sc = new SparkContext(clusterUrl, "test", conf)
// This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output
// file should be about 2.5 MB
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
test("kryo without resizable output buffer should fail on large array") {
val conf = new SparkConf(false)
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
conf.set("spark.kryoserializer.buffer.mb", "1")
conf.set("spark.kryoserializer.buffer", "1m")
conf.set("spark.kryoserializer.buffer.max.mb", "1")
val sc = new SparkContext("local", "test", conf)
intercept[SparkException](sc.parallelize(x).collect())
Expand All @@ -43,7 +43,7 @@ class KryoSerializerResizableOutputSuite extends FunSuite {
test("kryo with resizable output buffer should succeed on large array") {
val conf = new SparkConf(false)
conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
conf.set("spark.kryoserializer.buffer.mb", "1")
conf.set("spark.kryoserializer.buffer", "1m")
conf.set("spark.kryoserializer.buffer.max.mb", "2")
val sc = new SparkContext("local", "test", conf)
assert(sc.parallelize(x).collect() === x)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd
val allStores = new ArrayBuffer[BlockManager]

// Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
conf.set("spark.kryoserializer.buffer.mb", "1")
conf.set("spark.kryoserializer.buffer", "1m")
val serializer = new KryoSerializer(conf)

// Implicitly convert strings to BlockIds for test clarity.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach
val shuffleManager = new HashShuffleManager(conf)

// Reuse a serializer across tests to avoid creating a new thread-local buffer on each test
conf.set("spark.kryoserializer.buffer.mb", "1")
conf.set("spark.kryoserializer.buffer", "1m")
val serializer = new KryoSerializer(conf)

// Implicitly convert strings to BlockIds for test clarity.
Expand Down
28 changes: 14 additions & 14 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -96,8 +96,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(Utils.byteStringAsGb("1048575k") === 0)
assert(Utils.byteStringAsGb("1048576k") === 1)
assert(Utils.byteStringAsGb("1k") === 0)
assert(Utils.byteStringAsGb("1t") === ByteUnit.TB.toGB(1))
assert(Utils.byteStringAsGb("1p") === ByteUnit.PB.toGB(1))
assert(Utils.byteStringAsGb("1t") === ByteUnit.TiB.toGiB(1))
assert(Utils.byteStringAsGb("1p") === ByteUnit.PiB.toGiB(1))

assert(Utils.byteStringAsMb("1") === 1)
assert(Utils.byteStringAsMb("1m") === 1)
Expand All @@ -106,23 +106,23 @@ class UtilsSuite extends FunSuite with ResetSystemProperties {
assert(Utils.byteStringAsMb("1023k") === 0)
assert(Utils.byteStringAsMb("1024k") === 1)
assert(Utils.byteStringAsMb("3645k") === 3)
assert(Utils.byteStringAsMb("1g") === ByteUnit.GB.toMB(1))
assert(Utils.byteStringAsMb("1t") === ByteUnit.TB.toMB(1))
assert(Utils.byteStringAsMb("1p") === ByteUnit.PB.toMB(1))
assert(Utils.byteStringAsMb("1g") === ByteUnit.GiB.toMiB(1))
assert(Utils.byteStringAsMb("1t") === ByteUnit.TiB.toMiB(1))
assert(Utils.byteStringAsMb("1p") === ByteUnit.PiB.toMiB(1))

assert(Utils.byteStringAsKb("1") === 1)
assert(Utils.byteStringAsKb("1k") === 1)
assert(Utils.byteStringAsKb("1m") === ByteUnit.MB.toKB(1))
assert(Utils.byteStringAsKb("1g") === ByteUnit.GB.toKB(1))
assert(Utils.byteStringAsKb("1t") === ByteUnit.TB.toKB(1))
assert(Utils.byteStringAsKb("1p") === ByteUnit.PB.toKB(1))
assert(Utils.byteStringAsKb("1m") === ByteUnit.MiB.toKiB(1))
assert(Utils.byteStringAsKb("1g") === ByteUnit.GiB.toKiB(1))
assert(Utils.byteStringAsKb("1t") === ByteUnit.TiB.toKiB(1))
assert(Utils.byteStringAsKb("1p") === ByteUnit.PiB.toKiB(1))

assert(Utils.byteStringAsBytes("1") === 1)
assert(Utils.byteStringAsBytes("1k") === ByteUnit.KB.toBytes(1))
assert(Utils.byteStringAsBytes("1m") === ByteUnit.MB.toBytes(1))
assert(Utils.byteStringAsBytes("1g") === ByteUnit.GB.toBytes(1))
assert(Utils.byteStringAsBytes("1t") === ByteUnit.TB.toBytes(1))
assert(Utils.byteStringAsBytes("1p") === ByteUnit.PB.toBytes(1))
assert(Utils.byteStringAsBytes("1k") === ByteUnit.KiB.toBytes(1))
assert(Utils.byteStringAsBytes("1m") === ByteUnit.MiB.toBytes(1))
assert(Utils.byteStringAsBytes("1g") === ByteUnit.GiB.toBytes(1))
assert(Utils.byteStringAsBytes("1t") === ByteUnit.TiB.toBytes(1))
assert(Utils.byteStringAsBytes("1p") === ByteUnit.PiB.toBytes(1))

// Test invalid strings
intercept[NumberFormatException] {
Expand Down
12 changes: 6 additions & 6 deletions docs/configuration.md
Original file line number Diff line number Diff line change
Expand Up @@ -366,10 +366,10 @@ Apart from these, the following properties are also available, and may be useful
<table class="table">
<tr><th>Property Name</th><th>Default</th><th>Meaning</th></tr>
<tr>
<td><code>spark.reducer.maxMbInFlight</code></td>
<td>48</td>
<td><code>spark.reducer.maxSizeInFlight</code></td>
<td>48m</td>
<td>
Maximum size (in megabytes) of map outputs to fetch simultaneously from each reduce task. Since
Maximum size of map outputs to fetch simultaneously from each reduce task. Since
each output requires us to create a buffer to receive it, this represents a fixed memory
overhead per reduce task, so keep it small unless you have a large amount of memory.
</td>
Expand Down Expand Up @@ -650,10 +650,10 @@ Apart from these, the following properties are also available, and may be useful
</td>
</tr>
<tr>
<td><code>spark.kryoserializer.buffer.mb</code></td>
<td>0.064</td>
<td><code>spark.kryoserializer.buffer</code></td>
<td>64k</td>
<td>
Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer
Initial size of Kryo's serialization buffer. Note that there will be one buffer
<i>per core</i> on each worker. This buffer will grow up to
<code>spark.kryoserializer.buffer.max.mb</code> if needed.
</td>
Expand Down
2 changes: 1 addition & 1 deletion docs/tuning.md
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ val sc = new SparkContext(conf)
The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes more advanced
registration options, such as adding custom serialization code.

If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb`
If your objects are large, you may also need to increase the `spark.kryoserializer.buffer`
config property. The default is 2, but this value needs to be large enough to hold the *largest*
object you will serialize.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ object MovieLensALS {
val conf = new SparkConf().setAppName(s"MovieLensALS with $params")
if (params.kryo) {
conf.registerKryoClasses(Array(classOf[mutable.BitSet], classOf[Rating]))
.set("spark.kryoserializer.buffer.mb", "8")
.set("spark.kryoserializer.buffer", "8m")
}
val sc = new SparkContext(conf)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,46 +25,46 @@ public enum ByteUnit {
},

/** Kibibyte (KiB) = 1024 Byte */
KB {
public long toBytes(long d) { return x(d, C_KB); }
KiB {
public long toBytes(long d) { return x(d, C_KiB); }

public long convert(long d, ByteUnit u) { return u.toKB(d); }
public long convert(long d, ByteUnit u) { return u.toKiB(d); }
},

/** Mebibyte (MiB) = (1024^2) Byte */
MB {
public long toBytes(long d) { return x(d, C_MB); }
MiB {
public long toBytes(long d) { return x(d, C_MiB); }

public long convert(long d, ByteUnit u) { return u.toMB(d); }
public long convert(long d, ByteUnit u) { return u.toMiB(d); }
},

/** Gibibyte (GiB) = (1024^3) Byte */
GB {
public long toBytes(long d) { return x(d, C_GB);
GiB {
public long toBytes(long d) { return x(d, C_GiB);
}

public long convert(long d, ByteUnit u) { return u.toGB(d); }
public long convert(long d, ByteUnit u) { return u.toGiB(d); }
},
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

style:

GiB {
  public long toBytes(...) { return ... };
  public long convert(...) { return ... };
}

the new lines aren't super necessary


/** Tebibyte (TiB) = (1024^4) Byte */
TB {
public long toBytes(long d) { return x(d, C_TB); }
TiB {
public long toBytes(long d) { return x(d, C_TiB); }

public long convert(long d, ByteUnit u) { return u.toTB(d); }
public long convert(long d, ByteUnit u) { return u.toTiB(d); }
},

/** Pebibyte (PB) = (1024^5) Byte */
PB {
public long toBytes(long d) { return x(d, C_PB); }
PiB {
public long toBytes(long d) { return x(d, C_PiB); }

public long convert(long d, ByteUnit u) { return u.toPB(d); }
public long convert(long d, ByteUnit u) { return u.toPiB(d); }
};

static final long C_KB = 1024l;
static final long C_MB = (long) Math.pow(1024l, 2l);
static final long C_GB = (long) Math.pow(1024l, 3l);
static final long C_TB = (long) Math.pow(1024l, 4l);
static final long C_PB = (long) Math.pow(1024l, 5l);
static final long C_KiB = 1024l;
static final long C_MiB = (long) Math.pow(1024l, 2l);
static final long C_GiB = (long) Math.pow(1024l, 3l);
static final long C_TiB = (long) Math.pow(1024l, 4l);
static final long C_PiB = (long) Math.pow(1024l, 5l);

static final long MAX = Long.MAX_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not used anywhere?

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wanted to still keep these in case they were used down the line. Would you recommend getting rid of them?


Expand All @@ -78,19 +78,18 @@ static long x(long d, long m) {
if (d < -over) return Long.MIN_VALUE;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Negative byte counts sound a little weird, but well. Same comment as above, though.

return d * m;
}

public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This would be simpler if you had a "multiplier" in the constructor. e.g.

BYTE(1),
KiB(C_KiB),
...

private ByteUnit(long multiplier) {
  this.multiplier = multiplier;
}

long toBytes(long d) { return multiplier * d; }

long convert(long d, ByteUnit u) { return toBytes(d) / u.multiplier; }

long toKiB(long d) { return convert(d, KiB); }

Or something.


public long toBytes(long d) { throw new AbstractMethodError(); }
public long convert(long d, ByteUnit u) { throw new AbstractMethodError(); }

public long toKB(long d) { return toBytes(d) / C_KB; }
public long toKiB(long d) { return toBytes(d) / C_KiB; }

public long toMB(long d) { return toBytes(d) / C_MB; }
public long toMiB(long d) { return toBytes(d) / C_MiB; }

public long toGB(long d) { return toBytes(d) / C_GB; }
public long toGiB(long d) { return toBytes(d) / C_GiB; }

public long toTB(long d) { return toBytes(d) / C_TB; }
public long toTiB(long d) { return toBytes(d) / C_TiB; }

public long toPB(long d) { return toBytes(d) / C_PB; }


public long toPiB(long d) { return toBytes(d) / C_PiB; }
}
Loading