Skip to content

Commit

Permalink
[hotfix] [gelly] Improve generic type formatting
Browse files Browse the repository at this point in the history
Add spacing between type parameters.

For example, 'Vertex<K,VV>' has been updated to 'Vertex<K, VV>'.
  • Loading branch information
greghogan committed Jan 17, 2017
1 parent b408d61 commit 53716a4
Show file tree
Hide file tree
Showing 55 changed files with 332 additions and 331 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ public static void main(String[] args) throws Exception {
graph = graph.run(new Simplify<LongValue, NullValue, NullValue>(clipAndFlip));
}

DataSet<Tuple2<LongValue,LongValue>> edges = graph
DataSet<Tuple2<LongValue, LongValue>> edges = graph
.getEdges()
.project(0, 1);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,11 +53,11 @@ public static DataSet<Edge<Long, NullValue>> getDefaultEdgeDataSet(ExecutionEnvi

public static final String RESULTED_NUMBER_OF_TRIANGLES = "3";

public static List<Tuple3<Long,Long,Long>> getListOfTriangles() {
ArrayList<Tuple3<Long,Long,Long>> ret = new ArrayList<>(3);
ret.add(new Tuple3<>(1L,2L,3L));
ret.add(new Tuple3<>(2L,3L,6L));
ret.add(new Tuple3<>(4L,3L,5L));
public static List<Tuple3<Long, Long, Long>> getListOfTriangles() {
ArrayList<Tuple3<Long, Long, Long>> ret = new ArrayList<>(3);
ret.add(new Tuple3<>(1L, 2L, 3L));
ret.add(new Tuple3<>(2L, 3L, 6L));
ret.add(new Tuple3<>(4L, 3L, 5L));
return ret;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,11 +46,11 @@ public void testTriangleEnumerator() throws Exception {
Graph<Long, NullValue, NullValue> graph = Graph.fromDataSet(TriangleCountData.getDefaultEdgeDataSet(env),
env);

List<Tuple3<Long,Long,Long>> actualOutput = graph.run(new TriangleEnumerator<Long, NullValue, NullValue>()).collect();
List<Tuple3<Long,Long,Long>> expectedResult = TriangleCountData.getListOfTriangles();
List<Tuple3<Long, Long, Long>> actualOutput = graph.run(new TriangleEnumerator<Long, NullValue, NullValue>()).collect();
List<Tuple3<Long, Long, Long>> expectedResult = TriangleCountData.getListOfTriangles();

Assert.assertEquals(expectedResult.size(), actualOutput.size());
for(Tuple3<Long,Long,Long> resultTriangle:actualOutput) {
for(Tuple3<Long, Long, Long> resultTriangle:actualOutput) {
Assert.assertTrue(expectedResult.indexOf(resultTriangle)>=0);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -82,7 +82,7 @@ public void before() throws Exception {
public void testIncrementalSSSP() throws Exception {
IncrementalSSSP.main(new String[]{verticesPath, edgesPath, edgesInSSSPPath,
IncrementalSSSPData.SRC_EDGE_TO_BE_REMOVED, IncrementalSSSPData.TRG_EDGE_TO_BE_REMOVED,
IncrementalSSSPData.VAL_EDGE_TO_BE_REMOVED,resultPath, IncrementalSSSPData.NUM_VERTICES + ""});
IncrementalSSSPData.VAL_EDGE_TO_BE_REMOVED, resultPath, IncrementalSSSPData.NUM_VERTICES + ""});
expected = IncrementalSSSPData.RESULTED_VERTICES;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -544,7 +544,7 @@ public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> ma
* @param returnType the explicit return type.
* @return a new graph
*/
public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper, TypeInformation<Vertex<K,NV>> returnType) {
public <NV> Graph<K, NV, EV> mapVertices(final MapFunction<Vertex<K, VV>, NV> mapper, TypeInformation<Vertex<K, NV>> returnType) {
DataSet<Vertex<K, NV>> mappedVertices = vertices.map(
new MapFunction<Vertex<K, VV>, Vertex<K, NV>>() {
private Vertex<K, NV> output = new Vertex<>();
Expand Down Expand Up @@ -588,7 +588,7 @@ public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper)
* @param returnType the explicit return type.
* @return a new graph
*/
public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper, TypeInformation<Edge<K,NV>> returnType) {
public <NV> Graph<K, VV, NV> mapEdges(final MapFunction<Edge<K, EV>, NV> mapper, TypeInformation<Edge<K, NV>> returnType) {
DataSet<Edge<K, NV>> mappedEdges = edges.map(
new MapFunction<Edge<K, EV>, Edge<K, NV>>() {
private Edge<K, NV> output = new Edge<>();
Expand Down Expand Up @@ -924,7 +924,7 @@ private static final class CountNeighborsCoGroup<K, VV, EV>
private Tuple2<K, LongValue> vertexDegree = new Tuple2<>(null, degree);

@SuppressWarnings("unused")
public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Edge<K, EV>> outEdges,
public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Edge<K, EV>> outEdges,
Collector<Tuple2<K, LongValue>> out) {
long count = 0;
for (Edge<K, EV> edge : outEdges) {
Expand Down Expand Up @@ -1217,7 +1217,7 @@ public ApplyCoGroupFunctionOnAllEdges(EdgesFunctionWithVertexValue<K, VV, EV, T>
this.function = fun;
}

public void coGroup(Iterable<Vertex<K, VV>> vertex, final Iterable<Tuple2<K, Edge<K, EV>>> keysWithEdges,
public void coGroup(Iterable<Vertex<K, VV>> vertex, final Iterable<Tuple2<K, Edge<K, EV>>> keysWithEdges,
Collector<T> out) throws Exception {

final Iterator<Edge<K, EV>> edgesIterator = new Iterator<Edge<K, EV>>() {
Expand Down Expand Up @@ -1416,9 +1416,9 @@ public Graph<K, VV, EV> addEdge(Vertex<K, VV> source, Vertex<K, VV> target, EV e
*/
public Graph<K, VV, EV> addEdges(List<Edge<K, EV>> newEdges) {

DataSet<Edge<K,EV>> newEdgesDataSet = this.context.fromCollection(newEdges);
DataSet<Edge<K, EV>> newEdgesDataSet = this.context.fromCollection(newEdges);

DataSet<Edge<K,EV>> validNewEdges = this.getVertices().join(newEdgesDataSet)
DataSet<Edge<K, EV>> validNewEdges = this.getVertices().join(newEdgesDataSet)
.where(0).equalTo(0)
.with(new JoinVerticesWithEdgesOnSrc<K, VV, EV>()).name("Join with source")
.join(this.getVertices()).where(1).equalTo(0)
Expand Down Expand Up @@ -1516,7 +1516,7 @@ public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Vertex<K, VV>> vert
}

@ForwardedFieldsSecond("f0; f1; f2")
private static final class ProjectEdgeToBeRemoved<K,VV,EV> implements JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
private static final class ProjectEdgeToBeRemoved<K, VV, EV> implements JoinFunction<Vertex<K, VV>, Edge<K, EV>, Edge<K, EV>> {
@Override
public Edge<K, EV> join(Vertex<K, VV> vertex, Edge<K, EV> edge) throws Exception {
return edge;
Expand Down Expand Up @@ -1559,12 +1559,12 @@ public boolean filter(Edge<K, EV> edge) {
public Graph<K, VV, EV> removeEdges(List<Edge<K, EV>> edgesToBeRemoved) {

DataSet<Edge<K, EV>> newEdges = getEdges().coGroup(this.context.fromCollection(edgesToBeRemoved))
.where(0,1).equalTo(0,1).with(new EdgeRemovalCoGroup<K, EV>()).name("Remove edges");
.where(0, 1).equalTo(0, 1).with(new EdgeRemovalCoGroup<K, EV>()).name("Remove edges");

return new Graph<>(this.vertices, newEdges, context);
}

private static final class EdgeRemovalCoGroup<K,EV> implements CoGroupFunction<Edge<K, EV>, Edge<K, EV>, Edge<K, EV>> {
private static final class EdgeRemovalCoGroup<K, EV> implements CoGroupFunction<Edge<K, EV>, Edge<K, EV>, Edge<K, EV>> {

@Override
public void coGroup(Iterable<Edge<K, EV>> edge, Iterable<Edge<K, EV>> edgeToBeRemoved,
Expand Down Expand Up @@ -1608,8 +1608,8 @@ public Graph<K, VV, EV> union(Graph<K, VV, EV> graph) {
* @param graph the graph to perform difference with
* @return a new graph where the common vertices and edges have been removed
*/
public Graph<K,VV,EV> difference(Graph<K,VV,EV> graph) {
DataSet<Vertex<K,VV>> removeVerticesData = graph.getVertices();
public Graph<K, VV, EV> difference(Graph<K, VV, EV> graph) {
DataSet<Vertex<K, VV>> removeVerticesData = graph.getVertices();
return this.removeVertices(removeVerticesData);
}

Expand Down Expand Up @@ -1688,7 +1688,7 @@ private DataSet<Edge<K, EV>> getPairwiseEdgeIntersection(DataSet<Edge<K, EV>> ed
* @param <EV> edge value type
*/
private static final class MatchingEdgeReducer<K, EV>
implements CoGroupFunction<Edge<K,EV>, Edge<K,EV>, Edge<K, EV>> {
implements CoGroupFunction<Edge<K, EV>, Edge<K, EV>, Edge<K, EV>> {

@Override
public void coGroup(Iterable<Edge<K, EV>> edgesLeft, Iterable<Edge<K, EV>> edgesRight, Collector<Edge<K, EV>> out)
Expand Down Expand Up @@ -2149,12 +2149,12 @@ public ApplyNeighborCoGroupFunction(NeighborsFunctionWithVertexValue<K, VV, EV,

public void coGroup(Iterable<Vertex<K, VV>> vertex, Iterable<Tuple2<Edge<K, EV>, Vertex<K, VV>>> neighbors,
Collector<T> out) throws Exception {
function.iterateNeighbors(vertex.iterator().next(), neighbors, out);
function.iterateNeighbors(vertex.iterator().next(), neighbors, out);
}

@Override
public TypeInformation<T> getProducedType() {
return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null);
return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null);
}
}

Expand Down Expand Up @@ -2209,7 +2209,7 @@ public Iterator<Tuple2<Edge<K, EV>, Vertex<K, VV>>> iterator() {

@Override
public TypeInformation<T> getProducedType() {
return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null);
return TypeExtractor.createTypeInfo(NeighborsFunctionWithVertexValue.class, function.getClass(), 3, null, null);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ public class TranslateEdgeValues<K, VV, OLD, NEW>
extends GraphAlgorithmWrappingGraph<K, VV, OLD, K, VV, NEW> {

// Required configuration
private TranslateFunction<OLD,NEW> translator;
private TranslateFunction<OLD, NEW> translator;

// Optional configuration
private int parallelism = PARALLELISM_DEFAULT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ public class TranslateGraphIds<OLD, NEW, VV, EV>
extends GraphAlgorithmWrappingGraph<OLD, VV, EV, NEW, VV, EV> {

// Required configuration
private TranslateFunction<OLD,NEW> translator;
private TranslateFunction<OLD, NEW> translator;

// Optional configuration
private int parallelism = PARALLELISM_DEFAULT;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ public abstract class AbstractGraphGenerator<K, VV, EV>
protected int parallelism = PARALLELISM_DEFAULT;

@Override
public GraphGenerator<K,VV,EV> setParallelism(int parallelism) {
public GraphGenerator<K, VV, EV> setParallelism(int parallelism) {
this.parallelism = parallelism;

return this;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,14 +58,14 @@ public CompleteGraph(ExecutionEnvironment env, long vertexCount) {
}

@Override
public Graph<LongValue,NullValue,NullValue> generate() {
public Graph<LongValue, NullValue, NullValue> generate() {
// Vertices
DataSet<Vertex<LongValue,NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount);
DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount);

// Edges
LongValueSequenceIterator iterator = new LongValueSequenceIterator(0, this.vertexCount - 1);

DataSet<Edge<LongValue,NullValue>> edges = env
DataSet<Edge<LongValue, NullValue>> edges = env
.fromParallelCollection(iterator, LongValue.class)
.setParallelism(parallelism)
.name("Edge iterators")
Expand All @@ -79,20 +79,20 @@ public Graph<LongValue,NullValue,NullValue> generate() {

@ForwardedFields("*->f0")
public class LinkVertexToAll
implements FlatMapFunction<LongValue, Edge<LongValue,NullValue>> {
implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {

private final long vertexCount;

private LongValue target = new LongValue();

private Edge<LongValue,NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());

public LinkVertexToAll(long vertex_count) {
this.vertexCount = vertex_count;
}

@Override
public void flatMap(LongValue source, Collector<Edge<LongValue,NullValue>> out)
public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
throws Exception {
edge.f0 = source;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ public CycleGraph(ExecutionEnvironment env, long vertexCount) {
}

@Override
public Graph<LongValue,NullValue,NullValue> generate() {
public Graph<LongValue, NullValue, NullValue> generate() {
return new GridGraph(env)
.addDimension(vertexCount, true)
.setParallelism(parallelism)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,16 +60,16 @@ public EmptyGraph(ExecutionEnvironment env, long vertexCount) {
}

@Override
public Graph<LongValue,NullValue,NullValue> generate() {
public Graph<LongValue, NullValue, NullValue> generate() {
// Vertices
DataSet<Vertex<LongValue,NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount);
DataSet<Vertex<LongValue, NullValue>> vertices = GraphGeneratorUtils.vertexSequence(env, parallelism, vertexCount);

// Edges
TypeInformation<Edge<LongValue,NullValue>> typeInformation = new TupleTypeInfo<>(
TypeInformation<Edge<LongValue, NullValue>> typeInformation = new TupleTypeInfo<>(
ValueTypeInfo.LONG_VALUE_TYPE_INFO, ValueTypeInfo.LONG_VALUE_TYPE_INFO, ValueTypeInfo.NULL_VALUE_TYPE_INFO);

DataSource<Edge<LongValue,NullValue>> edges = env
.fromCollection(Collections.<Edge<LongValue,NullValue>>emptyList(), typeInformation)
DataSource<Edge<LongValue, NullValue>> edges = env
.fromCollection(Collections.<Edge<LongValue ,NullValue>>emptyList(), typeInformation)
.setParallelism(parallelism)
.name("Empty edge set");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,13 +39,13 @@ public interface GraphGenerator<K, VV, EV> {
*
* @return generated graph
*/
Graph<K,VV,EV> generate();
Graph<K, VV, EV> generate();

/**
* Override the operator parallelism.
*
* @param parallelism operator parallelism
* @return this
*/
GraphGenerator<K,VV,EV> setParallelism(int parallelism);
GraphGenerator<K, VV, EV> setParallelism(int parallelism);
}
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ public class GraphGeneratorUtils {
* @param vertexCount number of sequential vertex labels
* @return {@link DataSet} of sequentially labeled {@link Vertex Vertices}
*/
public static DataSet<Vertex<LongValue,NullValue>> vertexSequence(ExecutionEnvironment env, int parallelism, long vertexCount) {
public static DataSet<Vertex<LongValue, NullValue>> vertexSequence(ExecutionEnvironment env, int parallelism, long vertexCount) {
LongValueSequenceIterator iterator = new LongValueSequenceIterator(0, vertexCount-1);

DataSource<LongValue> vertexLabels = env
Expand All @@ -58,9 +58,9 @@ public static DataSet<Vertex<LongValue,NullValue>> vertexSequence(ExecutionEnvir

@ForwardedFields("*->f0")
private static class CreateVertex
implements MapFunction<LongValue, Vertex<LongValue,NullValue>> {
implements MapFunction<LongValue, Vertex<LongValue, NullValue>> {

private Vertex<LongValue,NullValue> vertex = new Vertex<>(null, NullValue.getInstance());
private Vertex<LongValue, NullValue> vertex = new Vertex<>(null, NullValue.getInstance());

@Override
public Vertex<LongValue, NullValue> map(LongValue value)
Expand All @@ -84,8 +84,8 @@ public Vertex<LongValue, NullValue> map(LongValue value)
*
* @see Graph#fromDataSet(DataSet, DataSet, ExecutionEnvironment)
*/
public static <K,EV> DataSet<Vertex<K,NullValue>> vertexSet(DataSet<Edge<K,EV>> edges, int parallelism) {
DataSet<Vertex<K,NullValue>> vertexSet = edges
public static <K, EV> DataSet<Vertex<K, NullValue>> vertexSet(DataSet<Edge<K, EV>> edges, int parallelism) {
DataSet<Vertex<K, NullValue>> vertexSet = edges
.flatMap(new EmitSrcAndTarget<K, EV>())
.setParallelism(parallelism)
.name("Emit source and target labels");
Expand All @@ -99,13 +99,13 @@ public static <K,EV> DataSet<Vertex<K,NullValue>> vertexSet(DataSet<Edge<K,EV>>
/**
* @see Graph.EmitSrcAndTarget
*/
private static final class EmitSrcAndTarget<K,EV>
implements FlatMapFunction<Edge<K,EV>, Vertex<K,NullValue>> {
private static final class EmitSrcAndTarget<K, EV>
implements FlatMapFunction<Edge<K, EV>, Vertex<K, NullValue>> {

private Vertex<K,NullValue> output = new Vertex<>(null, new NullValue());
private Vertex<K, NullValue> output = new Vertex<>(null, new NullValue());

@Override
public void flatMap(Edge<K,EV> value, Collector<Vertex<K,NullValue>> out) throws Exception {
public void flatMap(Edge<K, EV> value, Collector<Vertex<K, NullValue>> out) throws Exception {
output.f0 = value.f0;
out.collect(output);
output.f0 = value.f1;
Expand Down
Loading

0 comments on commit 53716a4

Please sign in to comment.