Skip to content

Conversation

@ilganeli
Copy link

I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933.

@ilganeli
Copy link
Author

Also will need to update memoryStringToMb to use the new interface.

@SparkQA
Copy link

SparkQA commented Apr 18, 2015

Test build #30520 has finished for PR 5574 at commit 09ea450.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

Copy link
Member

Choose a reason for hiding this comment

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

It's pedantic, but kind of matters. I think we're all used to these memory strings from the JVM -Xmx option and refer to -Xmx4g as "4 gigabytes" but it does not give a heap of 4 x 10^9 bytes; it gives 4 x 1024^3 bytes which is really "4 gibibytes".

I think we should stick to the same convention, but, at least in docs, let's be specific that these are treated as kibibytes, mebibytes, and gibibytes, technically. I don't think we have to fix other comments or talk differently, but a bit of precision here would be useful.

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 agree and in terms of the underlying math, everything is treated as kibi, gibi, etc. Which docstrings would you suggest updating? Just those for `asBytes or all the associated functions.

Copy link
Member

Choose a reason for hiding this comment

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

I think all of the docs you've added. It can be unobtrusive. "... converts to mebibytes (1024*1024 bytes) ..." I know this risks being confusing since most code and doc in the universe gets this technically wrong including some Spark usage, but I think it will be helpful and not annoying here.

Copy link
Author

Choose a reason for hiding this comment

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

Should we rename the enums to TiB, KiB etc? I'm personally against doing this, I think that WOULD confuse folks but it would be more consistent than it is now.

Copy link
Member

Choose a reason for hiding this comment

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

Tough call. I like correctness; I do think people aren't used to seeing KiB. They will probably figure it out quickly. I suppose it would help if later we needed to add support for real kilobytes somewhere, since KB isn't taken.

@SparkQA
Copy link

SparkQA commented Apr 18, 2015

Test build #30523 has finished for PR 5574 at commit 851d691.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

Ilya Ganelin added 2 commits April 18, 2015 12:53
…ed docs to clarify that we use kibi, mebi etc instead of kilo, mega
…kryoserializer.buffer.mb and spark.reducer.maxMbInFlight
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)

@SparkQA
Copy link

SparkQA commented Apr 18, 2015

Test build #30530 has finished for PR 5574 at commit 0cdff35.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

Copy link
Contributor

Choose a reason for hiding this comment

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

If you want to be really correct here, you could avoid overflows by playing with the multipliers instead of converting things to bytes first.

I think what's bugging me is that the semantics of all these methods are a little weird. It seems like you're trying to cap the maximum amount to be represented to Long.MAX_VALUE bytes (so that having Long.MAX_VALUE PB, for example, would be wrong since you can't convert that to bytes). I'm not sure that's needed, but if you want that, it should be enforced differently (and not here). Otherwise, I'd rework these methods to avoid overflows where possible, and throw exceptions when they would happen.

Copy link
Contributor

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 using double - I don't think that's a great idea because doubles lose precision as you try to work with values at different orders of magniture.

Regarding the last paragraph of my comment above, I don't think it's going to be an issue in practice; but the code here can be changed to at least avoid overflows where possible. I checked j.u.c.TimeUnit, used in the time functions in this class, and it seems to follow the approach you took, than when an overflow is inevitable it caps the value at Long.MAX_VALUE. So that part is fine.

…zes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes
@ilganeli
Copy link
Author

@vanzin Thanks for the continued feedback. I've revamped how I handle overflow and added test cases for various boundary conditions. I think this should address the prior concerns.

@SparkQA
Copy link

SparkQA commented Apr 23, 2015

Test build #30878 has finished for PR 5574 at commit 852a407.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

Copy link
Contributor

Choose a reason for hiding this comment

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

As I mentioned, I do not think that using double is the correct thing. Not only the API looks weird (toBytes returns double while the others return long), but doubles do not have infinite precision.

Instead, for example, have this:

public long convertFrom(long d, ByteUnit u) {
  return u.convertTo(d, this);
}

public long convertTo(long d, ByteUnit u) {
  if (multiplier > u.multiplier) {
    long ratio = multiplier / u.multiplier;
    if (Long.MAX_VALUE / ratio < d) {
      throw new IllegalArgumentException("OVERFLOW!");
    }
    return d * ratio;
  } else {
    return d / (u.multiplier / multiplier);
  }
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Just to illustrate my concern, these are the kinds of things you run into when working with floating-point numbers:

scala> (1E15).asInstanceOf[Long] == (1E15+1).asInstanceOf[Long]
res0: Boolean = false

scala> (1E16).asInstanceOf[Long] == (1E16+1).asInstanceOf[Long]
res1: Boolean = true

@SparkQA
Copy link

SparkQA commented Apr 24, 2015

Test build #30887 has finished for PR 5574 at commit fc85733.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@SparkQA
Copy link

SparkQA commented Apr 24, 2015

Test build #30888 has finished for PR 5574 at commit 2ab886b.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@SparkQA
Copy link

SparkQA commented Apr 24, 2015

Test build #30889 has finished for PR 5574 at commit 49a8720.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.
  • This patch does not change any dependencies.

@ilganeli
Copy link
Author

@andrewor14 what do you think about this in its present state? I think it should be getting near done.

Copy link
Contributor

Choose a reason for hiding this comment

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

super nit: you could drop the JavaUtils. prefix in the call.

@vanzin
Copy link
Contributor

vanzin commented Apr 24, 2015

Latest version looks good, thanks for addressing all of the feedback. Just a few nits remaining.

@SparkQA
Copy link

SparkQA commented Apr 24, 2015

Test build #30946 has started for PR 5574 at commit 11f6999.

@ilganeli
Copy link
Author

retest this please

@SparkQA
Copy link

SparkQA commented Apr 27, 2015

Test build #31003 has started for PR 5574 at commit 11f6999.

@ilganeli
Copy link
Author

@andrewor14 How does this look?

@andrewor14
Copy link
Contributor

Hi @ilganeli sorry for slipping on this. I've been busy with my own patches and I will look at this right now.

@andrewor14
Copy link
Contributor

LGTM I have merged this into master thanks @vanzin @ilganeli @srowen

@asfgit asfgit closed this in 2d222fb Apr 28, 2015
jeanlyn pushed a commit to jeanlyn/spark that referenced this pull request May 14, 2015
I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933.

Author: Ilya Ganelin <[email protected]>

Closes apache#5574 from ilganeli/SPARK-5932 and squashes the following commits:

11f6999 [Ilya Ganelin] Nit fixes
49a8720 [Ilya Ganelin] Whitespace fix
2ab886b [Ilya Ganelin] Scala style
fc85733 [Ilya Ganelin] Got rid of floating point math
852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes
9ee779c [Ilya Ganelin] Simplified fraction matches
22413b1 [Ilya Ganelin] Made MAX private
3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values.
e428049 [Ilya Ganelin] resolving merge conflict
8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles
84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer
d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer
fe286b4 [Ilya Ganelin] Resolved merge conflict
c7803cd [Ilya Ganelin] Empty lines
54b78b4 [Ilya Ganelin] Simplified byteUnit class
69e2f20 [Ilya Ganelin] Updates to code
f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion
f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size
0f4443e [Ilya Ganelin]     Merge remote-tracking branch 'upstream/master' into SPARK-5932
35a7fa7 [Ilya Ganelin] Minor formatting
928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints
5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates
7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer
afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold
ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size
2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes
1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize
eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb
b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max
0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight
475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega
851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces
a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion
747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion
09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils
5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932
db9a963 [Ilya Ganelin] Closing second spark context
1dc0444 [Ilya Ganelin] Added ref equality check
8c884fa [Ilya Ganelin] Made getOrCreate synchronized
cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup
270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes
15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude
0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference
dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference
733ec9f [Ilya Ganelin] Fixed some bugs in test code
8be2f83 [Ilya Ganelin] Replaced match with if
e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired
a99032f [Ilya Ganelin] Spacing fix
d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
nemccarthy pushed a commit to nemccarthy/spark that referenced this pull request Jun 19, 2015
I've added an interface to JavaUtils to do byte conversion and added hooks within Utils.scala to handle conversion within Spark code (like for time strings). I've added matching tests for size conversion, and then updated all deprecated configs and documentation as per SPARK-5933.

Author: Ilya Ganelin <[email protected]>

Closes apache#5574 from ilganeli/SPARK-5932 and squashes the following commits:

11f6999 [Ilya Ganelin] Nit fixes
49a8720 [Ilya Ganelin] Whitespace fix
2ab886b [Ilya Ganelin] Scala style
fc85733 [Ilya Ganelin] Got rid of floating point math
852a407 [Ilya Ganelin] [SPARK-5932] Added much improved overflow handling. Can now handle sizes up to Long.MAX_VALUE Petabytes instead of being capped at Long.MAX_VALUE Bytes
9ee779c [Ilya Ganelin] Simplified fraction matches
22413b1 [Ilya Ganelin] Made MAX private
3dfae96 [Ilya Ganelin] Fixed some nits. Added automatic conversion of old paramter for kryoserializer.mb to new values.
e428049 [Ilya Ganelin] resolving merge conflict
8b43748 [Ilya Ganelin] Fixed error in pattern matching for doubles
84a2581 [Ilya Ganelin] Added smoother handling of fractional values for size parameters. This now throws an exception and added a warning for old spark.kryoserializer.buffer
d3d09b6 [Ilya Ganelin] [SPARK-5932] Fixing error in KryoSerializer
fe286b4 [Ilya Ganelin] Resolved merge conflict
c7803cd [Ilya Ganelin] Empty lines
54b78b4 [Ilya Ganelin] Simplified byteUnit class
69e2f20 [Ilya Ganelin] Updates to code
f32bc01 [Ilya Ganelin] [SPARK-5932] Fixed error in API in SparkConf.scala where Kb conversion wasn't being done properly (was Mb). Added test cases for both timeUnit and ByteUnit conversion
f15f209 [Ilya Ganelin] Fixed conversion of kryo buffer size
0f4443e [Ilya Ganelin]     Merge remote-tracking branch 'upstream/master' into SPARK-5932
35a7fa7 [Ilya Ganelin] Minor formatting
928469e [Ilya Ganelin] [SPARK-5932] Converted some longs to ints
5d29f90 [Ilya Ganelin] [SPARK-5932] Finished documentation updates
7a6c847 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer
afc9a38 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize and spark.storage.memoryMapThreshold
ae7e9f6 [Ilya Ganelin] [SPARK-5932] Updated spark.io.compression.snappy.block.size
2d15681 [Ilya Ganelin] [SPARK-5932] Updated spark.executor.logs.rolling.size.maxBytes
1fbd435 [Ilya Ganelin] [SPARK-5932] Updated spark.broadcast.blockSize
eba4de6 [Ilya Ganelin] [SPARK-5932] Updated spark.shuffle.file.buffer.kb
b809a78 [Ilya Ganelin] [SPARK-5932] Updated spark.kryoserializer.buffer.max
0cdff35 [Ilya Ganelin] [SPARK-5932] Updated to use bibibytes in method names. Updated spark.kryoserializer.buffer.mb and spark.reducer.maxMbInFlight
475370a [Ilya Ganelin] [SPARK-5932] Simplified ByteUnit code, switched to using longs. Updated docs to clarify that we use kibi, mebi etc instead of kilo, mega
851d691 [Ilya Ganelin] [SPARK-5932] Updated memoryStringToMb to use new interfaces
a9f4fcf [Ilya Ganelin] [SPARK-5932] Added unit tests for unit conversion
747393a [Ilya Ganelin] [SPARK-5932] Added unit tests for ByteString conversion
09ea450 [Ilya Ganelin] [SPARK-5932] Added byte string conversion to Jav utils
5390fd9 [Ilya Ganelin] Merge remote-tracking branch 'upstream/master' into SPARK-5932
db9a963 [Ilya Ganelin] Closing second spark context
1dc0444 [Ilya Ganelin] Added ref equality check
8c884fa [Ilya Ganelin] Made getOrCreate synchronized
cb0c6b7 [Ilya Ganelin] Doc updates and code cleanup
270cfe3 [Ilya Ganelin] [SPARK-6703] Documentation fixes
15e8dea [Ilya Ganelin] Updated comments and added MiMa Exclude
0e1567c [Ilya Ganelin] Got rid of unecessary option for AtomicReference
dfec4da [Ilya Ganelin] Changed activeContext to AtomicReference
733ec9f [Ilya Ganelin] Fixed some bugs in test code
8be2f83 [Ilya Ganelin] Replaced match with if
e92caf7 [Ilya Ganelin] [SPARK-6703] Added test to ensure that getOrCreate both allows creation, retrieval, and a second context if desired
a99032f [Ilya Ganelin] Spacing fix
d7a06b8 [Ilya Ganelin] Updated SparkConf class to add getOrCreate method. Started test suite implementation
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants