zhijiangW commented on a change in pull request #8485: [FLINK-12555] Introduce 
an encapsulated metric group layout for shuffle API
URL: https://github.com/apache/flink/pull/8485#discussion_r291932569
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java
 ##########
 @@ -228,91 +211,111 @@ public void 
releasePartitions(Collection<ResultPartitionID> partitionIds) {
         * @return collection of partitions which still occupy some resources 
locally on this task executor
         * and have been not released yet.
         */
-       public Collection<ResultPartitionID> getUnreleasedPartitions() {
+       @Override
+       public Collection<ResultPartitionID> 
getPartitionsOccupyingLocalResources() {
                return resultPartitionManager.getUnreleasedPartitions();
        }
 
        // 
--------------------------------------------------------------------------------------------
        //  Create Output Writers and Input Readers
        // 
--------------------------------------------------------------------------------------------
 
-       public ResultPartition[] createResultPartitionWriters(
-                       String taskName,
-                       ExecutionAttemptID executionId,
-                       Collection<ResultPartitionDeploymentDescriptor> 
resultPartitionDeploymentDescriptors,
-                       MetricGroup outputGroup,
-                       MetricGroup buffersGroup) {
+       @Override
+       public ShuffleIOOwnerContext createShuffleIOOwnerContext(
+                       String ownerName,
+                       ExecutionAttemptID executionAttemptID,
+                       MetricGroup parentGroup) {
+               MetricGroup nettyGroup = 
createShuffleIOOwnerMetricGroup(checkNotNull(parentGroup));
+               return new ShuffleIOOwnerContext(
+                       checkNotNull(ownerName),
+                       checkNotNull(executionAttemptID),
+                       parentGroup,
+                       nettyGroup.addGroup(METRIC_GROUP_INPUT),
+                       nettyGroup.addGroup(METRIC_GROUP_OUTPUT));
+       }
+
+       @Override
+       public Collection<ResultPartition> createResultPartitionWriters(
+                       ShuffleIOOwnerContext ownerContext,
+                       Collection<ResultPartitionDeploymentDescriptor> 
resultPartitionDeploymentDescriptors) {
                synchronized (lock) {
-                       Preconditions.checkState(!isShutdown, "The 
NetworkEnvironment has already been shut down.");
+                       Preconditions.checkState(!isClosed, "The 
NettyShuffleEnvironment has already been shut down.");
 
                        ResultPartition[] resultPartitions = new 
ResultPartition[resultPartitionDeploymentDescriptors.size()];
                        int counter = 0;
                        for (ResultPartitionDeploymentDescriptor rpdd : 
resultPartitionDeploymentDescriptors) {
-                               resultPartitions[counter++] = 
resultPartitionFactory.create(taskName, executionId, rpdd);
+                               resultPartitions[counter++] = 
resultPartitionFactory.create(
+                                       ownerContext.getOwnerName(),
+                                       ownerContext.getExecutionAttemptID(),
+                                       rpdd);
                        }
 
-                       registerOutputMetrics(outputGroup, buffersGroup, 
resultPartitions);
-                       return resultPartitions;
+                       MetricGroup outputMetricGroup = 
ownerContext.getOutputGroup();
+                       registerOutputMetrics(
+                               config.isNetworkDetailedMetrics(),
+                               outputMetricGroup,
+                               
outputMetricGroup.addGroup(METRIC_GROUP_BUFFERS),
+                               resultPartitions);
+                       return  Arrays.asList(resultPartitions);
                }
        }
 
-       public SingleInputGate[] createInputGates(
-                       String taskName,
-                       ExecutionAttemptID executionId,
+       @Override
+       public Collection<SingleInputGate> createInputGates(
+                       ShuffleIOOwnerContext ownerContext,
                        PartitionProducerStateProvider 
partitionProducerStateProvider,
-                       Collection<InputGateDeploymentDescriptor> 
inputGateDeploymentDescriptors,
-                       MetricGroup parentGroup,
-                       MetricGroup inputGroup,
-                       MetricGroup buffersGroup) {
+                       Collection<InputGateDeploymentDescriptor> 
inputGateDeploymentDescriptors) {
                synchronized (lock) {
-                       Preconditions.checkState(!isShutdown, "The 
NetworkEnvironment has already been shut down.");
+                       Preconditions.checkState(!isClosed, "The 
NettyShuffleEnvironment has already been shut down.");
+
+                       MetricGroup networkInputGroup = 
ownerContext.getInputGroup();
+                       @SuppressWarnings("deprecation")
+                       InputChannelMetrics inputChannelMetrics = new 
InputChannelMetrics(networkInputGroup, ownerContext.getParentGroup());
 
-                       InputChannelMetrics inputChannelMetrics = new 
InputChannelMetrics(parentGroup);
                        SingleInputGate[] inputGates = new 
SingleInputGate[inputGateDeploymentDescriptors.size()];
                        int counter = 0;
                        for (InputGateDeploymentDescriptor igdd : 
inputGateDeploymentDescriptors) {
                                SingleInputGate inputGate = 
singleInputGateFactory.create(
-                                       taskName,
+                                       ownerContext.getOwnerName(),
                                        igdd,
                                        partitionProducerStateProvider,
                                        inputChannelMetrics);
-                               InputGateID id = new 
InputGateID(igdd.getConsumedResultId(), executionId);
+                               InputGateID id = new 
InputGateID(igdd.getConsumedResultId(), ownerContext.getExecutionAttemptID());
                                inputGatesById.put(id, inputGate);
                                inputGate.getCloseFuture().thenRun(() -> 
inputGatesById.remove(id));
                                inputGates[counter++] = inputGate;
                        }
 
-                       registerInputMetrics(inputGroup, buffersGroup, 
inputGates);
-                       return inputGates;
+                       registerInputMetrics(
+                               config.isNetworkDetailedMetrics(),
+                               networkInputGroup,
+                               
networkInputGroup.addGroup(METRIC_GROUP_BUFFERS),
 
 Review comment:
   ditto: for the issue of `networkInputGroup.addGroup(METRIC_GROUP_BUFFERS)`

----------------------------------------------------------------
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