zhijiangW commented on a change in pull request #8608: [FLINK-11392][network] 
Introduce ShuffleEnvironment interface
URL: https://github.com/apache/flink/pull/8608#discussion_r290574645
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/ShuffleEnvironment.java
 ##########
 @@ -0,0 +1,163 @@
+/*
+ * 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.runtime.io.network;
+
+import org.apache.flink.metrics.MetricGroup;
+import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
+import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor;
+import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
+import org.apache.flink.runtime.executiongraph.PartitionInfo;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import 
org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider;
+import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
+import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.runtime.taskexecutor.TaskExecutor;
+
+import java.io.IOException;
+import java.util.Collection;
+
+/**
+ * Interface for the implementation of shuffle service locally on task 
executor.
+ *
+ * <p>Input/Output interface of local shuffle service is based on memory 
{@link org.apache.flink.runtime.io.network.buffer.Buffer}s.
+ * The task can request next available memory buffers from created here {@link 
ResultPartitionWriter}s to write shuffle data
+ * and buffers from created here {@link InputGate}s to read it.
+ *
+ * <h2>Service lifecycle management.</h2>
+ *
+ * <p>The interface contains method's to manage the lifecycle of the local 
shuffle environment:
+ * <ol>
+ *     <li>{@code start} is called when the {@link TaskExecutor} is being 
started.</li>
+ *     <li>{@code shutdown} is called when the {@link TaskExecutor} is being 
stopped.</li>
+ * </ol>
+ *
+ * <h2>Shuffle Input/Output management.</h2>
+ *
+ * <h3>Result partition management.</h3>
+ *
+ * <p>The interface implements a factory of result partition writers for the 
task output: {@code createResultPartitionWriters}.
+ * The created writers are grouped per task and handed over to the task thread 
upon its startup.
+ * The task is responsible for the writers lifecycle from that moment.
+ *
+ * <p>Partitions are released in the following cases:
+ * <ol>
+ *     <li>{@link ResultPartitionWriter#fail(Throwable)} and {@link 
ResultPartitionWriter#close()} are called
+ *     if the production has failed.</li>
+ *     <li>{@link ResultPartitionWriter#finish()} and {@link 
ResultPartitionWriter#close()} are called
+ *     if the production is done. The actual release can take some time
+ *     if 'the end of consumption' confirmation is being awaited implicitly
+ *     or the partition is later released by {@code 
releasePartitions(Collection<ResultPartitionID>)}.</li>
+ *     <li>{@code releasePartitions(Collection<ResultPartitionID>)} is called 
outside of the task thread,
+ *     e.g. to manage the local resource lifecycle of external partitions 
which outlive the task production.</li>
+ * </ol>
+ * The partitions, which currently still occupy local resources, can be 
queried with {@code updatePartitionInfo}.
+ *
+ * <h3>Input gate management.</h3>
+ *
+ * <p>The interface implements a factory for the task input gates: {@code 
createInputGates}.
+ * The created gates are grouped per task and handed over to the task thread 
upon its startup.
+ * The task is responsible for the gates lifecycle from that moment.
+ *
+ * <p>When tha task is deployed and the input gates are created, it can happen 
that not all consumed partitions
+ * are known at that moment e.g. because their producers have not been started 
yet.
+ * Therefore, the {@link ShuffleEnvironment} provides a method {@code 
updatePartitionInfo} to update them
+ * externally, ouside of the task thread, when the producer becomes known.
+ */
+public interface ShuffleEnvironment {
+
+       /**
+        * Starts the internal related services upon {@link TaskExecutor}'s 
startup.
 
 Review comment:
   Starts -> Start to keep the same form with the following methods.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to