[ 
https://issues.apache.org/jira/browse/FLINK-3780?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15292103#comment-15292103
 ] 

ASF GitHub Bot commented on FLINK-3780:
---------------------------------------

Github user greghogan commented on a diff in the pull request:

    https://github.com/apache/flink/pull/1980#discussion_r63951105
  
    --- Diff: 
flink-libraries/flink-gelly-examples/src/main/java/org/apache/flink/graph/examples/JaccardIndex.java
 ---
    @@ -0,0 +1,132 @@
    +/*
    + * 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.examples;
    +
    +import org.apache.commons.math3.random.JDKRandomGenerator;
    +import org.apache.flink.api.common.JobExecutionResult;
    +import org.apache.flink.api.java.DataSet;
    +import org.apache.flink.api.java.ExecutionEnvironment;
    +import org.apache.flink.api.java.io.CsvOutputFormat;
    +import org.apache.flink.api.java.utils.DataSetUtils;
    +import org.apache.flink.api.java.utils.ParameterTool;
    +import org.apache.flink.graph.Graph;
    +import org.apache.flink.graph.asm.translate.LongValueToIntValue;
    +import org.apache.flink.graph.asm.translate.TranslateGraphIds;
    +import org.apache.flink.graph.generator.RMatGraph;
    +import org.apache.flink.graph.generator.random.JDKRandomGeneratorFactory;
    +import org.apache.flink.graph.generator.random.RandomGenerableFactory;
    +import org.apache.flink.types.IntValue;
    +import org.apache.flink.types.LongValue;
    +import org.apache.flink.types.NullValue;
    +
    +import java.text.NumberFormat;
    +
    +/**
    + * Driver for the library implementation of Jaccard Index.
    + *
    + * This example generates an undirected RMat graph with the given scale and
    + * edge factor then calculates all non-zero Jaccard Index similarity scores
    + * between vertices.
    + *
    + * @see org.apache.flink.graph.library.similarity.JaccardIndex
    + */
    +public class JaccardIndex {
    +
    +   public static final int DEFAULT_SCALE = 10;
    +
    +   public static final int DEFAULT_EDGE_FACTOR = 16;
    +
    +   public static final boolean DEFAULT_CLIP_AND_FLIP = true;
    +
    +   public static void main(String[] args) throws Exception {
    +           // Set up the execution environment
    +           final ExecutionEnvironment env = 
ExecutionEnvironment.getExecutionEnvironment();
    +           env.getConfig().enableObjectReuse();
    +
    +           ParameterTool parameters = ParameterTool.fromArgs(args);
    +
    +           // Generate RMat graph
    +           int scale = parameters.getInt("scale", DEFAULT_SCALE);
    +           int edgeFactor = parameters.getInt("edge_factor", 
DEFAULT_EDGE_FACTOR);
    +
    +           RandomGenerableFactory<JDKRandomGenerator> rnd = new 
JDKRandomGeneratorFactory();
    +
    +           long vertexCount = 1L << scale;
    +           long edgeCount = vertexCount * edgeFactor;
    +
    +           boolean clipAndFlip = parameters.getBoolean("clip_and_flip", 
DEFAULT_CLIP_AND_FLIP);
    +
    +           Graph<LongValue, NullValue, NullValue> graph = new 
RMatGraph<>(env, rnd, vertexCount, edgeCount)
    +                   .setSimpleGraph(true, clipAndFlip)
    +                   .generate();
    +
    +           DataSet js;
    +
    +           if (scale > 32) {
    +                   js = graph
    +                           .run(new 
org.apache.flink.graph.library.similarity.JaccardIndex<LongValue, NullValue, 
NullValue>());
    +           } else {
    +                   js = graph
    +                           .run(new TranslateGraphIds<LongValue, IntValue, 
NullValue, NullValue>(new LongValueToIntValue()))
    +                           .run(new 
org.apache.flink.graph.library.similarity.JaccardIndex<IntValue, NullValue, 
NullValue>());
    +           }
    +
    +           switch (parameters.get("output", "")) {
    --- End diff --
    
    Is the usage statement sufficient?


> Jaccard Similarity
> ------------------
>
>                 Key: FLINK-3780
>                 URL: https://issues.apache.org/jira/browse/FLINK-3780
>             Project: Flink
>          Issue Type: New Feature
>          Components: Gelly
>    Affects Versions: 1.1.0
>            Reporter: Greg Hogan
>            Assignee: Greg Hogan
>             Fix For: 1.1.0
>
>
> Implement a Jaccard Similarity algorithm computing all non-zero similarity 
> scores. This algorithm is similar to {{TriangleListing}} but instead of 
> joining two-paths against an edge list we count two-paths.
> {{flink-gelly-examples}} currently has {{JaccardSimilarityMeasure}} which 
> relies on {{Graph.getTriplets()}} so only computes similarity scores for 
> neighbors but not neighbors-of-neighbors.
> This algorithm is easily modified for other similarity scores such as 
> Adamic-Adar similarity where the sum of endpoint degrees is replaced by the 
> degree of the middle vertex.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to