Uploaded image for project: 'Flink'
  1. Flink
  2. FLINK-5506

Java 8 - CommunityDetection.java:158 - java.lang.NullPointerException

    XMLWordPrintableJSON

Details

    Description

      Reporting this here as per Vasia's advice.
      I am having the following problem while trying out the org.apache.flink.graph.library.CommunityDetection algorithm of the Gelly API (Java).

      Specs: JDK 1.8.0_102 x64
      Apache Flink: 1.1.4

      Suppose I have a very small (I tried an example with 38 vertices as well) dataset stored in a tab-separated file 3-vertex.tsv:

      #id1 id2 score
      0    1    0
      0    2    0
      0    3    0
      

      This is just a central vertex with 3 neighbors (disconnected between themselves).
      I am loading the dataset and executing the algorithm with the following code:

      // Load the data from the .tsv file.
      final DataSet<Tuple3<Long, Long, Double>> edgeTuples = env.readCsvFile(inputPath)
                      .fieldDelimiter("\t") // node IDs are separated by spaces
                      .ignoreComments("#")  // comments start with "%"
                      .types(Long.class, Long.class, Double.class);
      
      // Generate a graph and add reverse edges (undirected).
      final Graph<Long, Long, Double> graph = Graph.fromTupleDataSet(edgeTuples, new MapFunction<Long, Long>() {
                  private static final long serialVersionUID = 8713516577419451509L;
                  public Long map(Long value) {
                      return value;
                  }
              },
      env).getUndirected();
      
      // CommunityDetection parameters.
      final double hopAttenuationDelta = 0.5d;
      final int iterationCount = 10;
      
      // Prepare and trigger the execution.
      DataSet<Vertex<Long, Long>> vs = graph.run(new org.apache.flink.graph.library.CommunityDetection<Long>(iterationCount, hopAttenuationDelta)).getVertices();
      
      vs.print();
      

      ​Running this code throws the following exception​ (check the bold line):

      ​org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
          at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply$mcV$sp(JobManager.scala:805)
          at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:751)
          at org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1$$anonfun$applyOrElse$8.apply(JobManager.scala:751)
          at scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1(Future.scala:24)
          at scala.concurrent.impl.Future$PromiseCompletingRunnable.run(Future.scala:24)
          at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:41)
          at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:401)
          at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
          at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.pollAndExecAll(ForkJoinPool.java:1253)
          at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1346)
          at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
          at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
      Caused by: java.lang.NullPointerException
          at org.apache.flink.graph.library.CommunityDetection$VertexLabelUpdater.updateVertex(CommunityDetection.java:158)
          at org.apache.flink.graph.spargel.ScatterGatherIteration$GatherUdfSimpleVV.coGroup(ScatterGatherIteration.java:389)
          at org.apache.flink.runtime.operators.CoGroupWithSolutionSetSecondDriver.run(CoGroupWithSolutionSetSecondDriver.java:218)
          at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:486)
          at org.apache.flink.runtime.iterative.task.AbstractIterativeTask.run(AbstractIterativeTask.java:146)
          at org.apache.flink.runtime.iterative.task.IterationTailTask.run(IterationTailTask.java:107)
          at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:351)
          at org.apache.flink.runtime.taskmanager.Task.run(Task.java:642)
          at java.lang.Thread.run(Thread.java:745)​
      

      ​After a further look, I set a breakpoint (Eclipse IDE debugging) at the line in bold:

      org.apache.flink.graph.library.CommunityDetection.java (source code accessed automatically by Maven)
      // find the highest score of maxScoreLabel
      double highestScore = labelsWithHighestScore.get(maxScoreLabel);​

      ​- maxScoreLabel has the value 3.​

      • labelsWithHighestScore was initialized as: Map<Long, Double> labelsWithHighestScore = new TreeMap<>();
      • labelsWithHighestScore is a TreeMap<Long, Double> and has the values:
      {0=0.0}

      null
      null
      [0=0.0]
      null
      1​

      ​It seems that the value 3 should have been added to that ​labelsWithHighestScore some time during execution, but because it wasn't, an exception is thrown.

      In the mailing list, Vasia speculates that the problem is that the implementation assumes that labelsWithHighestScores contains the vertex itself as initial label.

      Attachments

        Issue Links

          Activity

            People

              greghogan Greg Hogan
              mcoimbra Miguel E. Coimbra
              Votes:
              2 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved:

                Time Tracking

                  Estimated:
                  Original Estimate - 2h
                  2h
                  Remaining:
                  Remaining Estimate - 2h
                  2h
                  Logged:
                  Time Spent - Not Specified
                  Not Specified