Skip to content

Commit 15d9534

Browse files
committed
Merge pull request apache#318 from srowen/master
Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance Sorry if this is too abrupt or not a welcome set of changes, but thought I'd see if I could contribute a little. I'm a Java developer and just getting seriously into Spark. So I thought I'd suggest a number of small changes to the couple Java parts of the code to make it a little tighter, more standard and even a bit faster. Feel free to take all, some or none of this. Happy to explain any of it.
2 parents 468af0f + 4b92a20 commit 15d9534

20 files changed

+174
-123
lines changed

core/src/main/java/org/apache/spark/network/netty/FileClient.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,15 +31,16 @@
3131

3232
class FileClient {
3333

34-
private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
34+
private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName());
35+
3536
private final FileClientHandler handler;
3637
private Channel channel = null;
3738
private Bootstrap bootstrap = null;
3839
private EventLoopGroup group = null;
3940
private final int connectTimeout;
4041
private final int sendTimeout = 60; // 1 min
4142

42-
public FileClient(FileClientHandler handler, int connectTimeout) {
43+
FileClient(FileClientHandler handler, int connectTimeout) {
4344
this.handler = handler;
4445
this.connectTimeout = connectTimeout;
4546
}

core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ class FileClientChannelInitializer extends ChannelInitializer<SocketChannel> {
2525

2626
private final FileClientHandler fhandler;
2727

28-
public FileClientChannelInitializer(FileClientHandler handler) {
28+
FileClientChannelInitializer(FileClientHandler handler) {
2929
fhandler = handler;
3030
}
3131

core/src/main/java/org/apache/spark/network/netty/FileServer.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -33,15 +33,14 @@
3333
*/
3434
class FileServer {
3535

36-
private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
36+
private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName());
3737

3838
private EventLoopGroup bossGroup = null;
3939
private EventLoopGroup workerGroup = null;
4040
private ChannelFuture channelFuture = null;
4141
private int port = 0;
42-
private Thread blockingThread = null;
4342

44-
public FileServer(PathResolver pResolver, int port) {
43+
FileServer(PathResolver pResolver, int port) {
4544
InetSocketAddress addr = new InetSocketAddress(port);
4645

4746
// Configure the server.
@@ -70,7 +69,8 @@ public FileServer(PathResolver pResolver, int port) {
7069
* Start the file server asynchronously in a new thread.
7170
*/
7271
public void start() {
73-
blockingThread = new Thread() {
72+
Thread blockingThread = new Thread() {
73+
@Override
7474
public void run() {
7575
try {
7676
channelFuture.channel().closeFuture().sync();

core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,9 +25,9 @@
2525

2626
class FileServerChannelInitializer extends ChannelInitializer<SocketChannel> {
2727

28-
PathResolver pResolver;
28+
private final PathResolver pResolver;
2929

30-
public FileServerChannelInitializer(PathResolver pResolver) {
30+
FileServerChannelInitializer(PathResolver pResolver) {
3131
this.pResolver = pResolver;
3232
}
3333

core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -31,11 +31,11 @@
3131

3232
class FileServerHandler extends SimpleChannelInboundHandler<String> {
3333

34-
private Logger LOG = LoggerFactory.getLogger(this.getClass().getName());
34+
private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName());
3535

3636
private final PathResolver pResolver;
3737

38-
public FileServerHandler(PathResolver pResolver){
38+
FileServerHandler(PathResolver pResolver){
3939
this.pResolver = pResolver;
4040
}
4141

@@ -61,7 +61,7 @@ public void channelRead0(ChannelHandlerContext ctx, String blockIdString) {
6161
ctx.flush();
6262
return;
6363
}
64-
int len = new Long(length).intValue();
64+
int len = (int) length;
6565
ctx.write((new FileHeader(len, blockId)).buffer());
6666
try {
6767
ctx.write(new DefaultFileRegion(new FileInputStream(file)
Lines changed: 26 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -1,26 +1,26 @@
1-
/*
2-
* Licensed to the Apache Software Foundation (ASF) under one or more
3-
* contributor license agreements. See the NOTICE file distributed with
4-
* this work for additional information regarding copyright ownership.
5-
* The ASF licenses this file to You under the Apache License, Version 2.0
6-
* (the "License"); you may not use this file except in compliance with
7-
* the License. You may obtain a copy of the License at
8-
*
9-
* http://www.apache.org/licenses/LICENSE-2.0
10-
*
11-
* Unless required by applicable law or agreed to in writing, software
12-
* distributed under the License is distributed on an "AS IS" BASIS,
13-
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14-
* See the License for the specific language governing permissions and
15-
* limitations under the License.
16-
*/
17-
18-
package org.apache.spark.network.netty;
19-
20-
import org.apache.spark.storage.BlockId;
21-
import org.apache.spark.storage.FileSegment;
22-
23-
public interface PathResolver {
24-
/** Get the file segment in which the given block resides. */
25-
public FileSegment getBlockLocation(BlockId blockId);
26-
}
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.network.netty;
19+
20+
import org.apache.spark.storage.BlockId;
21+
import org.apache.spark.storage.FileSegment;
22+
23+
public interface PathResolver {
24+
/** Get the file segment in which the given block resides. */
25+
FileSegment getBlockLocation(BlockId blockId);
26+
}

examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java

Lines changed: 16 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -24,19 +24,19 @@
2424

2525
import java.io.Serializable;
2626
import java.util.Arrays;
27-
import java.util.StringTokenizer;
2827
import java.util.Random;
28+
import java.util.regex.Pattern;
2929

3030
/**
3131
* Logistic regression based classification.
3232
*/
33-
public class JavaHdfsLR {
33+
public final class JavaHdfsLR {
3434

35-
static int D = 10; // Number of dimensions
36-
static Random rand = new Random(42);
35+
private static final int D = 10; // Number of dimensions
36+
private static final Random rand = new Random(42);
3737

3838
static class DataPoint implements Serializable {
39-
public DataPoint(double[] x, double y) {
39+
DataPoint(double[] x, double y) {
4040
this.x = x;
4141
this.y = y;
4242
}
@@ -46,20 +46,22 @@ public DataPoint(double[] x, double y) {
4646
}
4747

4848
static class ParsePoint extends Function<String, DataPoint> {
49+
private static final Pattern SPACE = Pattern.compile(" ");
50+
51+
@Override
4952
public DataPoint call(String line) {
50-
StringTokenizer tok = new StringTokenizer(line, " ");
51-
double y = Double.parseDouble(tok.nextToken());
53+
String[] tok = SPACE.split(line);
54+
double y = Double.parseDouble(tok[0]);
5255
double[] x = new double[D];
53-
int i = 0;
54-
while (i < D) {
55-
x[i] = Double.parseDouble(tok.nextToken());
56-
i += 1;
56+
for (int i = 0; i < D; i++) {
57+
x[i] = Double.parseDouble(tok[i + 1]);
5758
}
5859
return new DataPoint(x, y);
5960
}
6061
}
6162

6263
static class VectorSum extends Function2<double[], double[], double[]> {
64+
@Override
6365
public double[] call(double[] a, double[] b) {
6466
double[] result = new double[D];
6567
for (int j = 0; j < D; j++) {
@@ -70,12 +72,13 @@ public double[] call(double[] a, double[] b) {
7072
}
7173

7274
static class ComputeGradient extends Function<DataPoint, double[]> {
73-
double[] weights;
75+
private final double[] weights;
7476

75-
public ComputeGradient(double[] weights) {
77+
ComputeGradient(double[] weights) {
7678
this.weights = weights;
7779
}
7880

81+
@Override
7982
public double[] call(DataPoint p) {
8083
double[] gradient = new double[D];
8184
for (int i = 0; i < D; i++) {

examples/src/main/java/org/apache/spark/examples/JavaKMeans.java

Lines changed: 15 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -27,19 +27,24 @@
2727

2828
import java.util.List;
2929
import java.util.Map;
30+
import java.util.regex.Pattern;
3031

3132
/**
3233
* K-means clustering using Java API.
3334
*/
34-
public class JavaKMeans {
35+
public final class JavaKMeans {
36+
37+
private static final Pattern SPACE = Pattern.compile(" ");
3538

3639
/** Parses numbers split by whitespace to a vector */
3740
static Vector parseVector(String line) {
38-
String[] splits = line.split(" ");
41+
String[] splits = SPACE.split(line);
3942
double[] data = new double[splits.length];
4043
int i = 0;
41-
for (String s : splits)
42-
data[i] = Double.parseDouble(splits[i++]);
44+
for (String s : splits) {
45+
data[i] = Double.parseDouble(s);
46+
i++;
47+
}
4348
return new Vector(data);
4449
}
4550

@@ -82,7 +87,7 @@ public static void main(String[] args) throws Exception {
8287
JavaRDD<Vector> data = sc.textFile(path).map(
8388
new Function<String, Vector>() {
8489
@Override
85-
public Vector call(String line) throws Exception {
90+
public Vector call(String line) {
8691
return parseVector(line);
8792
}
8893
}
@@ -96,7 +101,7 @@ public Vector call(String line) throws Exception {
96101
JavaPairRDD<Integer, Vector> closest = data.map(
97102
new PairFunction<Vector, Integer, Vector>() {
98103
@Override
99-
public Tuple2<Integer, Vector> call(Vector vector) throws Exception {
104+
public Tuple2<Integer, Vector> call(Vector vector) {
100105
return new Tuple2<Integer, Vector>(
101106
closestPoint(vector, centroids), vector);
102107
}
@@ -107,7 +112,8 @@ public Tuple2<Integer, Vector> call(Vector vector) throws Exception {
107112
JavaPairRDD<Integer, List<Vector>> pointsGroup = closest.groupByKey();
108113
Map<Integer, Vector> newCentroids = pointsGroup.mapValues(
109114
new Function<List<Vector>, Vector>() {
110-
public Vector call(List<Vector> ps) throws Exception {
115+
@Override
116+
public Vector call(List<Vector> ps) {
111117
return average(ps);
112118
}
113119
}).collectAsMap();
@@ -122,8 +128,9 @@ public Vector call(List<Vector> ps) throws Exception {
122128
} while (tempDist > convergeDist);
123129

124130
System.out.println("Final centers:");
125-
for (Vector c : centroids)
131+
for (Vector c : centroids) {
126132
System.out.println(c);
133+
}
127134

128135
System.exit(0);
129136

examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -35,9 +35,9 @@
3535
/**
3636
* Executes a roll up-style query against Apache logs.
3737
*/
38-
public class JavaLogQuery {
38+
public final class JavaLogQuery {
3939

40-
public static List<String> exampleApacheLogs = Lists.newArrayList(
40+
public static final List<String> exampleApacheLogs = Lists.newArrayList(
4141
"10.10.10.10 - \"FRED\" [18/Jan/2013:17:56:07 +1100] \"GET http://images.com/2013/Generic.jpg " +
4242
"HTTP/1.1\" 304 315 \"http://referall.com/\" \"Mozilla/4.0 (compatible; MSIE 7.0; " +
4343
"Windows NT 5.1; GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; " +
@@ -51,14 +51,14 @@ public class JavaLogQuery {
5151
"3.5.30729; Release=ARP)\" \"UD-1\" - \"image/jpeg\" \"whatever\" 0.352 \"-\" - \"\" 256 977 988 \"\" " +
5252
"0 73.23.2.15 images.com 1358492557 - Whatup");
5353

54-
public static Pattern apacheLogRegex = Pattern.compile(
54+
public static final Pattern apacheLogRegex = Pattern.compile(
5555
"^([\\d.]+) (\\S+) (\\S+) \\[([\\w\\d:/]+\\s[+\\-]\\d{4})\\] \"(.+?)\" (\\d{3}) ([\\d\\-]+) \"([^\"]+)\" \"([^\"]+)\".*");
5656

5757
/** Tracks the total query count and number of aggregate bytes for a particular group. */
5858
public static class Stats implements Serializable {
5959

60-
private int count;
61-
private int numBytes;
60+
private final int count;
61+
private final int numBytes;
6262

6363
public Stats(int count, int numBytes) {
6464
this.count = count;
@@ -92,12 +92,12 @@ public static Stats extractStats(String line) {
9292
if (m.find()) {
9393
int bytes = Integer.parseInt(m.group(7));
9494
return new Stats(1, bytes);
95-
}
96-
else
95+
} else {
9796
return new Stats(1, 0);
97+
}
9898
}
9999

100-
public static void main(String[] args) throws Exception {
100+
public static void main(String[] args) {
101101
if (args.length == 0) {
102102
System.err.println("Usage: JavaLogQuery <master> [logFile]");
103103
System.exit(1);
@@ -110,14 +110,14 @@ public static void main(String[] args) throws Exception {
110110

111111
JavaPairRDD<Tuple3<String, String, String>, Stats> extracted = dataSet.map(new PairFunction<String, Tuple3<String, String, String>, Stats>() {
112112
@Override
113-
public Tuple2<Tuple3<String, String, String>, Stats> call(String s) throws Exception {
113+
public Tuple2<Tuple3<String, String, String>, Stats> call(String s) {
114114
return new Tuple2<Tuple3<String, String, String>, Stats>(extractKey(s), extractStats(s));
115115
}
116116
});
117117

118118
JavaPairRDD<Tuple3<String, String, String>, Stats> counts = extracted.reduceByKey(new Function2<Stats, Stats, Stats>() {
119119
@Override
120-
public Stats call(Stats stats, Stats stats2) throws Exception {
120+
public Stats call(Stats stats, Stats stats2) {
121121
return stats.merge(stats2);
122122
}
123123
});

examples/src/main/java/org/apache/spark/examples/JavaPageRank.java

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@
1717

1818
package org.apache.spark.examples;
1919

20-
import org.apache.spark.SparkContext;
2120
import scala.Tuple2;
2221
import org.apache.spark.api.java.JavaPairRDD;
2322
import org.apache.spark.api.java.JavaRDD;
@@ -29,6 +28,7 @@
2928

3029
import java.util.List;
3130
import java.util.ArrayList;
31+
import java.util.regex.Pattern;
3232

3333
/**
3434
* Computes the PageRank of URLs from an input file. Input file should
@@ -39,7 +39,9 @@
3939
* ...
4040
* where URL and their neighbors are separated by space(s).
4141
*/
42-
public class JavaPageRank {
42+
public final class JavaPageRank {
43+
private static final Pattern SPACES = Pattern.compile("\\s+");
44+
4345
private static class Sum extends Function2<Double, Double, Double> {
4446
@Override
4547
public Double call(Double a, Double b) {
@@ -67,15 +69,15 @@ public static void main(String[] args) throws Exception {
6769
JavaPairRDD<String, List<String>> links = lines.map(new PairFunction<String, String, String>() {
6870
@Override
6971
public Tuple2<String, String> call(String s) {
70-
String[] parts = s.split("\\s+");
72+
String[] parts = SPACES.split(s);
7173
return new Tuple2<String, String>(parts[0], parts[1]);
7274
}
7375
}).distinct().groupByKey().cache();
7476

7577
// Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one.
7678
JavaPairRDD<String, Double> ranks = links.mapValues(new Function<List<String>, Double>() {
7779
@Override
78-
public Double call(List<String> rs) throws Exception {
80+
public Double call(List<String> rs) {
7981
return 1.0;
8082
}
8183
});
@@ -98,7 +100,7 @@ public Iterable<Tuple2<String, Double>> call(Tuple2<List<String>, Double> s) {
98100
// Re-calculates URL ranks based on neighbor contributions.
99101
ranks = contribs.reduceByKey(new Sum()).mapValues(new Function<Double, Double>() {
100102
@Override
101-
public Double call(Double sum) throws Exception {
103+
public Double call(Double sum) {
102104
return 0.15 + sum * 0.85;
103105
}
104106
});

0 commit comments

Comments
 (0)