javeme commented on code in PR #248:
URL: 
https://github.com/apache/incubator-hugegraph-computer/pull/248#discussion_r1209209405


##########
computer-core/src/main/java/org/apache/hugegraph/computer/core/worker/load/LoadService.java:
##########
@@ -131,6 +142,7 @@ private Vertex 
convert(org.apache.hugegraph.structure.graph.Vertex
             computerVertex.properties(properties);
             return computerVertex;
         }
+

Review Comment:
   unexpected blank line



##########
computer-core/src/main/java/org/apache/hugegraph/computer/core/input/WorkerInputManager.java:
##########
@@ -63,33 +91,55 @@ public void init(Config config) {
     public void close(Config config) {
         this.loadService.close();
         this.sendManager.close(config);
+        this.sendExecutor.shutdown();
     }
 
     public void service(InputSplitRpcService rpcService) {
         this.loadService.rpcService(rpcService);
     }
 
     /**
-     * TODO: Load vertices and edges parallel.
      * When this method finish, it means that all vertices and edges are sent,
      * but there is no guarantee that all of them has been received.
      */
     public void loadGraph() {
+        List<CompletableFuture<?>> futures = new ArrayList<>();
+        CompletableFuture<?> future;
         this.sendManager.startSend(MessageType.VERTEX);
-        Iterator<Vertex> iterator = 
this.loadService.createIteratorFromVertex();
-        while (iterator.hasNext()) {
-            Vertex vertex = iterator.next();
-            this.sendManager.sendVertex(vertex);
+        for (int i = 0; i < this.sendThreadNum; i++) {
+            future = send(this.sendManager::sendVertex, 
this.loadService::createIteratorFromVertex);
+            futures.add(future);
         }
+        CompletableFuture.allOf(futures.toArray(new 
CompletableFuture[0])).exceptionally(e -> {
+            throw new ComputerException("An exception occurred during parallel 
" +
+                                        "sending vertices", e);
+        }).join();
         this.sendManager.finishSend(MessageType.VERTEX);
 
+        futures.clear();
+
         this.sendManager.startSend(MessageType.EDGE);
-        iterator = this.loadService.createIteratorFromEdge();
-        while (iterator.hasNext()) {
-            Vertex vertex = iterator.next();
-            this.sendManager.sendEdge(vertex);
+        for (int i = 0; i < this.sendThreadNum; i++) {
+            future = send(this.sendManager::sendEdge, 
this.loadService::createIteratorFromEdge);
+            futures.add(future);
         }
+        CompletableFuture.allOf(futures.toArray(new 
CompletableFuture[0])).exceptionally(e -> {
+            throw new ComputerException("An exception occurred during parallel 
" +
+                                        "sending edges", e);
+        }).join();
         this.sendManager.finishSend(MessageType.EDGE);
         this.sendManager.clearBuffer();
     }
+
+    private CompletableFuture<?> send(Consumer<Vertex> sendConsumer,
+                                      Supplier<Iterator<Vertex>> 
iteratorSupplier) {
+        return CompletableFuture.runAsync(() -> {
+            Iterator<Vertex> iterator = iteratorSupplier.get();
+            while (iterator.hasNext()) {
+                Vertex vertex = iterator.next();
+                sendConsumer.accept(vertex);
+            }
+        }, this.sendExecutor);
+    }
+

Review Comment:
   ditto



##########
computer-core/src/main/java/org/apache/hugegraph/computer/core/worker/load/LoadService.java:
##########
@@ -43,33 +44,37 @@ public class LoadService {
 
     private final GraphFactory graphFactory;
     private final Config config;
-    /*
-     * GraphFetcher include:
-     *   VertexFetcher vertexFetcher;
-     *   EdgeFetcher edgeFetcher;
-     */
-    private GraphFetcher fetcher;
+
     // Service proxy on the client
     private InputSplitRpcService rpcService;
     private final InputFilter inputFilter;
 
+    private final int fetcherNum;
+    private final GraphFetcher[] fetchers;
+    private final AtomicInteger fetcherIdx;
+
     public LoadService(ComputerContext context) {
         this.graphFactory = context.graphFactory();
         this.config = context.config();
-        this.fetcher = null;
         this.rpcService = null;
         this.inputFilter = context.config().createObject(
                 ComputerOptions.INPUT_FILTER_CLASS);
+        this.fetcherNum = 
this.config.get(ComputerOptions.INPUT_SEND_THREAD_NUMS);
+        this.fetchers = new GraphFetcher[this.fetcherNum];
+        this.fetcherIdx = new AtomicInteger(0);
     }
 
-    public void init() {
-        assert this.rpcService != null;
-        this.fetcher = InputSourceFactory.createGraphFetcher(this.config,
-                                                             this.rpcService);
+    public void close() {
+        for (GraphFetcher fetcher : this.fetchers) {
+            fetcher.close();
+        }
     }
 
-    public void close() {
-        this.fetcher.close();
+    public void init() {

Review Comment:
   the init() method expect to be moved before the close() method



##########
computer-core/src/main/java/org/apache/hugegraph/computer/core/worker/load/LoadService.java:
##########
@@ -227,5 +243,6 @@ private Edge 
convert(org.apache.hugegraph.structure.graph.Edge edge) {
             computerEdge.properties(properties);
             return computerEdge;
         }
+

Review Comment:
   ditto



##########
computer-core/src/main/java/org/apache/hugegraph/computer/core/worker/load/LoadService.java:
##########
@@ -43,33 +44,37 @@ public class LoadService {
 
     private final GraphFactory graphFactory;
     private final Config config;
-    /*
-     * GraphFetcher include:
-     *   VertexFetcher vertexFetcher;
-     *   EdgeFetcher edgeFetcher;
-     */
-    private GraphFetcher fetcher;
+
     // Service proxy on the client
     private InputSplitRpcService rpcService;
     private final InputFilter inputFilter;
 
+    private final int fetcherNum;
+    private final GraphFetcher[] fetchers;
+    private final AtomicInteger fetcherIdx;
+
     public LoadService(ComputerContext context) {
         this.graphFactory = context.graphFactory();
         this.config = context.config();
-        this.fetcher = null;
         this.rpcService = null;
         this.inputFilter = context.config().createObject(
                 ComputerOptions.INPUT_FILTER_CLASS);
+        this.fetcherNum = 
this.config.get(ComputerOptions.INPUT_SEND_THREAD_NUMS);
+        this.fetchers = new GraphFetcher[this.fetcherNum];
+        this.fetcherIdx = new AtomicInteger(0);
     }
 
-    public void init() {
-        assert this.rpcService != null;
-        this.fetcher = InputSourceFactory.createGraphFetcher(this.config,
-                                                             this.rpcService);
+    public void close() {
+        for (GraphFetcher fetcher : this.fetchers) {
+            fetcher.close();
+        }
     }
 
-    public void close() {
-        this.fetcher.close();
+    public void init() {
+        assert this.rpcService != null;
+        for (int i = 0; i < this.fetcherNum; i++) {
+            this.fetchers[i] = 
InputSourceFactory.createGraphFetcher(this.config, this.rpcService);

Review Comment:
   could you please also add some comment for the reason why we hold a fetchers 
array



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to