You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by fanzhidongyzby <gi...@git.apache.org> on 2017/04/30 10:19:21 UTC

[GitHub] flink pull request #3802: Add evenly graph

GitHub user fanzhidongyzby opened a pull request:

    https://github.com/apache/flink/pull/3802

    Add evenly graph

    Thanks for contributing to Apache Flink. Before you open your pull request, please take the following check list into consideration.
    If your changes take all of the items into account, feel free to open your pull request. For more information and/or questions please refer to the [How To Contribute guide](http://flink.apache.org/how-to-contribute.html).
    In addition to going through the list, please provide a meaningful description of your changes.
    
    - [x] General
      - The pull request references the related JIRA issue ("[FLINK-XXX] Jira title text")
      - The pull request addresses only one issue
      - Each commit in the PR has a meaningful commit message (including the JIRA id)
    
    - [x] Documentation
      - Documentation has been added for new functionality
      - Old documentation affected by the pull request has been updated
      - JavaDoc for public methods has been added
    
    - [x] Tests & Build
      - Functionality added by the pull request is covered by tests
      - `mvn clean verify` has been executed successfully locally or a Travis build has passed
    
    EvenlyGraph means every vertex in the graph has the same degree, so the graph can be treated as evenly due to all the edges in the graph are distributed evenly. when vertex degree is 0, an empty graph will be generated. when vertex degree is vertex count - 1, complete graph will be generated.
    
    The core idea is based on the concept of central symmetry. From the view of any vertex in the graph, the other vertices and edges are the same. So in the intermediate cases, edges are created from one vertex to the opposite vertex(if exists) and vertices on both sides of it.
    
    Considering the suggestion proposed by @greghogan , I have implemented CirculantGraph, and rewrite CompleteGraph and EvenlyGraph with CirculantGraph. EmptyGraph is not rewrote because @greghogan wants to keep as it is. CycleGraph is not rewrote because it has been implemented by GridGraph.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/fanzhidongyzby/flink add-evenly-graph

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/3802.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #3802
    
----
commit 8d58ac97da66e1be143ad732173891add3fffa4e
Author: FlorianFan <fa...@163.com>
Date:   2017-04-27T12:41:53Z

    [FLINK-6393] [gelly] Add Evenly Graph Generator to Flink Gelly

commit 1c45d8cc362e06e4c15bb1c21d648de2af95a189
Author: FlorianFan <fa...@163.com>
Date:   2017-04-30T10:05:56Z

    [FLINK-6393] [gelly] Add Circulant Graph Generator to Flink Gelly

commit f78fd99a97e5fcf09914fc5238a2d975d47fc5fa
Author: FlorianFan <fa...@163.com>
Date:   2017-04-30T10:06:22Z

    [FLINK-6393] [gelly] Rewrite implementation of CompleteGraph and EvenlyGraph with CirculantGraph Generator

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114074970
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    --- End diff --
    
    `CirculantGraph` is undirected graph, just like it's mentioned in **mathworld**.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073211
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    --- End diff --
    
    Can use the diamond operator: `ArrayList<>()`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114094440
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param offset appoint the vertices' position should be linked by any vertex
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffset(long offset) {
    --- End diff --
    
    I got it ! the method has been implemented assumes length is  1, I need add length parameter to enable add multiple offsets once, just with two parameters: offset and length.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073710
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    +            implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +        private final long vertexCount;
    +
    +        private final List<Long> offsets;
    +
    +        private LongValue target = new LongValue();
    +
    +        private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +        public LinkVertexToOffset(long vertexCount, List<Long> offsets) {
    +            this.vertexCount = vertexCount;
    +            this.offsets = offsets;
    +        }
    +
    +        @Override
    +        public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +                throws Exception {
    +            // empty graph
    +            if (offsets == null || offsets.isEmpty()) {
    --- End diff --
    
    I think better to not allow `offsets` to be `null`, then this block can be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073233
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    --- End diff --
    
    Description needs updating.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , ok, thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114074617
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    --- End diff --
    
    diamond operator will be used to replace `ArrayList<Long>`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    The tests sometimes fail. The build matrix splits the tests into three groups, each run with four different configurations. As you have noted, based on the changed code the single error can be ignored.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073291
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    --- End diff --
    
    Should we use the builder pattern as with `GridGraph`? To be efficient we should pair an offset with a length to allow a range of values to be specified concisely.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    Hi, @greghogan , 
    `CirculantGraph` Generator is implemented ok, and `EvenlyGraph` and `CompleteGraph` was rewrote and tested ok.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , The compiling task runs successfully, please have a look at the submitted code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114159796
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,145 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    --- End diff --
    
    the code has been move into fatMap


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114074838
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    --- End diff --
    
    done!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114466564
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Tuple2<Long, Long>> startOffsetPairs = new ArrayList<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    +			long offset = startOffset + i;
    +			Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +					"Offset must be at least " + MINIMUM_OFFSET);
    +			Preconditions.checkArgument(offset <= maxOffset,
    +					"Offset must be at most " + maxOffset);
    +		}
    +
    +		// save startOffset and length pair
    +		startOffsetPairs.add(new Tuple2<>(startOffset, length));
    +
    +		return this;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +				.fromParallelCollection(iterator, LongValue.class)
    +				.setParallelism(parallelism)
    +				.name("Edge iterators")
    +				.flatMap(new LinkVertexToOffsets(vertexCount, startOffsetPairs))
    +				.setParallelism(parallelism)
    +				.name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +	@FunctionAnnotation.ForwardedFields("*->f0")
    +	private static class LinkVertexToOffsets
    +			implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +		private final long vertexCount;
    +
    +		private final List<Tuple2<Long, Long>> startOffsetPairs;
    +
    +		private LongValue target = new LongValue();
    +
    +		private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +		public LinkVertexToOffsets(long vertexCount, List<Tuple2<Long, Long>> startOffsetPairs) {
    +			this.vertexCount = vertexCount;
    +			this.startOffsetPairs = startOffsetPairs;
    +		}
    +
    +		@Override
    +		public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +				throws Exception {
    +			edge.f0 = source;
    +
    +			// parse startOffsetPairs to offsets
    +			List<Long> offsets = new ArrayList<>();
    +			long maxOffset = vertexCount / 2;
    +			for (Tuple2<Long, Long> offsetPair : startOffsetPairs) {
    --- End diff --
    
    the iteration was simplified


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073355
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    --- End diff --
    
    This should be a `private static class`. I know it was copied from `CompleteGraph`, but inner classes should be `static` to avoid serialization errors.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114075640
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    --- End diff --
    
    done!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    What makes this graph 'evenly'? See `org.apache.flink.graph.drivers.input.CompleteGraph` for creating an input for `Runner` (the default class executed when doing a `/bin/flink run` on the flink-gelly-examples jar; see [usage](https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/libs/gelly/index.html)). Also need to update [`graph_generators.md`](https://ci.apache.org/projects/flink/flink-docs-release-1.3/dev/libs/gelly/graph_generators.html).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114129999
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,145 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    --- End diff --
    
    The range offset and length should be saved in an object and only iterated over within the `flatMap` function.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , the compiling error confused me:
    ```
    [ERROR] Plugin org.apache.maven.plugins:maven-surefire-plugin:2.12.2 or one of its dependencies could not be resolved: Failed to read artifact descriptor for org.apache.maven.plugins:maven-surefire-plugin:jar:2.12.2: Could not transfer artifact org.apache.maven.plugins:maven-surefire-plugin:pom:2.12.2 from/to central (https://repo.maven.apache.org/maven2): Connection reset -> [Help 1]
    [ERROR] 
    [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.
    [ERROR] Re-run Maven using the -X switch to enable full debug logging.
    [ERROR] 
    [ERROR] For more information about the errors and possible solutions, please read the following articles:
    [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/PluginResolutionException
    Trying to KILL watchdog (1542).
    ./tools/travis_mvn_watchdog.sh: line 210:  1542 Terminated              watchdog
    MVN exited with EXIT CODE: 1.
    java.io.FileNotFoundException: build-target/lib/flink-dist*.jar (No such file or directory)
    	at java.util.zip.ZipFile.open(Native Method)
    	at java.util.zip.ZipFile.<init>(ZipFile.java:220)
    	at java.util.zip.ZipFile.<init>(ZipFile.java:150)
    	at java.util.zip.ZipFile.<init>(ZipFile.java:121)
    	at sun.tools.jar.Main.list(Main.java:1060)
    	at sun.tools.jar.Main.run(Main.java:291)
    	at sun.tools.jar.Main.main(Main.java:1233)
    find: `./flink-yarn-tests/target/flink-yarn-tests*': No such file or directory
    PRODUCED build artifacts.
    build_info  mvn-1.log  mvn-2.log  mvn.out
    COMPRESSING build artifacts.
    ```
    My pull request does not change the code in `flink-yarn-tests`, can you give me some tips ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114466541
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,145 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    --- End diff --
    
    The loop check was removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , when would this pr be merged ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114324999
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,161 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.ArrayList;
    +import java.util.List;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Tuple2<Long, Long>> startOffsetPairs = new ArrayList<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    +			long offset = startOffset + i;
    +			Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +					"Offset must be at least " + MINIMUM_OFFSET);
    +			Preconditions.checkArgument(offset <= maxOffset,
    +					"Offset must be at most " + maxOffset);
    +		}
    +
    +		// save startOffset and length pair
    +		startOffsetPairs.add(new Tuple2<>(startOffset, length));
    +
    +		return this;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +				.fromParallelCollection(iterator, LongValue.class)
    +				.setParallelism(parallelism)
    +				.name("Edge iterators")
    +				.flatMap(new LinkVertexToOffsets(vertexCount, startOffsetPairs))
    +				.setParallelism(parallelism)
    +				.name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +	@FunctionAnnotation.ForwardedFields("*->f0")
    +	private static class LinkVertexToOffsets
    +			implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +		private final long vertexCount;
    +
    +		private final List<Tuple2<Long, Long>> startOffsetPairs;
    +
    +		private LongValue target = new LongValue();
    +
    +		private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +		public LinkVertexToOffsets(long vertexCount, List<Tuple2<Long, Long>> startOffsetPairs) {
    +			this.vertexCount = vertexCount;
    +			this.startOffsetPairs = startOffsetPairs;
    +		}
    +
    +		@Override
    +		public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +				throws Exception {
    +			edge.f0 = source;
    +
    +			// parse startOffsetPairs to offsets
    +			List<Long> offsets = new ArrayList<>();
    +			long maxOffset = vertexCount / 2;
    +			for (Tuple2<Long, Long> offsetPair : startOffsetPairs) {
    --- End diff --
    
    There is no need to create an array of offsets, certainly not in every `flatMap`. Can simply iterate over the list of `Tuple2` and collect new edges.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , `CirculantGraph` and `EchoGraph` are added to `Runner`, and graph_generators.md is updated well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    Am working on merging this ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114085668
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param offset appoint the vertices' position should be linked by any vertex
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffset(long offset) {
    --- End diff --
    
    By making this a `range` with both `offset` and `length` then users need not specifying hundreds or thousands of consecutive offsets. Will make use on the command line much simpler, and there is also less data for Flink to broadcast.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @greghogan , What should I do to process this error? All the modified code has been pushed, how to trigger the pr to be compiled again?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    Hi, @greghogan , the problems mentioned above has been fixed, but some may needn't to be fixed, the reason has been replied one by one, please review the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    This pull request is recreated, because [last pull request ](https://github.com/apache/flink/pull/3788) was closed by myself unexpectedly. 
    
    Please use this new pull request.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/3802


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073660
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    --- End diff --
    
    Why not allow the generation of directed graphs?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114073386
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    +            implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +        private final long vertexCount;
    +
    +        private final List<Long> offsets;
    +
    +        private LongValue target = new LongValue();
    +
    +        private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +        public LinkVertexToOffset(long vertexCount, List<Long> offsets) {
    +            this.vertexCount = vertexCount;
    +            this.offsets = offsets;
    +        }
    +
    +        @Override
    +        public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +                throws Exception {
    +            // empty graph
    +            if (offsets == null || offsets.isEmpty()) {
    +                return;
    +            }
    +
    +            edge.f0 = source;
    +
    +            // link to offset vertex
    +            for (long offset : offsets) {
    +                target.setValue((source.getValue() + offset + vertexCount) % vertexCount);
    --- End diff --
    
    No need to add the mod value. Also, I'd save `source.getValue()` into a long outside the loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114323050
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,145 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +import java.util.HashSet;
    +import java.util.Set;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private Set<Long> signedOffsets = new HashSet<>();
    +
    +	/**
    +	 * An undirected {@link Graph} whose {@link Vertex} connects to targets appointed by an offset list.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	/**
    +	 * Required configuration for each offset of the graph.
    +	 *
    +	 * @param startOffset first offset appointing the vertices' position should be linked by any vertex
    +	 * @param length offset in [startOffset, startOffset + length) will be added
    +	 * @return this
    +	 */
    +	public CirculantGraph addOffsets(long startOffset, long length) {
    +		long maxOffset = vertexCount / 2;
    +		for (int i = 0; i < length; i++) {
    --- End diff --
    
    Check arguments without a loop.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114074979
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    +            implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +        private final long vertexCount;
    +
    +        private final List<Long> offsets;
    +
    +        private LongValue target = new LongValue();
    +
    +        private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +        public LinkVertexToOffset(long vertexCount, List<Long> offsets) {
    +            this.vertexCount = vertexCount;
    +            this.offsets = offsets;
    +        }
    +
    +        @Override
    +        public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +                throws Exception {
    +            // empty graph
    +            if (offsets == null || offsets.isEmpty()) {
    --- End diff --
    
    ok, this code block will be removed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114075066
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    +            for (long offset : offsetList) {
    +                Preconditions.checkArgument(offset >= MINIMUM_OFFSET,
    +                        "Offset must be at least " + MINIMUM_OFFSET);
    +                Preconditions.checkArgument(offset <= maxOffset,
    +                        "Offset must be at most " + maxOffset);
    +
    +                // add sign, ignore negative max offset when vertex count is even
    +                signedOffsetList.add(offset);
    +                if (!(vertexCount % 2 == 0 && offset == maxOffset)) {
    +                    signedOffsetList.add(-offset);
    +                }
    +            }
    +        }
    +
    +		this.env = env;
    +		this.vertexCount = vertexCount;
    +	}
    +
    +	@Override
    +	public Graph<LongValue, NullValue, NullValue> generate() {
    +		// Vertices
    +		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
    +                .fromParallelCollection(iterator, LongValue.class)
    +                .setParallelism(parallelism)
    +                .name("Edge iterators")
    +                .flatMap(new LinkVertexToOffset(vertexCount, signedOffsetList))
    +                .setParallelism(parallelism)
    +                .name("Circulant graph edges");
    +
    +		// Graph
    +		return Graph.fromDataSet(vertices, edges, env);
    +	}
    +
    +    @FunctionAnnotation.ForwardedFields("*->f0")
    +    public class LinkVertexToOffset
    +            implements FlatMapFunction<LongValue, Edge<LongValue, NullValue>> {
    +
    +        private final long vertexCount;
    +
    +        private final List<Long> offsets;
    +
    +        private LongValue target = new LongValue();
    +
    +        private Edge<LongValue, NullValue> edge = new Edge<>(null, target, NullValue.getInstance());
    +
    +        public LinkVertexToOffset(long vertexCount, List<Long> offsets) {
    +            this.vertexCount = vertexCount;
    +            this.offsets = offsets;
    +        }
    +
    +        @Override
    +        public void flatMap(LongValue source, Collector<Edge<LongValue, NullValue>> out)
    +                throws Exception {
    +            // empty graph
    +            if (offsets == null || offsets.isEmpty()) {
    +                return;
    +            }
    +
    +            edge.f0 = source;
    +
    +            // link to offset vertex
    +            for (long offset : offsets) {
    +                target.setValue((source.getValue() + offset + vertexCount) % vertexCount);
    --- End diff --
    
    1. source.getValue() will be moved out side of the loop.
    2. mod operator should be used, because`offsets` in `LinkVertexToOffset` may be negative (offsets is signed).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114085607
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    +		Preconditions.checkArgument(vertexCount >= MINIMUM_VERTEX_COUNT,
    +			"Vertex count must be at least " + MINIMUM_VERTEX_COUNT);
    +
    +        if (offsetList != null && !offsetList.isEmpty()) {
    +            Preconditions.checkArgument(new HashSet<>(offsetList).size() == offsetList.size(),
    +                    "Offset must not be duplicated");
    +
    +            long maxOffset = vertexCount / 2;
    --- End diff --
    
    Yes, and we could allow users to select between `undirected` and `oriented`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink pull request #3802: Add Evenly Graph Generator to Flink Gelly

Posted by fanzhidongyzby <gi...@git.apache.org>.
Github user fanzhidongyzby commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3802#discussion_r114074928
  
    --- Diff: flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/generator/CirculantGraph.java ---
    @@ -0,0 +1,141 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.graph.generator;
    +
    +import java.util.*;
    +
    +import org.apache.flink.api.common.functions.FlatMapFunction;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.functions.FunctionAnnotation;
    +import org.apache.flink.graph.Edge;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.Vertex;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +import org.apache.flink.util.Collector;
    +import org.apache.flink.util.LongValueSequenceIterator;
    +import org.apache.flink.util.Preconditions;
    +
    +/*
    + * @see <a href="http://mathworld.wolfram.com/CirculantGraph.html">Circulant Graph at Wolfram MathWorld</a>
    + */
    +public class CirculantGraph
    +extends AbstractGraphGenerator<LongValue, NullValue, NullValue> {
    +
    +	public static final int MINIMUM_VERTEX_COUNT = 1;
    +
    +	public static final int MINIMUM_OFFSET = 1;
    +
    +	// Required to create the DataSource
    +	private final ExecutionEnvironment env;
    +
    +	// Required configuration
    +	private long vertexCount;
    +
    +	private List<Long> signedOffsetList = new ArrayList<Long>();
    +
    +	/**
    +	 * The {@link Graph} containing no edges.
    +	 *
    +	 * @param env the Flink execution environment
    +	 * @param vertexCount number of vertices
    +	 */
    +	public CirculantGraph(ExecutionEnvironment env, long vertexCount, List<Long> offsetList) {
    --- End diff --
    
    What's your meaning about **pair an offset with a length**, my comprehension is that an offset list(element is distinct) will be used, and a method `addOffset(long offset)` will be created to specify one offset config.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] flink issue #3802: Add Evenly Graph Generator to Flink Gelly

Posted by greghogan <gi...@git.apache.org>.
Github user greghogan commented on the issue:

    https://github.com/apache/flink/pull/3802
  
    @fanzhidongyzby are you using IntelliJ? You'll want to setup [checkstyle](https://ci.apache.org/projects/flink/flink-docs-release-1.3/internals/ide_setup.html#checkstyle) which should note the formatting errors that are causing tests to fail. Don't worry about the strict checkstyle.
    
    We don't yet have a means to import IntelliJ settings, so you'll also need to configure Editor -> Code Style -> Java -> Imports and set "Class count to use import with '*'" and "Name count to use static import with '*'" to 999.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---