Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@

public enum ByteUnit {
BYTE(1),
KiB(1024L),
MiB((long) Math.pow(1024L, 2L)),
GiB((long) Math.pow(1024L, 3L)),
TiB((long) Math.pow(1024L, 4L)),
PiB((long) Math.pow(1024L, 5L));
KiB(1L << 10),
MiB(1L << 20),
GiB(1L << 30),
TiB(1L << 40),
PiB(1L << 50);

ByteUnit(long multiplier) {
this.multiplier = multiplier;
Expand Down Expand Up @@ -50,7 +50,7 @@ public long convertTo(long d, ByteUnit u) {
}
}

public double toBytes(long d) {
public long toBytes(long d) {
if (d < 0) {
throw new IllegalArgumentException("Negative size value. Size must be positive: " + d);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -309,8 +309,8 @@ public int chunkFetchHandlerThreads() {
}
int chunkFetchHandlerThreadsPercent =
conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100);
return (int)Math.ceil(
(this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) *
chunkFetchHandlerThreadsPercent/(double)100);
int threads =
this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors();
return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.0));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -712,7 +712,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff
final long recordOffset = offset;
UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize);
UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen);
offset += (2 * uaoSize);
offset += (2L * uaoSize);
Platform.copyMemory(kbase, koff, base, offset, klen);
offset += klen;
Platform.copyMemory(vbase, voff, base, offset, vlen);
Expand Down Expand Up @@ -780,7 +780,7 @@ private void allocate(int capacity) {
assert (capacity >= 0);
capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64);
assert (capacity <= MAX_CAPACITY);
longArray = allocateArray(capacity * 2);
longArray = allocateArray(capacity * 2L);
longArray.zeroOut();

this.growthThreshold = (int) (capacity * loadFactor);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public static void main(String[] args) {

JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext());

Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2;
int slices = (args.length > 0) ? Integer.parseInt(args[0]): 2;
JavaPairRDD<Integer, Integer> tc = jsc.parallelizePairs(generateGraph(), slices).cache();

// Linear transitive closure: each round grows paths by one edge,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ public static void main(String[] args) {
.setMetricName("rmse")
.setLabelCol("rating")
.setPredictionCol("prediction");
Double rmse = evaluator.evaluate(predictions);
double rmse = evaluator.evaluate(predictions);
System.out.println("Root-mean-square error = " + rmse);

// Generate top 10 movie recommendations for each user
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public static void main(String[] args) {

// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method.
// If a method is not specified, Pearson's method will be used by default.
Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson");
double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson");
System.out.println("Correlation is: " + correlation);

// note that each Vector is a row and not a column
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,14 +48,14 @@ public static void main(String[] args) {

// Train a RandomForest model.
// Empty categoricalFeaturesInfo indicates all features are continuous.
Integer numClasses = 2;
int numClasses = 2;
Map<Integer, Integer> categoricalFeaturesInfo = new HashMap<>();
Integer numTrees = 3; // Use more in practice.
String featureSubsetStrategy = "auto"; // Let the algorithm choose.
String impurity = "gini";
Integer maxDepth = 5;
Integer maxBins = 32;
Integer seed = 12345;
int maxDepth = 5;
int maxBins = 32;
int seed = 12345;

RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses,
categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -318,18 +318,17 @@ protected void handle(Message msg) throws IOException {
throw new IllegalArgumentException("Received Hello for unknown client.");
}
} else {
String msgClassName = msg != null ? msg.getClass().getName() : "no message";
if (handle == null) {
throw new IllegalArgumentException("Expected hello, got: " +
msg != null ? msg.getClass().getName() : null);
throw new IllegalArgumentException("Expected hello, got: " + msgClassName);
}
if (msg instanceof SetAppId) {
SetAppId set = (SetAppId) msg;
handle.setAppId(set.appId);
} else if (msg instanceof SetState) {
handle.setState(((SetState)msg).state);
} else {
throw new IllegalArgumentException("Invalid message: " +
msg != null ? msg.getClass().getName() : null);
throw new IllegalArgumentException("Invalid message: " + msgClassName);
}
}
} catch (Exception e) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ public void initialize(int numElements) {
}

private long getElementOffset(int ordinal) {
return startingOffset + headerInBytes + ordinal * elementSize;
return startingOffset + headerInBytes + ordinal * (long) elementSize;
}

private void setNullBit(int ordinal) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ public void setNull8Bytes(int ordinal) {
}

public long getFieldOffset(int ordinal) {
return startingOffset + nullBitsSize + 8 * ordinal;
return startingOffset + nullBitsSize + 8L * ordinal;
}

public void write(int ordinal, boolean value) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ public long skip(long ns) throws IOException {
return 0;
}
// Bound skip by beginning and end of the source
long n = Math.min(length - next, ns);
int n = (int) Math.min(length - next, ns);
n = Math.max(-next, n);
next += n;
return n;
Expand Down