Skip to content

Commit

Permalink
[FLINK-12201][network,metrics] Introduce InputGateWithMetrics in Task…
Browse files Browse the repository at this point in the history
… to increment numBytesIn metric

Incrementing of numBytesIn metric in SingleInputGate does not depend on shuffle service and can be moved out of network
internals into Task. Task could wrap InputGate provided by ShuffleService with InputGateWithMetrics which would increment
numBytesIn metric.
  • Loading branch information
zhijiangW committed May 31, 2019
1 parent ec4042f commit fc5d18f
Show file tree
Hide file tree
Showing 11 changed files with 158 additions and 49 deletions.
Expand Up @@ -20,7 +20,6 @@

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.JobID;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.Gauge;
import org.apache.flink.metrics.MetricGroup;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
Expand Down Expand Up @@ -252,8 +251,7 @@ public SingleInputGate[] createInputGates(
Collection<InputGateDeploymentDescriptor> inputGateDeploymentDescriptors,
MetricGroup parentGroup,
MetricGroup inputGroup,
MetricGroup buffersGroup,
Counter numBytesInCounter) {
MetricGroup buffersGroup) {
synchronized (lock) {
Preconditions.checkState(!isShutdown, "The NetworkEnvironment has already been shut down.");

Expand All @@ -265,8 +263,7 @@ public SingleInputGate[] createInputGates(
taskName,
igdd,
partitionProducerStateProvider,
inputChannelMetrics,
numBytesInCounter);
inputChannelMetrics);
InputGateID id = new InputGateID(igdd.getConsumedResultId(), executionId);
inputGatesById.put(id, inputGate);
inputGate.getCloseFuture().thenRun(() -> inputGatesById.remove(id));
Expand Down
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.runtime.event.AbstractEvent;
import org.apache.flink.runtime.io.network.buffer.Buffer;

Expand All @@ -43,26 +44,32 @@ public class BufferOrEvent {

private int channelIndex;

BufferOrEvent(Buffer buffer, int channelIndex, boolean moreAvailable) {
private final int size;

public BufferOrEvent(Buffer buffer, int channelIndex, boolean moreAvailable, int size) {
this.buffer = checkNotNull(buffer);
this.event = null;
this.channelIndex = channelIndex;
this.moreAvailable = moreAvailable;
this.size = size;
}

BufferOrEvent(AbstractEvent event, int channelIndex, boolean moreAvailable) {
public BufferOrEvent(AbstractEvent event, int channelIndex, boolean moreAvailable, int size) {
this.buffer = null;
this.event = checkNotNull(event);
this.channelIndex = channelIndex;
this.moreAvailable = moreAvailable;
this.size = size;
}

@VisibleForTesting
public BufferOrEvent(Buffer buffer, int channelIndex) {
this(buffer, channelIndex, true);
this(buffer, channelIndex, true, 0);
}

@VisibleForTesting
public BufferOrEvent(AbstractEvent event, int channelIndex) {
this(event, channelIndex, true);
this(event, channelIndex, true, 0);
}

public boolean isBuffer() {
Expand Down Expand Up @@ -96,11 +103,15 @@ boolean moreAvailable() {

@Override
public String toString() {
return String.format("BufferOrEvent [%s, channelIndex = %d]",
isBuffer() ? buffer : event, channelIndex);
return String.format("BufferOrEvent [%s, channelIndex = %d, size = %d]",
isBuffer() ? buffer : event, channelIndex, size);
}

public void setMoreAvailable(boolean moreAvailable) {
this.moreAvailable = moreAvailable;
}

public int getSize() {
return size;
}
}
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.metrics.Counter;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionLocation;
import org.apache.flink.runtime.event.AbstractEvent;
Expand Down Expand Up @@ -165,8 +164,6 @@ public class SingleInputGate extends InputGate {
/** A timer to retrigger local partition requests. Only initialized if actually needed. */
private Timer retriggerLocalRequestTimer;

private final Counter numBytesIn;

private final SupplierWithException<BufferPool, IOException> bufferPoolFactory;

private final CompletableFuture<Void> closeFuture;
Expand All @@ -178,7 +175,6 @@ public SingleInputGate(
int consumedSubpartitionIndex,
int numberOfInputChannels,
PartitionProducerStateProvider partitionProducerStateProvider,
Counter numBytesIn,
boolean isCreditBased,
SupplierWithException<BufferPool, IOException> bufferPoolFactory) {

Expand All @@ -200,8 +196,6 @@ public SingleInputGate(

this.partitionProducerStateProvider = checkNotNull(partitionProducerStateProvider);

this.numBytesIn = checkNotNull(numBytesIn);

this.isCreditBased = isCreditBased;

this.closeFuture = new CompletableFuture<>();
Expand Down Expand Up @@ -566,9 +560,9 @@ private BufferOrEvent transformToBufferOrEvent(
Buffer buffer,
boolean moreAvailable,
InputChannel currentChannel) throws IOException, InterruptedException {
numBytesIn.inc(buffer.getSizeUnsafe());
final int size = buffer.getSizeUnsafe();
if (buffer.isBuffer()) {
return new BufferOrEvent(buffer, currentChannel.getChannelIndex(), moreAvailable);
return new BufferOrEvent(buffer, currentChannel.getChannelIndex(), moreAvailable, size);
}
else {
final AbstractEvent event;
Expand Down Expand Up @@ -596,7 +590,7 @@ private BufferOrEvent transformToBufferOrEvent(
currentChannel.releaseAllResources();
}

return new BufferOrEvent(event, currentChannel.getChannelIndex(), moreAvailable);
return new BufferOrEvent(event, currentChannel.getChannelIndex(), moreAvailable, size);
}
}

Expand Down
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.metrics.Counter;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
import org.apache.flink.runtime.deployment.ResultPartitionLocation;
Expand Down Expand Up @@ -99,8 +98,7 @@ public SingleInputGate create(
@Nonnull String owningTaskName,
@Nonnull InputGateDeploymentDescriptor igdd,
@Nonnull PartitionProducerStateProvider partitionProducerStateProvider,
@Nonnull InputChannelMetrics metrics,
@Nonnull Counter numBytesInCounter) {
@Nonnull InputChannelMetrics metrics) {
final IntermediateDataSetID consumedResultId = checkNotNull(igdd.getConsumedResultId());
final ResultPartitionType consumedPartitionType = checkNotNull(igdd.getConsumedPartitionType());

Expand All @@ -116,7 +114,6 @@ public SingleInputGate create(
consumedSubpartitionIndex,
icdd.length,
partitionProducerStateProvider,
numBytesInCounter,
isCreditBased,
createBufferPoolFactory(icdd.length, consumedPartitionType));

Expand Down
@@ -0,0 +1,106 @@
/*
* 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.taskmanager;

import org.apache.flink.metrics.Counter;
import org.apache.flink.runtime.event.TaskEvent;
import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.metrics.groups.TaskIOMetricGroup;

import java.io.IOException;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;

import static org.apache.flink.util.Preconditions.checkNotNull;

/**
* This class wraps {@link InputGate} provided by shuffle service and it is mainly
* used for increasing general input metrics from {@link TaskIOMetricGroup}.
*/
public class InputGateWithMetrics extends InputGate {

private final InputGate inputGate;

private final Counter numBytesIn;

public InputGateWithMetrics(InputGate inputGate, Counter numBytesIn) {
this.inputGate = checkNotNull(inputGate);
this.numBytesIn = checkNotNull(numBytesIn);
}

@Override
public CompletableFuture<?> isAvailable() {
return inputGate.isAvailable();
}

@Override
public int getNumberOfInputChannels() {
return inputGate.getNumberOfInputChannels();
}

@Override
public String getOwningTaskName() {
return inputGate.getOwningTaskName();
}

@Override
public boolean isFinished() {
return inputGate.isFinished();
}

@Override
public void setup() throws IOException {
inputGate.setup();
}

@Override
public void requestPartitions() throws IOException, InterruptedException {
inputGate.requestPartitions();
}

@Override
public Optional<BufferOrEvent> getNextBufferOrEvent() throws IOException, InterruptedException {
Optional<BufferOrEvent> bufferOrEvent = inputGate.getNextBufferOrEvent();
bufferOrEvent.ifPresent(b -> numBytesIn.inc(b.getSize()));
return bufferOrEvent;
}

@Override
public Optional<BufferOrEvent> pollNextBufferOrEvent() throws IOException, InterruptedException {
Optional<BufferOrEvent> bufferOrEvent = inputGate.pollNextBufferOrEvent();
bufferOrEvent.ifPresent(b -> numBytesIn.inc(b.getSize()));
return bufferOrEvent;
}

@Override
public void sendTaskEvent(TaskEvent event) throws IOException {
inputGate.sendTaskEvent(event);
}

@Override
public int getPageSize() {
return inputGate.getPageSize();
}

@Override
public void close() throws Exception {
inputGate.close();
}
}
Expand Up @@ -56,7 +56,6 @@
import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier;
import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate;
import org.apache.flink.runtime.jobgraph.IntermediateDataSetID;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
Expand Down Expand Up @@ -192,7 +191,7 @@ public class Task implements Runnable, TaskActions, PartitionProducerStateProvid

private final ResultPartitionWriter[] producedPartitions;

private final SingleInputGate[] inputGates;
private final InputGate[] inputGates;

/** Connection to the task manager. */
private final TaskManagerActions taskManagerActions;
Expand Down Expand Up @@ -380,15 +379,20 @@ public Task(
buffersGroup);

// consumed intermediate result partitions
this.inputGates = networkEnvironment.createInputGates(
InputGate[] gates = networkEnvironment.createInputGates(
taskNameWithSubtaskAndId,
executionId,
this,
inputGateDeploymentDescriptors,
metrics.getIOMetricGroup(),
inputGroup,
buffersGroup,
metrics.getIOMetricGroup().getNumBytesInCounter());
buffersGroup);

this.inputGates = new InputGate[gates.length];
int counter = 0;
for (InputGate gate : gates) {
inputGates[counter++] = new InputGateWithMetrics(gate, metrics.getIOMetricGroup().getNumBytesInCounter());
}

invokableHasBeenCanceled = new AtomicBoolean(false);

Expand Down
Expand Up @@ -18,7 +18,6 @@

package org.apache.flink.runtime.io.network.partition;

import org.apache.flink.metrics.SimpleCounter;
import org.apache.flink.runtime.io.network.ConnectionManager;
import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
import org.apache.flink.runtime.io.network.api.serialization.EventSerializer;
Expand Down Expand Up @@ -331,7 +330,6 @@ public FairnessVerifyingInputGate(
consumedSubpartitionIndex,
numberOfInputChannels,
SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER,
new SimpleCounter(),
isCreditBased,
STUB_BUFFER_POOL_FACTORY);

Expand Down
Expand Up @@ -18,8 +18,6 @@

package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.SimpleCounter;
import org.apache.flink.runtime.io.network.NetworkEnvironment;
import org.apache.flink.runtime.io.network.buffer.BufferPool;
import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider;
Expand Down Expand Up @@ -51,8 +49,6 @@ public class SingleInputGateBuilder {

private final PartitionProducerStateProvider partitionProducerStateProvider = NO_OP_PRODUCER_CHECKER;

private final Counter numBytesInCounter = new SimpleCounter();

private boolean isCreditBased = true;

private SupplierWithException<BufferPool, IOException> bufferPoolFactory = () -> {
Expand Down Expand Up @@ -99,7 +95,6 @@ public SingleInputGate build() {
consumedSubpartitionIndex,
numberOfChannels,
partitionProducerStateProvider,
numBytesInCounter,
isCreditBased,
bufferPoolFactory);
}
Expand Down
Expand Up @@ -19,7 +19,6 @@
package org.apache.flink.runtime.io.network.partition.consumer;

import org.apache.flink.core.memory.MemorySegmentFactory;
import org.apache.flink.metrics.SimpleCounter;
import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
import org.apache.flink.runtime.deployment.InputChannelDeploymentDescriptor;
import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor;
Expand Down Expand Up @@ -347,8 +346,7 @@ public void testRequestBackoffConfiguration() throws Exception {
"TestTask",
gateDesc,
SingleInputGateBuilder.NO_OP_PRODUCER_CHECKER,
InputChannelTestUtils.newUnregisteredInputChannelMetrics(),
new SimpleCounter());
InputChannelTestUtils.newUnregisteredInputChannelMetrics());

try {
assertEquals(gateDesc.getConsumedPartitionType(), gate.getConsumedPartitionType());
Expand Down Expand Up @@ -599,8 +597,7 @@ private static Map<InputGateID, SingleInputGate> createInputGateWithLocalChannel
Arrays.asList(gateDescs),
new UnregisteredMetricsGroup(),
new UnregisteredMetricsGroup(),
new UnregisteredMetricsGroup(),
new SimpleCounter());
new UnregisteredMetricsGroup());
Map<InputGateID, SingleInputGate> inputGatesById = new HashMap<>();
for (int i = 0; i < numberOfGates; i++) {
inputGatesById.put(new InputGateID(ids[i], consumerID), gates[i]);
Expand Down

0 comments on commit fc5d18f

Please sign in to comment.