Graph Generators
- Circulant Graph
- Complete Graph
- Cycle Graph
- Echo Graph
- Empty Graph
- Grid Graph
- Hypercube Graph
- Path Graph
- RMat Graph
- Singleton Edge Graph
- Star Graph
Gelly provides a collection of scalable graph generators. Each generator is
- parallelizable, in order to create large datasets
- scale-free, generating the same graph regardless of parallelism
- thrifty, using as few operators as possible
Graph generators are configured using the builder pattern. The parallelism of generatoroperators can be set explicitly by calling setParallelism(parallelism)
. Lowering theparallelism will reduce the allocation of memory and network buffers.
Graph-specific configuration must be called first, then configuration common to allgenerators, and lastly the call to generate()
. The following example configures agrid graph with two dimensions, configures the parallelism, and generates the graph.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
boolean wrapEndpoints = false;
int parallelism = 4;
Graph<LongValue, NullValue, NullValue> graph = new GridGraph(env)
.addDimension(2, wrapEndpoints)
.addDimension(4, wrapEndpoints)
.setParallelism(parallelism)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.GridGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
wrapEndpoints = false
val parallelism = 4
val graph = new GridGraph(env.getJavaEnv).addDimension(2, wrapEndpoints).addDimension(4, wrapEndpoints).setParallelism(parallelism).generate()
Circulant Graph
A circulant graph is anoriented graph configuredwith one or more contiguous ranges of offsets. Edges connect integer vertex IDswhose difference equals a configured offset. The circulant graph with no offsetsis the empty graph and the graph with the maximum range is thecomplete graph.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5;
Graph<LongValue, NullValue, NullValue> graph = new CirculantGraph(env, vertexCount)
.addRange(1, 2)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.CirculantGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val graph = new CirculantGraph(env.getJavaEnv, vertexCount).addRange(1, 2).generate()
Complete Graph
An undirected graph connecting every distinct pair of vertices.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5;
Graph<LongValue, NullValue, NullValue> graph = new CompleteGraph(env, vertexCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.CompleteGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val graph = new CompleteGraph(env.getJavaEnv, vertexCount).generate()
Cycle Graph
An undirected graph where the set of edges form a single cycle by connectingeach vertex to two adjacent vertices in a chained loop.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5;
Graph<LongValue, NullValue, NullValue> graph = new CycleGraph(env, vertexCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.CycleGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val graph = new CycleGraph(env.getJavaEnv, vertexCount).generate()
Echo Graph
An echo graph is acirculant graph with n
vertices defined by the width of asingle range of offsets centered at n/2
. A vertex is connected to ‘far’vertices, which connect to ‘near’ vertices, which connect to ‘far’ vertices, ….
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5;
long vertexDegree = 2;
Graph<LongValue, NullValue, NullValue> graph = new EchoGraph(env, vertexCount, vertexDegree)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.EchoGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val vertexDegree = 2
val graph = new EchoGraph(env.getJavaEnv, vertexCount, vertexDegree).generate()
Empty Graph
A graph containing no edges.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5;
Graph<LongValue, NullValue, NullValue> graph = new EmptyGraph(env, vertexCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.EmptyGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val graph = new EmptyGraph(env.getJavaEnv, vertexCount).generate()
Grid Graph
An undirected graph connecting vertices in a regular tiling in one or more dimensions.Each dimension is configured separately. When the dimension size is at least three theendpoints are optionally connected by setting wrapEndpoints
. Changing the followingexample to addDimension(4, true)
would connect 0
to 3
and 4
to 7
.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
boolean wrapEndpoints = false;
Graph<LongValue, NullValue, NullValue> graph = new GridGraph(env)
.addDimension(2, wrapEndpoints)
.addDimension(4, wrapEndpoints)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.GridGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val wrapEndpoints = false
val graph = new GridGraph(env.getJavaEnv).addDimension(2, wrapEndpoints).addDimension(4, wrapEndpoints).generate()
Hypercube Graph
An undirected graph where edges form an n
-dimensional hypercube. Each vertexin a hypercube connects to one other vertex in each dimension.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long dimensions = 3;
Graph<LongValue, NullValue, NullValue> graph = new HypercubeGraph(env, dimensions)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.HypercubeGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val dimensions = 3
val graph = new HypercubeGraph(env.getJavaEnv, dimensions).generate()
Path Graph
An undirected graph where the set of edges form a single path by connectingtwo endpoint
vertices with degree 1
and all midpoint vertices with degree2
. A path graph can be formed by removing a single edge from a cycle graph.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 5
Graph<LongValue, NullValue, NullValue> graph = new PathGraph(env, vertexCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.PathGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 5
val graph = new PathGraph(env.getJavaEnv, vertexCount).generate()
RMat Graph
A directed power-law multigraph generated using theRecursive Matrix (R-Mat) model.
RMat is a stochastic generator configured with a source of randomness implementing theRandomGenerableFactory
interface. Provided implementations are JDKRandomGeneratorFactory
and MersenneTwisterFactory
. These generate an initial sequence of random values which arethen used as seeds for generating the edges.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
int vertexCount = 1 << scale;
int edgeCount = edgeFactor * vertexCount;
Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.RMatGraph
val env = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 1 << scale
val edgeCount = edgeFactor * vertexCount
val graph = new RMatGraph(env.getJavaEnv, rnd, vertexCount, edgeCount).generate()
The default RMat constants can be overridden as shown in the following example.The constants define the interdependence of bits from each generated edge’s sourceand target labels. The RMat noise can be enabled and progressively perturbs theconstants while generating each edge.
The RMat generator can be configured to produce a simple graph by removing self-loopsand duplicate edges. Symmetrization is performed either by a “clip-and-flip” throwing awaythe half matrix above the diagonal or a full “flip” preserving and mirroring all edges.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
RandomGenerableFactory<JDKRandomGenerator> rnd = new JDKRandomGeneratorFactory();
int vertexCount = 1 << scale;
int edgeCount = edgeFactor * vertexCount;
boolean clipAndFlip = false;
Graph<LongValue, NullValue, NullValue> graph = new RMatGraph<>(env, rnd, vertexCount, edgeCount)
.setConstants(0.57f, 0.19f, 0.19f)
.setNoise(true, 0.10f)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.RMatGraph
val env = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 1 << scale
val edgeCount = edgeFactor * vertexCount
clipAndFlip = false
val graph = new RMatGraph(env.getJavaEnv, rnd, vertexCount, edgeCount).setConstants(0.57f, 0.19f, 0.19f).setNoise(true, 0.10f).generate()
Singleton Edge Graph
An undirected graph containing isolated two-paths where every vertex has degree1
.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexPairCount = 4
// note: configured with the number of vertex pairs
Graph<LongValue, NullValue, NullValue> graph = new SingletonEdgeGraph(env, vertexPairCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.SingletonEdgeGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexPairCount = 4
// note: configured with the number of vertex pairs
val graph = new SingletonEdgeGraph(env.getJavaEnv, vertexPairCount).generate()
Star Graph
An undirected graph containing a single central vertex connected to all other leaf vertices.
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
long vertexCount = 6;
Graph<LongValue, NullValue, NullValue> graph = new StarGraph(env, vertexCount)
.generate();
import org.apache.flink.api.scala._
import org.apache.flink.graph.generator.StarGraph
val env: ExecutionEnvironment = ExecutionEnvironment.getExecutionEnvironment
val vertexCount = 6
val graph = new StarGraph(env.getJavaEnv, vertexCount).generate()