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] Added byte string conversion to Jav utils
  • Loading branch information
Ilya Ganelin committed Apr 18, 2015
commit 09ea450fd5dfa8849c71dbd7420e955d9c22887f
Original file line number Diff line number Diff line change
@@ -0,0 +1,172 @@
/*
* 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.network.util;

/**
* Code based on https://github.com/fabian-barney/Utils
Copy link
Member

Choose a reason for hiding this comment

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

We have to be a little careful if importing third-party code, especially source. Looks like this is AL2 licensed: https://github.com/fabian-barney/Utils/blob/master/utils/LICENSE And it has no NOTICE file. So should be OK.

However, you can't remove his copyright statement. If you also added changes or new code, you can also include the ASF header to cover your work, but should retain his too.

However, I'm not sure this is the best code to reuse, and I base that almost entirely on the fact that it treats numbers of bytes as doubles which seems wrong. This feels like something that doesn't need to be this complex, so could be reimplemented easily.

Copy link
Author

Choose a reason for hiding this comment

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

Sean - I think his primary reason for using doubles was to allow handling of larger byte sizes. On second thought though, using long allows us up to 8 Exabytes which should be more than we ever need.

I'll rewrite it from scratch to avoid the copyright issues and to fix this.

* Copyright 2011 Fabian Barney
*
* @author Fabian Barney
*/
public enum ByteUnit {
/**
* <pre>
* Byte (B)
* 1 Byte
*/
BYTE {
@Override
public double toBytes(double d) {
return d;
}

@Override
public double convert(double d, ByteUnit u) {
return u.toBytes(d);
}
},

/**
* <pre>
* Kilobyte (kB)
* 10^3 Byte = 1.000 Byte
*/
KB {
@Override
public double toBytes(double d) {
return safeMulti(d, C_KB);
}

@Override
public double convert(double d, ByteUnit u) {
return u.toKB(d);
}
},

/**
* <pre>
* Megabyte (MB)
* 10^6 Byte = 1.000.000 Byte
*/
MB {
@Override
public double toBytes(double d) {
return safeMulti(d, C_MB);
}

@Override
public double convert(double d, ByteUnit u) {
return u.toMB(d);
}
},

/**
* <pre>
* Gigabyte (GB)
* 10^9 Byte = 1.000.000.000 Byte
*/
GB {
@Override
public double toBytes(double d) {
return safeMulti(d, C_GB);
}

@Override
public double convert(double d, ByteUnit u) {
return u.toGB(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


/**
* <pre>
* Terabyte (TB)
* 10^12 Byte = 1.000.000.000.000 Byte
*/
TB {
@Override
public double toBytes(double d) {
return safeMulti(d, C_TB);
}

@Override
public double convert(double d, ByteUnit u) {
return u.toTB(d);
}
},

/**
* <pre>
* Petabyte (PB)
* 10^15 Byte = 1.000.000.000.000.000 Byte
*/
PB {
@Override
public double toBytes(double d) {
return safeMulti(d, C_PB);
}

@Override
public double convert(double d, ByteUnit u) {
return u.toPB(d);
}
};

static final double C_KB = Math.pow(10d, 3d);
static final double C_MB = Math.pow(10d, 6d);
static final double C_GB = Math.pow(10d, 9d);
static final double C_TB = Math.pow(10d, 12d);
static final double C_PB = Math.pow(10d, 15d);

private static final double MAX = Double.MAX_VALUE;

static double safeMulti(double d, double multi) {
double limit = MAX / multi;

if (d > limit) {
return Double.MAX_VALUE;
}
if (d < -limit) {
return Double.MIN_VALUE;
}

return d * multi;
}

public abstract double toBytes(double d);

public final double toKB(double d) {
return toBytes(d) / C_KB;
}

public final double toMB(double d) {
return toBytes(d) / C_MB;
}

public final double toGB(double d) {
return toBytes(d) / C_GB;
}

public final double toTB(double d) {
return toBytes(d) / C_TB;
}

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

public abstract double convert(double d, ByteUnit u);
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,12 +22,14 @@
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.TimeUnit;

import java.util.regex.Matcher;
import java.util.regex.Pattern;

import com.google.common.base.Charsets;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import com.sun.javafx.css.SizeUnits;
import io.netty.buffer.Unpooled;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -137,6 +139,16 @@ private static boolean isSymlink(File file) throws IOException {
.put("d", TimeUnit.DAYS)
.build();

private static ImmutableMap<String, ByteUnit> byteSuffixes =
Copy link
Contributor

Choose a reason for hiding this comment

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

final

ImmutableMap.<String, ByteUnit>builder()
.put("b", ByteUnit.BYTE)
.put("kb", ByteUnit.KB)
.put("mb", ByteUnit.MB)
.put("gb", ByteUnit.GB)
.put("tb", ByteUnit.TB)
.put("pb", ByteUnit.PB)
.build();

/**
* Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for
* internal use. If no suffix is provided a direct conversion is attempted.
Expand Down Expand Up @@ -164,7 +176,7 @@ private static long parseTimeString(String str, TimeUnit unit) {
return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit);
} catch (NumberFormatException e) {
String timeError = "Time must be specified as seconds (s), " +
"milliseconds (ms), microseconds (us), minutes (m or min) hour (h), or day (d). " +
"milliseconds (ms), microseconds (us), minutes (m or min), hour (h), or day (d). " +
"E.g. 50s, 100ms, or 250us.";

throw new NumberFormatException(timeError + "\n" + e.getMessage());
Expand All @@ -186,5 +198,81 @@ public static long timeStringAsMs(String str) {
public static long timeStringAsSec(String str) {
return parseTimeString(str, TimeUnit.SECONDS);
}

/**
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to a ByteUnit for
* internal use. If no suffix is provided a direct conversion of the provided default is
* attempted.
*/
private static long parseByteString(String str, ByteUnit unit) {
String lower = str.toLowerCase().trim();

try {
String suffix;
long val;
Matcher m = Pattern.compile("([0-9]+)([a-z]+)?").matcher(lower);
if (m.matches()) {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit:

if (!m.matches()) {
  throw...
}

String suffix = ...;
long val = ...;

val = Long.parseLong(m.group(1));
suffix = m.group(2);
} else {
throw new NumberFormatException("Failed to parse byte string: " + str);
}

// Check for invalid suffixes
if (suffix != null && !byteSuffixes.containsKey(suffix)) {
throw new NumberFormatException("Invalid suffix: \"" + suffix + "\"");
}

// If suffix is valid use that, otherwise none was provided and use the default passed
return (long) unit.convert(val, suffix != null ? byteSuffixes.get(suffix) : unit);
} catch (NumberFormatException e) {
String timeError = "Size must be specified as bytes (b), " +
"kilobytes (kb), megabytes (mb), gigabytes (gb), terabytes (tb), or petabytes(pb). " +
"E.g. 50b, 100kb, or 250mb.";

throw new NumberFormatException(timeError + "\n" + e.getMessage());
}
}

/**
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to bytes for
* internal use.
*
* If no suffix is provided, the passed number is assumed to be in bytes.
*/
public static long byteStringAsBytes(String str) {
return parseByteString(str, ByteUnit.BYTE);
}

/**
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to kilobytes for
* internal use.
*
* If no suffix is provided, the passed number is assumed to be in kilobytes.
*/
public static long byteStringAsKB(String str) {
return parseByteString(str, ByteUnit.KB);
}

/**
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to megabytes for
* internal use.
*
* If no suffix is provided, the passed number is assumed to be in megabytes.
*/
public static long byteStringAsMB(String str) {
return parseByteString(str, ByteUnit.MB);
}

/**
* Convert a passed byte string (e.g. 50b, 100kb, or 250mb) to gigabytes for
* internal use.
*
* If no suffix is provided, the passed number is assumed to be in gigabytes.
*/
public static long byteStringAsGB(String str) {
return parseByteString(str, ByteUnit.GB);
}


}