diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java index 20ba45e521b7e..bd992c52fc44e 100644 --- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java +++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java @@ -27,7 +27,7 @@ import org.apache.flink.api.java.LocalEnvironment; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.java.BatchTableEnvironment; @@ -362,7 +362,7 @@ private abstract static class LimitNetworkBuffersTestEnvironment extends Executi public static void setAsContext() { Configuration config = new Configuration(); // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(80L << 20)); // 80 MB + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(80L << 20)); // 80 MB final LocalEnvironment le = new LocalEnvironment(config); initializeContextEnvironment(new ExecutionEnvironmentFactory() { diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index b1b97136af8f0..a4de6cc3e3f83 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -204,7 +204,7 @@ public final class ConfigConstants { public static final String TASK_MANAGER_IPC_PORT_KEY = "taskmanager.rpc.port"; /** - * @deprecated use {@link NetworkEnvironmentOptions#DATA_PORT} instead + * @deprecated use {@link NettyShuffleEnvironmentOptions#DATA_PORT} instead */ @Deprecated public static final String TASK_MANAGER_DATA_PORT_KEY = "taskmanager.data.port"; @@ -212,7 +212,7 @@ public final class ConfigConstants { /** * Config parameter to override SSL support for taskmanager's data transport. * - * @deprecated use {@link NetworkEnvironmentOptions#DATA_SSL_ENABLED} instead + * @deprecated use {@link NettyShuffleEnvironmentOptions#DATA_SSL_ENABLED} instead */ @Deprecated public static final String TASK_MANAGER_DATA_SSL_ENABLED = "taskmanager.data.ssl.enabled"; @@ -270,7 +270,7 @@ public final class ConfigConstants { * The config parameter defining the number of buffers used in the network stack. This defines the * number of possible tasks and shuffles. * - * @deprecated Use {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS} instead + * @deprecated Use {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS} instead */ @Deprecated public static final String TASK_MANAGER_NETWORK_NUM_BUFFERS_KEY = "taskmanager.network.numberOfBuffers"; @@ -1392,7 +1392,7 @@ public final class ConfigConstants { * The default network port the task manager expects to receive transfer envelopes on. The {@code 0} means that * the TaskManager searches for a free port. * - * @deprecated use {@link NetworkEnvironmentOptions#DATA_PORT} instead + * @deprecated use {@link NettyShuffleEnvironmentOptions#DATA_PORT} instead */ @Deprecated public static final int DEFAULT_TASK_MANAGER_DATA_PORT = 0; @@ -1400,7 +1400,7 @@ public final class ConfigConstants { /** * The default value to override ssl support for task manager's data transport. * - * @deprecated use {@link NetworkEnvironmentOptions#DATA_SSL_ENABLED} instead + * @deprecated use {@link NettyShuffleEnvironmentOptions#DATA_SSL_ENABLED} instead */ @Deprecated public static final boolean DEFAULT_TASK_MANAGER_DATA_SSL_ENABLED = true; @@ -1424,7 +1424,7 @@ public final class ConfigConstants { /** * Config key has been deprecated. Therefore, no default value required. * - * @deprecated {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS} provides the default value now + * @deprecated {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS} provides the default value now */ @Deprecated public static final int DEFAULT_TASK_MANAGER_NETWORK_NUM_BUFFERS = 2048; diff --git a/flink-core/src/main/java/org/apache/flink/configuration/NetworkEnvironmentOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java similarity index 99% rename from flink-core/src/main/java/org/apache/flink/configuration/NetworkEnvironmentOptions.java rename to flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java index 920cf5bcd28b1..699bd90cd5f1f 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/NetworkEnvironmentOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/NettyShuffleEnvironmentOptions.java @@ -29,7 +29,7 @@ */ @PublicEvolving @ConfigGroups(groups = @ConfigGroup(name = "NetworkNetty", keyPrefix = "taskmanager.network.netty")) -public class NetworkEnvironmentOptions { +public class NettyShuffleEnvironmentOptions { // ------------------------------------------------------------------------ // Network General Options @@ -212,5 +212,5 @@ public class NetworkEnvironmentOptions { // ------------------------------------------------------------------------ /** Not intended to be instantiated. */ - private NetworkEnvironmentOptions() {} + private NettyShuffleEnvironmentOptions() {} } diff --git a/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java b/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java index 9d6b638cb4b35..87cf19481c064 100644 --- a/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java +++ b/flink-dist/src/test/java/org/apache/flink/dist/TaskManagerHeapSizeCalculationJavaBashTest.java @@ -20,10 +20,10 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.taskexecutor.TaskManagerServices; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.OperatingSystem; import org.apache.flink.util.TestLogger; @@ -71,7 +71,7 @@ public void checkOperatingSystem() { } /** - * Tests that {@link NetworkEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} + * Tests that {@link NettyShuffleEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} * has the same result as the shell script. */ @Test @@ -159,9 +159,9 @@ private static Configuration getConfig( config.setLong(KEY_TASKM_MEM_SIZE, javaMemMB); config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, useOffHeap); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, netBufMemFrac); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(netBufMemMin)); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(netBufMemMax)); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, netBufMemFrac); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(netBufMemMin)); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(netBufMemMax)); if (managedMemSizeMB == 0) { config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); @@ -202,7 +202,7 @@ private static Configuration getRandomConfig(final Random ran) { Configuration config = getConfig(javaMemMB, useOffHeap, frac, min, max, managedMemSize, managedMemFrac); long totalJavaMemorySize = ((long) javaMemMB) << 20; // megabytes to bytes final int networkBufMB = - (int) (NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(totalJavaMemorySize, config) >> 20); + (int) (NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(totalJavaMemorySize, config) >> 20); // max (exclusive): total - netbuf managedMemSize = Math.min(javaMemMB - networkBufMB - 1, ran.nextInt(Integer.MAX_VALUE)); } else { @@ -228,14 +228,14 @@ private void compareNetworkBufJavaVsScript(final Configuration config, final flo final long totalJavaMemorySizeMB = config.getLong(KEY_TASKM_MEM_SIZE, 0L); - long javaNetworkBufMem = NetworkEnvironmentConfiguration.calculateNetworkBufferMemory( + long javaNetworkBufMem = NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory( totalJavaMemorySizeMB << 20, config); String[] command = {"src/test/bin/calcTMNetBufMem.sh", totalJavaMemorySizeMB + "m", - String.valueOf(config.getFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION)), - config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN), - config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX)}; + String.valueOf(config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION)), + config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN), + config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX)}; String scriptOutput = executeScript(command); @@ -272,9 +272,9 @@ private void compareHeapSizeJavaVsScript(final Configuration config, float toler String[] command = {"src/test/bin/calcTMHeapSizeMB.sh", totalJavaMemorySizeMB + "m", String.valueOf(config.getBoolean(TaskManagerOptions.MEMORY_OFF_HEAP)), - String.valueOf(config.getFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION)), - config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN), - config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX), + String.valueOf(config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION)), + config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN), + config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX), config.getString(TaskManagerOptions.MANAGED_MEMORY_SIZE), String.valueOf(config.getFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION))}; String scriptOutput = executeScript(command); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java similarity index 82% rename from flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java index c633a5fd83008..9cf86cb6d4ac2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NetworkEnvironment.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironment.java @@ -49,14 +49,16 @@ import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; -import org.apache.flink.runtime.taskexecutor.TaskExecutor; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; +import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Arrays; import java.util.Collection; import java.util.Map; import java.util.Optional; @@ -66,12 +68,13 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Network I/O components of each {@link TaskExecutor} instance. The network environment contains - * the data structures that keep track of all intermediate results and all data exchanges. + * The implementation of {@link ShuffleEnvironment} based on netty network communication, local memory and disk files. + * The network environment contains the data structures that keep track of all intermediate results + * and shuffle data exchanges. */ -public class NetworkEnvironment { +public class NettyShuffleEnvironment implements ShuffleEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(NetworkEnvironment.class); + private static final Logger LOG = LoggerFactory.getLogger(NettyShuffleEnvironment.class); private static final String METRIC_GROUP_NETWORK = "Network"; private static final String METRIC_TOTAL_MEMORY_SEGMENT = "TotalMemorySegments"; @@ -86,7 +89,7 @@ public class NetworkEnvironment { private final ResourceID taskExecutorLocation; - private final NetworkEnvironmentConfiguration config; + private final NettyShuffleEnvironmentConfiguration config; private final NetworkBufferPool networkBufferPool; @@ -100,11 +103,11 @@ public class NetworkEnvironment { private final SingleInputGateFactory singleInputGateFactory; - private boolean isShutdown; + private boolean isClosed; - private NetworkEnvironment( + private NettyShuffleEnvironment( ResourceID taskExecutorLocation, - NetworkEnvironmentConfiguration config, + NettyShuffleEnvironmentConfiguration config, NetworkBufferPool networkBufferPool, ConnectionManager connectionManager, ResultPartitionManager resultPartitionManager, @@ -118,12 +121,12 @@ private NetworkEnvironment( this.inputGatesById = new ConcurrentHashMap<>(); this.resultPartitionFactory = resultPartitionFactory; this.singleInputGateFactory = singleInputGateFactory; - this.isShutdown = false; + this.isClosed = false; } - public static NetworkEnvironment create( + public static NettyShuffleEnvironment create( + NettyShuffleEnvironmentConfiguration config, ResourceID taskExecutorLocation, - NetworkEnvironmentConfiguration config, TaskEventPublisher taskEventPublisher, MetricGroup metricGroup, IOManager ioManager) { @@ -162,7 +165,7 @@ public static NetworkEnvironment create( taskEventPublisher, networkBufferPool); - return new NetworkEnvironment( + return new NettyShuffleEnvironment( taskExecutorLocation, config, networkBufferPool, @@ -202,7 +205,7 @@ public NetworkBufferPool getNetworkBufferPool() { } @VisibleForTesting - public NetworkEnvironmentConfiguration getConfiguration() { + public NettyShuffleEnvironmentConfiguration getConfiguration() { return config; } @@ -211,11 +214,7 @@ public Optional getInputGate(InputGateID id) { return Optional.ofNullable(inputGatesById.get(id)); } - /** - * Batch release intermediate result partitions. - * - * @param partitionIds partition ids to release - */ + @Override public void releasePartitions(Collection partitionIds) { for (ResultPartitionID partitionId : partitionIds) { resultPartitionManager.releasePartition(partitionId, null); @@ -228,7 +227,8 @@ public void releasePartitions(Collection partitionIds) { * @return collection of partitions which still occupy some resources locally on this task executor * and have been not released yet. */ - public Collection getUnreleasedPartitions() { + @Override + public Collection getPartitionsOccupyingLocalResources() { return resultPartitionManager.getUnreleasedPartitions(); } @@ -236,54 +236,56 @@ public Collection getUnreleasedPartitions() { // Create Output Writers and Input Readers // -------------------------------------------------------------------------------------------- - public ResultPartition[] createResultPartitionWriters( - String taskName, - ExecutionAttemptID executionId, + @Override + public Collection createResultPartitionWriters( + String ownerName, + ExecutionAttemptID executionAttemptID, Collection resultPartitionDeploymentDescriptors, MetricGroup outputGroup, MetricGroup buffersGroup) { 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(ownerName, executionAttemptID, rpdd); } registerOutputMetrics(outputGroup, buffersGroup, resultPartitions); - return resultPartitions; + return Arrays.asList(resultPartitions); } } - public SingleInputGate[] createInputGates( - String taskName, - ExecutionAttemptID executionId, + @Override + public Collection createInputGates( + String ownerName, + ExecutionAttemptID executionAttemptID, PartitionProducerStateProvider partitionProducerStateProvider, Collection inputGateDeploymentDescriptors, MetricGroup parentGroup, MetricGroup inputGroup, MetricGroup buffersGroup) { synchronized (lock) { - Preconditions.checkState(!isShutdown, "The NetworkEnvironment has already been shut down."); + Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); InputChannelMetrics inputChannelMetrics = new InputChannelMetrics(parentGroup); SingleInputGate[] inputGates = new SingleInputGate[inputGateDeploymentDescriptors.size()]; int counter = 0; for (InputGateDeploymentDescriptor igdd : inputGateDeploymentDescriptors) { SingleInputGate inputGate = singleInputGateFactory.create( - taskName, + ownerName, igdd, partitionProducerStateProvider, inputChannelMetrics); - InputGateID id = new InputGateID(igdd.getConsumedResultId(), executionId); + InputGateID id = new InputGateID(igdd.getConsumedResultId(), executionAttemptID); inputGatesById.put(id, inputGate); inputGate.getCloseFuture().thenRun(() -> inputGatesById.remove(id)); inputGates[counter++] = inputGate; } registerInputMetrics(inputGroup, buffersGroup, inputGates); - return inputGates; + return Arrays.asList(inputGates); } } @@ -303,16 +305,7 @@ private void registerInputMetrics(MetricGroup inputGroup, MetricGroup buffersGro buffersGroup.gauge(METRIC_INPUT_POOL_USAGE, new InputBufferPoolUsageGauge(inputGates)); } - /** - * Update consuming gate with newly available partition. - * - * @param consumerID execution id of consumer to identify belonging to it gate. - * @param partitionInfo telling where the partition can be retrieved from - * @return {@code true} if the partition has been updated or {@code false} if the partition is not available anymore. - * @throws IOException IO problem by the update - * @throws InterruptedException potentially blocking operation was interrupted - * @throws IllegalStateException the input gate with the id from the partitionInfo is not found - */ + @Override public boolean updatePartitionInfo( ExecutionAttemptID consumerID, PartitionInfo partitionInfo) throws IOException, InterruptedException { @@ -335,9 +328,10 @@ public boolean updatePartitionInfo( * * @return a port to connect to the task executor for shuffle data exchange, -1 if only local connection is possible. */ + @Override public int start() throws IOException { synchronized (lock) { - Preconditions.checkState(!isShutdown, "The NetworkEnvironment has already been shut down."); + Preconditions.checkState(!isClosed, "The NettyShuffleEnvironment has already been shut down."); LOG.info("Starting the network environment and its components."); @@ -353,9 +347,10 @@ public int start() throws IOException { /** * Tries to shut down all network I/O components. */ - public void shutdown() { + @Override + public void close() { synchronized (lock) { - if (isShutdown) { + if (isClosed) { return; } @@ -390,13 +385,27 @@ public void shutdown() { LOG.warn("Network buffer pool did not shut down properly.", t); } - isShutdown = true; + isClosed = true; } } - public boolean isShutdown() { + public boolean isClosed() { synchronized (lock) { - return isShutdown; + return isClosed; } } + + public static NettyShuffleEnvironment fromShuffleContext(ShuffleEnvironmentContext shuffleEnvironmentContext) { + NettyShuffleEnvironmentConfiguration networkConfig = NettyShuffleEnvironmentConfiguration.fromConfiguration( + shuffleEnvironmentContext.getConfiguration(), + shuffleEnvironmentContext.getMaxJvmHeapMemory(), + shuffleEnvironmentContext.isLocalCommunicationOnly(), + shuffleEnvironmentContext.getHostAddress()); + return create( + networkConfig, + shuffleEnvironmentContext.getLocation(), + shuffleEnvironmentContext.getEventPublisher(), + shuffleEnvironmentContext.getParentMetricGroup(), + shuffleEnvironmentContext.getIOManager()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java index 6c869e972f373..1945bb0a7d801 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/ResultPartitionWriter.java @@ -28,6 +28,11 @@ /** * A buffer-oriented runtime result writer API for producing results. + * + *

If {@link ResultPartitionWriter#close()} is called before {@link ResultPartitionWriter#fail(Throwable)} or + * {@link ResultPartitionWriter#finish()}, it abruptly triggers failure and cancellation of production. + * In this case {@link ResultPartitionWriter#fail(Throwable)} still needs to be called afterwards to fully release + * all resources associated the the partition and propagate failure cause to the consumer if possible. */ public interface ResultPartitionWriter extends AutoCloseable { @@ -73,8 +78,9 @@ public interface ResultPartitionWriter extends AutoCloseable { /** * Fail the production of the partition. * - *

This method propagates non-{@code null} failure causes to consumers on a best-effort basis. - * Closing of partition is still needed. + *

This method propagates non-{@code null} failure causes to consumers on a best-effort basis. This call also + * leads to the release of all resources associated with the partition. Closing of the partition is still needed + * afterwards if it has not been done before. * * @param throwable failure cause */ @@ -83,7 +89,7 @@ public interface ResultPartitionWriter extends AutoCloseable { /** * Successfully finish the production of the partition. * - *

Closing of partition is still needed. + *

Closing of partition is still needed afterwards. */ void finish() throws IOException; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java index 2053100baa380..51c6aa173804b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/buffer/NetworkBufferPool.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.buffer; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.core.memory.MemorySegment; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.core.memory.MemorySegmentProvider; @@ -149,9 +149,9 @@ public List requestMemorySegments() throws IOException { totalNumberOfMemorySegments - numTotalRequiredBuffers, totalNumberOfMemorySegments, memorySegmentSize, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key())); + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key())); } this.numTotalRequiredBuffers += numberOfSegmentsToRequest; @@ -284,9 +284,9 @@ public BufferPool createBufferPool(int numRequiredBuffers, int maxUsedBuffers, O totalNumberOfMemorySegments - numTotalRequiredBuffers, totalNumberOfMemorySegments, memorySegmentSize, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key())); + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key())); } this.numTotalRequiredBuffers += numRequiredBuffers; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java index f0a96fe85cffc..9730907ef99e2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyConfig.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.netty; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.runtime.net.SSLUtils; import org.slf4j.Logger; @@ -98,37 +98,37 @@ public int getNumberOfSlots() { // ------------------------------------------------------------------------ public int getServerConnectBacklog() { - return config.getInteger(NetworkEnvironmentOptions.CONNECT_BACKLOG); + return config.getInteger(NettyShuffleEnvironmentOptions.CONNECT_BACKLOG); } public int getNumberOfArenas() { // default: number of slots - final int configValue = config.getInteger(NetworkEnvironmentOptions.NUM_ARENAS); + final int configValue = config.getInteger(NettyShuffleEnvironmentOptions.NUM_ARENAS); return configValue == -1 ? numberOfSlots : configValue; } public int getServerNumThreads() { // default: number of task slots - final int configValue = config.getInteger(NetworkEnvironmentOptions.NUM_THREADS_SERVER); + final int configValue = config.getInteger(NettyShuffleEnvironmentOptions.NUM_THREADS_SERVER); return configValue == -1 ? numberOfSlots : configValue; } public int getClientNumThreads() { // default: number of task slots - final int configValue = config.getInteger(NetworkEnvironmentOptions.NUM_THREADS_CLIENT); + final int configValue = config.getInteger(NettyShuffleEnvironmentOptions.NUM_THREADS_CLIENT); return configValue == -1 ? numberOfSlots : configValue; } public int getClientConnectTimeoutSeconds() { - return config.getInteger(NetworkEnvironmentOptions.CLIENT_CONNECT_TIMEOUT_SECONDS); + return config.getInteger(NettyShuffleEnvironmentOptions.CLIENT_CONNECT_TIMEOUT_SECONDS); } public int getSendAndReceiveBufferSize() { - return config.getInteger(NetworkEnvironmentOptions.SEND_RECEIVE_BUFFER_SIZE); + return config.getInteger(NettyShuffleEnvironmentOptions.SEND_RECEIVE_BUFFER_SIZE); } public TransportType getTransportType() { - String transport = config.getString(NetworkEnvironmentOptions.TRANSPORT_TYPE); + String transport = config.getString(NettyShuffleEnvironmentOptions.TRANSPORT_TYPE); switch (transport) { case "nio": @@ -155,7 +155,7 @@ public SSLHandlerFactory createServerSSLEngineFactory() throws Exception { } public boolean getSSLEnabled() { - return config.getBoolean(NetworkEnvironmentOptions.DATA_SSL_ENABLED) + return config.getBoolean(NettyShuffleEnvironmentOptions.DATA_SSL_ENABLED) && SSLUtils.isInternalSSLEnabled(config); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java index 1d0a0402a22dc..1fccf1d2ef277 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/ResultPartitionFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferPoolFactory; import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner; @@ -38,7 +39,7 @@ import java.util.Optional; /** - * Factory for {@link ResultPartition} to use in {@link org.apache.flink.runtime.io.network.NetworkEnvironment}. + * Factory for {@link ResultPartition} to use in {@link NettyShuffleEnvironment}. */ public class ResultPartitionFactory { private static final Logger LOG = LoggerFactory.getLogger(ResultPartition.class); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java index 554d723581f01..60bc70d3e0d40 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.io.network.ConnectionManager; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferPoolFactory; @@ -33,7 +34,7 @@ import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.shuffle.NettyShuffleDescriptor; import org.apache.flink.runtime.shuffle.ShuffleDescriptor; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.function.SupplierWithException; import org.slf4j.Logger; @@ -46,7 +47,7 @@ import static org.apache.flink.runtime.shuffle.ShuffleUtils.applyWithShuffleTypeCheck; /** - * Factory for {@link SingleInputGate} to use in {@link org.apache.flink.runtime.io.network.NetworkEnvironment}. + * Factory for {@link SingleInputGate} to use in {@link NettyShuffleEnvironment}. */ public class SingleInputGateFactory { private static final Logger LOG = LoggerFactory.getLogger(SingleInputGate.class); @@ -78,7 +79,7 @@ public class SingleInputGateFactory { public SingleInputGateFactory( @Nonnull ResourceID taskExecutorLocation, - @Nonnull NetworkEnvironmentConfiguration networkConfig, + @Nonnull NettyShuffleEnvironmentConfiguration networkConfig, @Nonnull ConnectionManager connectionManager, @Nonnull ResultPartitionManager partitionManager, @Nonnull TaskEventPublisher taskEventPublisher, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java new file mode 100644 index 0000000000000..882e52cc7bddd --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironment.java @@ -0,0 +1,166 @@ +/* + * 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.shuffle; + +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.buffer.Buffer; +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 java.io.IOException; +import java.util.Collection; + +/** + * Interface for the implementation of shuffle service local environment. + * + *

Input/Output interface of local shuffle service environment is based on memory {@link Buffer Buffers}. A producer + * can write shuffle data into the buffers, obtained from the created {@link ResultPartitionWriter ResultPartitionWriters} + * and a consumer reads the buffers from the created {@link InputGate InputGates}. + * + *

Lifecycle management.

+ * + *

The interface contains method's to manage the lifecycle of the local shuffle service environment: + *

    + *
  1. {@link ShuffleEnvironment#start} must be called before using the shuffle service environment.
  2. + *
  3. {@link ShuffleEnvironment#close} is called to release the shuffle service environment.
  4. + *
+ * + *

Shuffle Input/Output management.

+ * + *

Result partition management.

+ * + *

The interface implements a factory of result partition writers to produce shuffle data: + * {@link ShuffleEnvironment#createResultPartitionWriters}. The created writers are grouped per owner. + * The owner is responsible for the writers' lifecycle from the moment of creation. + * + *

Partitions are released in the following cases: + *

    + *
  1. {@link ResultPartitionWriter#fail(Throwable)} and {@link ResultPartitionWriter#close()} are called + * if the production has failed. + *
  2. + *
  3. {@link ResultPartitionWriter#finish()} and {@link ResultPartitionWriter#close()} are called + * if the production is done. The actual release can take some time depending on implementation details, + * e.g. if the `end of consumption' confirmation from the consumer is being awaited implicitly + * or the partition is later released by {@link ShuffleEnvironment#releasePartitions(Collection)}.
  4. + *
  5. {@link ShuffleEnvironment#releasePartitions(Collection)} is called outside of the producer thread, + * e.g. to manage the lifecycle of BLOCKING result partitions which can outlive their producers.
  6. + *
+ * The partitions, which currently still occupy local resources, can be queried with + * {@link ShuffleEnvironment#getPartitionsOccupyingLocalResources}. + * + *

Input gate management.

+ * + *

The interface implements a factory for the input gates: {@link ShuffleEnvironment#createInputGates}. + * The created gates are grouped per owner. The owner is responsible for the gates' lifecycle from the moment of creation. + * + *

When 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 {@link ShuffleEnvironment#updatePartitionInfo} to update them externally, when the producer becomes known. + * The update mechanism has to be threadsafe because the updated gate can be read concurrently from a different thread. + * + * @param

type of provided result partition writers + * @param type of provided input gates + */ +public interface ShuffleEnvironment

extends AutoCloseable { + + /** + * Start the internal related services before using the shuffle service environment. + * + * @return a port to connect for the shuffle data exchange, -1 if only local connection is possible. + */ + int start() throws IOException; + + /** + * Factory method for the {@link ResultPartitionWriter ResultPartitionWriters} to produce result partitions. + * + *

The order of the {@link ResultPartitionWriter ResultPartitionWriters} in the returned collection + * should be the same as the iteration order of the passed {@code resultPartitionDeploymentDescriptors}. + * + * @param ownerName the owner name, used for logs + * @param executionAttemptID execution attempt id of the producer + * @param resultPartitionDeploymentDescriptors descriptors of the partition, produced by the owner + * @param outputGroup shuffle specific group for output metrics + * @param buffersGroup shuffle specific group for buffer metrics + * @return collection of the {@link ResultPartitionWriter ResultPartitionWriters} + */ + Collection

createResultPartitionWriters( + String ownerName, + ExecutionAttemptID executionAttemptID, + Collection resultPartitionDeploymentDescriptors, + MetricGroup outputGroup, + MetricGroup buffersGroup); + + /** + * Release local resources occupied by the given partitions. + * + * @param partitionIds identifying the partitions to be released + */ + void releasePartitions(Collection partitionIds); + + /** + * Report partitions which still occupy some resources locally. + * + * @return collection of partitions which still occupy some resources locally + * and have not been released yet. + */ + Collection getPartitionsOccupyingLocalResources(); + + /** + * Factory method for the {@link InputGate InputGates} to consume result partitions. + * + *

The order of the {@link InputGate InputGates} in the returned collection should be the same as the iteration order + * of the passed {@code inputGateDeploymentDescriptors}. + * + * @param ownerName the owner name, used for logs + * @param executionAttemptID execution attempt id of the consumer + * @param partitionProducerStateProvider producer state provider to query whether the producer is ready for consumption + * @param inputGateDeploymentDescriptors descriptors of the input gates to consume + * @param parentGroup parent of shuffle specific metric group + * @param inputGroup shuffle specific group for input metrics + * @param buffersGroup shuffle specific group for buffer metrics + * @return collection of the {@link InputGate InputGates} + */ + Collection createInputGates( + String ownerName, + ExecutionAttemptID executionAttemptID, + PartitionProducerStateProvider partitionProducerStateProvider, + Collection inputGateDeploymentDescriptors, + MetricGroup parentGroup, + MetricGroup inputGroup, + MetricGroup buffersGroup); + + /** + * Update a gate with the newly available partition information, previously unknown. + * + * @param consumerID execution id to distinguish gates with the same id from the different consumer executions + * @param partitionInfo information needed to consume the updated partition, e.g. network location + * @return {@code true} if the partition has been updated or {@code false} if the partition is not available anymore. + * @throws IOException IO problem by the update + * @throws InterruptedException potentially blocking operation was interrupted + */ + boolean updatePartitionInfo( + ExecutionAttemptID consumerID, + PartitionInfo partitionInfo) throws IOException, InterruptedException; +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java new file mode 100644 index 0000000000000..f7108b6b474d1 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/shuffle/ShuffleEnvironmentContext.java @@ -0,0 +1,94 @@ +/* + * 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.shuffle; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.clusterframework.types.ResourceID; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.TaskEventPublisher; + +import java.net.InetAddress; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Local context used to create {@link ShuffleEnvironment}. + */ +public class ShuffleEnvironmentContext { + private final Configuration configuration; + private final ResourceID location; + private final long maxJvmHeapMemory; + private final boolean localCommunicationOnly; + private final InetAddress hostAddress; + private final TaskEventPublisher eventPublisher; + private final MetricGroup parentMetricGroup; + private final IOManager ioManager; + + public ShuffleEnvironmentContext( + Configuration configuration, + ResourceID location, + long maxJvmHeapMemory, + boolean localCommunicationOnly, + InetAddress hostAddress, + TaskEventPublisher eventPublisher, + MetricGroup parentMetricGroup, + IOManager ioManager) { + this.configuration = checkNotNull(configuration); + this.location = checkNotNull(location); + this.maxJvmHeapMemory = maxJvmHeapMemory; + this.localCommunicationOnly = localCommunicationOnly; + this.hostAddress = checkNotNull(hostAddress); + this.eventPublisher = checkNotNull(eventPublisher); + this.parentMetricGroup = checkNotNull(parentMetricGroup); + this.ioManager = checkNotNull(ioManager); + } + + public Configuration getConfiguration() { + return configuration; + } + + public ResourceID getLocation() { + return location; + } + + public long getMaxJvmHeapMemory() { + return maxJvmHeapMemory; + } + + public boolean isLocalCommunicationOnly() { + return localCommunicationOnly; + } + + public InetAddress getHostAddress() { + return hostAddress; + } + + public TaskEventPublisher getEventPublisher() { + return eventPublisher; + } + + public MetricGroup getParentMetricGroup() { + return parentMetricGroup; + } + + public IOManager getIOManager() { + return ioManager; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java index e19282414734e..a31521e653608 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskExecutor.java @@ -48,7 +48,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.instance.HardwareDescription; import org.apache.flink.runtime.instance.InstanceID; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider; @@ -169,7 +169,7 @@ public class TaskExecutor extends RpcEndpoint implements TaskExecutorGateway { private final TaskExecutorLocalStateStoresManager localStateStoresManager; /** The network component in the task manager. */ - private final NetworkEnvironment networkEnvironment; + private final ShuffleEnvironment shuffleEnvironment; /** The kvState registration service in the task manager. */ private final KvStateService kvStateService; @@ -238,7 +238,7 @@ public TaskExecutor( this.jobLeaderService = taskExecutorServices.getJobLeaderService(); this.taskManagerLocation = taskExecutorServices.getTaskManagerLocation(); this.localStateStoresManager = taskExecutorServices.getTaskManagerStateStore(); - this.networkEnvironment = taskExecutorServices.getNetworkEnvironment(); + this.shuffleEnvironment = taskExecutorServices.getShuffleEnvironment(); this.kvStateService = taskExecutorServices.getKvStateService(); this.resourceManagerLeaderRetriever = haServices.getResourceManagerLeaderRetriever(); @@ -270,8 +270,7 @@ public TaskExecutor( @Override public CompletableFuture canBeReleased() { - return CompletableFuture.completedFuture( - taskExecutorServices.getNetworkEnvironment().getUnreleasedPartitions().isEmpty()); + return CompletableFuture.completedFuture(shuffleEnvironment.getPartitionsOccupyingLocalResources().isEmpty()); } // ------------------------------------------------------------------------ @@ -533,7 +532,7 @@ public CompletableFuture submitTask( tdd.getTargetSlotNumber(), taskExecutorServices.getMemoryManager(), taskExecutorServices.getIOManager(), - taskExecutorServices.getNetworkEnvironment(), + taskExecutorServices.getShuffleEnvironment(), taskExecutorServices.getKvStateService(), taskExecutorServices.getBroadcastVariableManager(), taskExecutorServices.getTaskEventDispatcher(), @@ -615,7 +614,7 @@ public CompletableFuture updatePartitions( CompletableFuture.runAsync( () -> { try { - if (!networkEnvironment.updatePartitionInfo(executionAttemptID, partitionInfo)) { + if (!shuffleEnvironment.updatePartitionInfo(executionAttemptID, partitionInfo)) { log.debug( "Discard update for input gate partition {} of result {} in task {}. " + "The partition is no longer available.", @@ -643,7 +642,7 @@ public CompletableFuture updatePartitions( @Override public void releasePartitions(Collection partitionIds) { try { - networkEnvironment.releasePartitions(partitionIds); + shuffleEnvironment.releasePartitions(partitionIds); } catch (Throwable t) { // TODO: Do we still need this catch branch? onFatalError(t); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java index 4288573cbd872..d5985b140c1c5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunner.java @@ -356,8 +356,10 @@ public static TaskExecutor startTaskManager( TaskManagerServicesConfiguration taskManagerServicesConfiguration = TaskManagerServicesConfiguration.fromConfiguration( configuration, - EnvironmentInformation.getMaxJvmHeapMemory(), + resourceID, remoteAddress, + EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag(), + EnvironmentInformation.getMaxJvmHeapMemory(), localCommunicationOnly); Tuple2 taskManagerMetricGroup = MetricUtils.instantiateTaskManagerMetricGroup( @@ -369,10 +371,7 @@ public static TaskExecutor startTaskManager( TaskManagerServices taskManagerServices = TaskManagerServices.fromConfiguration( taskManagerServicesConfiguration, taskManagerMetricGroup.f1, - resourceID, - rpcService.getExecutor(), // TODO replace this later with some dedicated executor for io. - EnvironmentInformation.getSizeOfFreeHeapMemoryWithDefrag(), - EnvironmentInformation.getMaxJvmHeapMemory()); + rpcService.getExecutor()); // TODO replace this later with some dedicated executor for io. TaskManagerConfiguration taskManagerConfiguration = TaskManagerConfiguration.fromConfiguration(configuration); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java index a9689ce9f4e00..5d6207614ac01 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java @@ -27,17 +27,18 @@ import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; +import org.apache.flink.runtime.shuffle.ShuffleEnvironmentContext; import org.apache.flink.runtime.state.TaskExecutorLocalStateStoresManager; import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.ConfigurationParserUtils; import org.apache.flink.util.ExceptionUtils; @@ -58,7 +59,7 @@ /** * Container for {@link TaskExecutor} services such as the {@link MemoryManager}, {@link IOManager}, - * {@link NetworkEnvironment}. All services are exclusive to a single {@link TaskExecutor}. + * {@link ShuffleEnvironment}. All services are exclusive to a single {@link TaskExecutor}. * Consequently, the respective {@link TaskExecutor} is responsible for closing them. */ public class TaskManagerServices { @@ -71,7 +72,7 @@ public class TaskManagerServices { private final TaskManagerLocation taskManagerLocation; private final MemoryManager memoryManager; private final IOManager ioManager; - private final NetworkEnvironment networkEnvironment; + private final ShuffleEnvironment shuffleEnvironment; private final KvStateService kvStateService; private final BroadcastVariableManager broadcastVariableManager; private final TaskSlotTable taskSlotTable; @@ -84,7 +85,7 @@ public class TaskManagerServices { TaskManagerLocation taskManagerLocation, MemoryManager memoryManager, IOManager ioManager, - NetworkEnvironment networkEnvironment, + ShuffleEnvironment shuffleEnvironment, KvStateService kvStateService, BroadcastVariableManager broadcastVariableManager, TaskSlotTable taskSlotTable, @@ -96,7 +97,7 @@ public class TaskManagerServices { this.taskManagerLocation = Preconditions.checkNotNull(taskManagerLocation); this.memoryManager = Preconditions.checkNotNull(memoryManager); this.ioManager = Preconditions.checkNotNull(ioManager); - this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); + this.shuffleEnvironment = Preconditions.checkNotNull(shuffleEnvironment); this.kvStateService = Preconditions.checkNotNull(kvStateService); this.broadcastVariableManager = Preconditions.checkNotNull(broadcastVariableManager); this.taskSlotTable = Preconditions.checkNotNull(taskSlotTable); @@ -118,8 +119,8 @@ public IOManager getIOManager() { return ioManager; } - public NetworkEnvironment getNetworkEnvironment() { - return networkEnvironment; + public ShuffleEnvironment getShuffleEnvironment() { + return shuffleEnvironment; } public KvStateService getKvStateService() { @@ -184,7 +185,7 @@ public void shutDown() throws FlinkException { } try { - networkEnvironment.shutdown(); + shuffleEnvironment.close(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } @@ -223,20 +224,14 @@ public void shutDown() throws FlinkException { * * @param taskManagerServicesConfiguration task manager configuration * @param taskManagerMetricGroup metric group of the task manager - * @param resourceID resource ID of the task manager * @param taskIOExecutor executor for async IO operations - * @param freeHeapMemoryWithDefrag an estimate of the size of the free heap memory - * @param maxJvmHeapMemory the maximum JVM heap size * @return task manager components * @throws Exception */ public static TaskManagerServices fromConfiguration( TaskManagerServicesConfiguration taskManagerServicesConfiguration, MetricGroup taskManagerMetricGroup, - ResourceID resourceID, - Executor taskIOExecutor, - long freeHeapMemoryWithDefrag, - long maxJvmHeapMemory) throws Exception { + Executor taskIOExecutor) throws Exception { // pre-start checks checkTempDirs(taskManagerServicesConfiguration.getTmpDirPaths()); @@ -246,24 +241,23 @@ public static TaskManagerServices fromConfiguration( // start the I/O manager, it will create some temp directories. final IOManager ioManager = new IOManagerAsync(taskManagerServicesConfiguration.getTmpDirPaths()); - final NetworkEnvironment network = NetworkEnvironment.create( - resourceID, - taskManagerServicesConfiguration.getNetworkConfig(), + final ShuffleEnvironment shuffleEnvironment = createShuffleEnvironment( + taskManagerServicesConfiguration, taskEventDispatcher, taskManagerMetricGroup, ioManager); - int dataPort = network.start(); + final int dataPort = shuffleEnvironment.start(); final KvStateService kvStateService = KvStateService.fromConfiguration(taskManagerServicesConfiguration); kvStateService.start(); final TaskManagerLocation taskManagerLocation = new TaskManagerLocation( - resourceID, + taskManagerServicesConfiguration.getResourceID(), taskManagerServicesConfiguration.getTaskManagerAddress(), dataPort); // this call has to happen strictly after the network stack has been initialized - final MemoryManager memoryManager = createMemoryManager(taskManagerServicesConfiguration, freeHeapMemoryWithDefrag, maxJvmHeapMemory); + final MemoryManager memoryManager = createMemoryManager(taskManagerServicesConfiguration); final BroadcastVariableManager broadcastVariableManager = new BroadcastVariableManager(); @@ -300,7 +294,7 @@ public static TaskManagerServices fromConfiguration( taskManagerLocation, memoryManager, ioManager, - network, + shuffleEnvironment, kvStateService, broadcastVariableManager, taskSlotTable, @@ -310,19 +304,34 @@ public static TaskManagerServices fromConfiguration( taskEventDispatcher); } + private static ShuffleEnvironment createShuffleEnvironment( + TaskManagerServicesConfiguration taskManagerServicesConfiguration, + TaskEventDispatcher taskEventDispatcher, + MetricGroup taskManagerMetricGroup, + IOManager ioManager) { + + final ShuffleEnvironmentContext shuffleEnvironmentContext = new ShuffleEnvironmentContext( + taskManagerServicesConfiguration.getConfiguration(), + taskManagerServicesConfiguration.getResourceID(), + taskManagerServicesConfiguration.getMaxJvmHeapMemory(), + taskManagerServicesConfiguration.isLocalCommunicationOnly(), + taskManagerServicesConfiguration.getTaskManagerAddress(), + taskEventDispatcher, + taskManagerMetricGroup, + ioManager); + + return NettyShuffleEnvironment.fromShuffleContext(shuffleEnvironmentContext); + } + /** * Creates a {@link MemoryManager} from the given {@link TaskManagerServicesConfiguration}. * * @param taskManagerServicesConfiguration to create the memory manager from - * @param freeHeapMemoryWithDefrag an estimate of the size of the free heap memory - * @param maxJvmHeapMemory the maximum JVM heap size * @return Memory manager * @throws Exception */ private static MemoryManager createMemoryManager( - TaskManagerServicesConfiguration taskManagerServicesConfiguration, - long freeHeapMemoryWithDefrag, - long maxJvmHeapMemory) throws Exception { + TaskManagerServicesConfiguration taskManagerServicesConfiguration) throws Exception { // computing the amount of memory to use depends on how much memory is available // it strictly needs to happen AFTER the network stack has been initialized @@ -347,6 +356,7 @@ private static MemoryManager createMemoryManager( float memoryFraction = taskManagerServicesConfiguration.getMemoryFraction(); if (memType == MemoryType.HEAP) { + long freeHeapMemoryWithDefrag = taskManagerServicesConfiguration.getFreeHeapMemoryWithDefrag(); // network buffers allocated off-heap -> use memoryFraction of the available heap: long relativeMemSize = (long) (freeHeapMemoryWithDefrag * memoryFraction); if (preAllocateMemory) { @@ -358,6 +368,7 @@ private static MemoryManager createMemoryManager( } memorySize = relativeMemSize; } else if (memType == MemoryType.OFF_HEAP) { + long maxJvmHeapMemory = taskManagerServicesConfiguration.getMaxJvmHeapMemory(); // The maximum heap memory has been adjusted according to the fraction (see // calculateHeapSizeMB(long totalJavaMemorySizeMB, Configuration config)), i.e. // maxJvmHeap = jvmTotalNoNet - jvmTotalNoNet * memoryFraction = jvmTotalNoNet * (1 - memoryFraction) @@ -382,7 +393,7 @@ private static MemoryManager createMemoryManager( memoryManager = new MemoryManager( memorySize, taskManagerServicesConfiguration.getNumberOfSlots(), - taskManagerServicesConfiguration.getNetworkConfig().networkBufferSize(), + taskManagerServicesConfiguration.getPageSize(), memType, preAllocateMemory); } catch (OutOfMemoryError e) { @@ -415,7 +426,7 @@ public static long calculateHeapSizeMB(long totalJavaMemorySizeMB, Configuration Preconditions.checkArgument(totalJavaMemorySizeMB > 0); // subtract the Java memory used for network buffers (always off-heap) - final long networkBufMB = NetworkEnvironmentConfiguration.calculateNetworkBufferMemory( + final long networkBufMB = NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory( totalJavaMemorySizeMB << 20, // megabytes to bytes config) >> 20; // bytes to megabytes final long remainingJavaMemorySizeMB = totalJavaMemorySizeMB - networkBufMB; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java index 198fd10b57170..64f5a30bcb71e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfiguration.java @@ -25,8 +25,8 @@ import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.akka.AkkaUtils; +import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.registration.RetryingRegistrationConfiguration; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.util.ConfigurationParserUtils; import javax.annotation.Nullable; @@ -38,24 +38,32 @@ import static org.apache.flink.util.Preconditions.checkNotNull; /** - * Configuration for the task manager services such as the network environment, the memory manager, + * Configuration for the task manager services such as the memory manager, * the io manager and the metric registry. */ public class TaskManagerServicesConfiguration { + private final Configuration configuration; + + private final ResourceID resourceID; + private final InetAddress taskManagerAddress; + private final boolean localCommunicationOnly; + private final String[] tmpDirPaths; private final String[] localRecoveryStateRootDirectories; private final int numberOfSlots; - private final NetworkEnvironmentConfiguration networkConfig; - @Nullable private final QueryableStateConfiguration queryableStateConfig; + private final long freeHeapMemoryWithDefrag; + + private final long maxJvmHeapMemory; + /** * Managed memory (in megabytes). * @@ -69,6 +77,8 @@ public class TaskManagerServicesConfiguration { private final float memoryFraction; + private final int pageSize; + private final long timerServiceShutdownTimeout; private final boolean localRecoveryEnabled; @@ -78,26 +88,35 @@ public class TaskManagerServicesConfiguration { private Optional

    *
  • {@link TaskManagerOptions#MANAGED_MEMORY_SIZE},
  • *
  • {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}, and
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS} (fallback if the ones above do not exist)
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}, and
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS} (fallback if the ones above do not exist)
  • *
. * * @param config configuration object @@ -223,7 +220,7 @@ public static long calculateNewNetworkBufferMemory(Configuration config, long ma // jvmHeapNoNet = jvmHeap - networkBufBytes // = jvmHeap - Math.min(networkBufMax, Math.max(networkBufMin, jvmHeap * netFraction) // jvmHeap = jvmHeapNoNet / (1.0 - networkBufFraction) - float networkBufFraction = config.getFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); + float networkBufFraction = config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); long networkBufSize = (long) (jvmHeapNoNet / (1.0 - networkBufFraction) * networkBufFraction); return calculateNewNetworkBufferMemory(config, networkBufSize, maxJvmHeapMemory); } @@ -234,10 +231,10 @@ public static long calculateNewNetworkBufferMemory(Configuration config, long ma * *

The following configuration parameters are involved: *

    - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}, and
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS} (fallback if the ones above do not exist)
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}, and
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS} (fallback if the ones above do not exist)
  • *
. * * @param totalJavaMemorySize overall available memory to use (in bytes) @@ -247,22 +244,22 @@ public static long calculateNewNetworkBufferMemory(Configuration config, long ma */ @SuppressWarnings("deprecation") public static long calculateNetworkBufferMemory(long totalJavaMemorySize, Configuration config) { - final int segmentSize = getPageSize(config); + final int segmentSize = ConfigurationParserUtils.getPageSize(config); final long networkBufBytes; if (hasNewNetworkConfig(config)) { - float networkBufFraction = config.getFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); + float networkBufFraction = config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); long networkBufSize = (long) (totalJavaMemorySize * networkBufFraction); networkBufBytes = calculateNewNetworkBufferMemory(config, networkBufSize, totalJavaMemorySize); } else { // use old (deprecated) network buffers parameter - int numNetworkBuffers = config.getInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS); + int numNetworkBuffers = config.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); networkBufBytes = (long) numNetworkBuffers * (long) segmentSize; checkOldNetworkConfig(numNetworkBuffers); ConfigurationParserUtils.checkConfigParameter(networkBufBytes < totalJavaMemorySize, - networkBufBytes, NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS.key(), + networkBufBytes, NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS.key(), "Network buffer memory size too large: " + networkBufBytes + " >= " + totalJavaMemorySize + " (total JVM memory size)"); } @@ -276,9 +273,9 @@ public static long calculateNetworkBufferMemory(long totalJavaMemorySize, Config * *

The following configuration parameters are involved: *

    - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • - *
  • {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN},
  • + *
  • {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}
  • *
. * * @param config configuration object @@ -288,11 +285,11 @@ public static long calculateNetworkBufferMemory(long totalJavaMemorySize, Config * @return memory to use for network buffers (in bytes) */ private static long calculateNewNetworkBufferMemory(Configuration config, long networkBufSize, long maxJvmHeapMemory) { - float networkBufFraction = config.getFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); - long networkBufMin = MemorySize.parse(config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN)).getBytes(); - long networkBufMax = MemorySize.parse(config.getString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX)).getBytes(); + float networkBufFraction = config.getFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION); + long networkBufMin = MemorySize.parse(config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN)).getBytes(); + long networkBufMax = MemorySize.parse(config.getString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX)).getBytes(); - int pageSize = getPageSize(config); + int pageSize = ConfigurationParserUtils.getPageSize(config); checkNewNetworkConfig(pageSize, networkBufFraction, networkBufMin, networkBufMax); @@ -300,9 +297,9 @@ private static long calculateNewNetworkBufferMemory(Configuration config, long n ConfigurationParserUtils.checkConfigParameter(networkBufBytes < maxJvmHeapMemory, "(" + networkBufFraction + ", " + networkBufMin + ", " + networkBufMax + ")", - "(" + NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key() + ", " + - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key() + ", " + - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key() + ")", + "(" + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key() + ", " + + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key() + ", " + + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key() + ")", "Network buffer memory size too large: " + networkBufBytes + " >= " + maxJvmHeapMemory + " (maximum JVM memory size)"); @@ -319,7 +316,7 @@ private static long calculateNewNetworkBufferMemory(Configuration config, long n @SuppressWarnings("deprecation") private static void checkOldNetworkConfig(final int numNetworkBuffers) { ConfigurationParserUtils.checkConfigParameter(numNetworkBuffers > 0, numNetworkBuffers, - NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS.key(), + NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS.key(), "Must have at least one network buffer"); } @@ -340,23 +337,23 @@ private static void checkNewNetworkConfig( final long networkBufMax) throws IllegalConfigurationException { ConfigurationParserUtils.checkConfigParameter(networkBufFraction > 0.0f && networkBufFraction < 1.0f, networkBufFraction, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), "Network buffer memory fraction of the free memory must be between 0.0 and 1.0"); ConfigurationParserUtils.checkConfigParameter(networkBufMin >= pageSize, networkBufMin, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), "Minimum memory for network buffers must allow at least one network " + "buffer with respect to the memory segment size"); ConfigurationParserUtils.checkConfigParameter(networkBufMax >= pageSize, networkBufMax, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), "Maximum memory for network buffers must allow at least one network " + "buffer with respect to the memory segment size"); ConfigurationParserUtils.checkConfigParameter(networkBufMax >= networkBufMin, networkBufMax, - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), "Maximum memory for network buffers must not be smaller than minimum memory (" + - NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key() + ": " + networkBufMin + ")"); + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key() + ": " + networkBufMin + ")"); } /** @@ -369,10 +366,10 @@ private static void checkNewNetworkConfig( @SuppressWarnings("deprecation") @VisibleForTesting public static boolean hasNewNetworkConfig(final Configuration config) { - return config.contains(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION) || - config.contains(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN) || - config.contains(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX) || - !config.contains(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS); + return config.contains(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION) || + config.contains(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN) || + config.contains(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX) || + !config.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); } /** @@ -382,8 +379,8 @@ public static boolean hasNewNetworkConfig(final Configuration config) { * @return the data port */ private static int getDataport(Configuration configuration) { - final int dataport = configuration.getInteger(NetworkEnvironmentOptions.DATA_PORT); - ConfigurationParserUtils.checkConfigParameter(dataport >= 0, dataport, NetworkEnvironmentOptions.DATA_PORT.key(), + final int dataport = configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_PORT); + ConfigurationParserUtils.checkConfigParameter(dataport >= 0, dataport, NettyShuffleEnvironmentOptions.DATA_PORT.key(), "Leave config parameter empty or use 0 to let the system choose a port automatically."); return dataport; @@ -401,19 +398,19 @@ private static int calculateNumberOfNetworkBuffers(Configuration configuration, final int numberOfNetworkBuffers; if (!hasNewNetworkConfig(configuration)) { // fallback: number of network buffers - numberOfNetworkBuffers = configuration.getInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS); + numberOfNetworkBuffers = configuration.getInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS); checkOldNetworkConfig(numberOfNetworkBuffers); } else { - if (configuration.contains(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS)) { + if (configuration.contains(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS)) { LOG.info("Ignoring old (but still present) network buffer configuration via {}.", - NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS.key()); + NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS.key()); } final long networkMemorySize = calculateNewNetworkBufferMemory(configuration, maxJvmHeapMemory); // tolerate offcuts between intended and allocated memory due to segmentation (will be available to the user-space memory) - long numberOfNetworkBuffersLong = networkMemorySize / getPageSize(configuration); + long numberOfNetworkBuffersLong = networkMemorySize / ConfigurationParserUtils.getPageSize(configuration); if (numberOfNetworkBuffersLong > Integer.MAX_VALUE) { throw new IllegalArgumentException("The given number of memory bytes (" + networkMemorySize + ") corresponds to more than MAX_INT pages."); @@ -444,8 +441,12 @@ private static NettyConfig createNettyConfig( if (!localTaskManagerCommunication) { final InetSocketAddress taskManagerInetSocketAddress = new InetSocketAddress(taskManagerAddress, dataport); - nettyConfig = new NettyConfig(taskManagerInetSocketAddress.getAddress(), taskManagerInetSocketAddress.getPort(), - getPageSize(configuration), ConfigurationParserUtils.getSlot(configuration), configuration); + nettyConfig = new NettyConfig( + taskManagerInetSocketAddress.getAddress(), + taskManagerInetSocketAddress.getPort(), + ConfigurationParserUtils.getPageSize(configuration), + ConfigurationParserUtils.getSlot(configuration), + configuration); } else { nettyConfig = null; } @@ -453,28 +454,6 @@ private static NettyConfig createNettyConfig( return nettyConfig; } - /** - * Parses the configuration to get the page size and validates the value. - * - * @param configuration configuration object - * @return size of memory segment - */ - public static int getPageSize(Configuration configuration) { - final int pageSize = checkedDownCast(MemorySize.parse( - configuration.getString(TaskManagerOptions.MEMORY_SEGMENT_SIZE)).getBytes()); - - // check page size of for minimum size - ConfigurationParserUtils.checkConfigParameter(pageSize >= MemoryManager.MIN_PAGE_SIZE, pageSize, - TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(), - "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); - // check page size for power of two - ConfigurationParserUtils.checkConfigParameter(MathUtils.isPowerOf2(pageSize), pageSize, - TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(), - "Memory segment size must be a power of 2."); - - return pageSize; - } - // ------------------------------------------------------------------------ @Override @@ -500,7 +479,7 @@ else if (obj == null || getClass() != obj.getClass()) { return false; } else { - final NetworkEnvironmentConfiguration that = (NetworkEnvironmentConfiguration) obj; + final NettyShuffleEnvironmentConfiguration that = (NettyShuffleEnvironmentConfiguration) obj; return this.numNetworkBuffers == that.numNetworkBuffers && this.networkBufferSize == that.networkBufferSize && @@ -515,7 +494,7 @@ else if (obj == null || getClass() != obj.getClass()) { @Override public String toString() { - return "NetworkEnvironmentConfiguration{" + + return "NettyShuffleEnvironmentConfiguration{" + ", numNetworkBuffers=" + numNetworkBuffers + ", networkBufferSize=" + networkBufferSize + ", partitionRequestInitialBackoff=" + partitionRequestInitialBackoff + diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java index 27d9cdcaa7f81..5512c549e14ce 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java @@ -49,7 +49,6 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; @@ -63,6 +62,7 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.CheckpointListener; import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; @@ -283,7 +283,7 @@ public Task( int targetSlotNumber, MemoryManager memManager, IOManager ioManager, - NetworkEnvironment networkEnvironment, + ShuffleEnvironment shuffleEnvironment, KvStateService kvStateService, BroadcastVariableManager bcVarManager, TaskEventDispatcher taskEventDispatcher, @@ -368,12 +368,12 @@ public Task( final MetricGroup inputGroup = networkGroup.addGroup("Input"); // produced intermediate result partitions - final ResultPartitionWriter[] resultPartitionWriters = networkEnvironment.createResultPartitionWriters( + final ResultPartitionWriter[] resultPartitionWriters = shuffleEnvironment.createResultPartitionWriters( taskNameWithSubtaskAndId, executionId, resultPartitionDeploymentDescriptors, outputGroup, - buffersGroup); + buffersGroup).toArray(new ResultPartitionWriter[] {}); this.consumableNotifyingPartitionWriters = ConsumableNotifyingResultPartitionWriterDecorator.decorate( resultPartitionDeploymentDescriptors, @@ -383,14 +383,14 @@ public Task( resultPartitionConsumableNotifier); // consumed intermediate result partitions - InputGate[] gates = networkEnvironment.createInputGates( + final InputGate[] gates = shuffleEnvironment.createInputGates( taskNameWithSubtaskAndId, executionId, this, inputGateDeploymentDescriptors, metrics.getIOMetricGroup(), inputGroup, - buffersGroup); + buffersGroup).toArray(new InputGate[] {}); this.inputGates = new InputGate[gates.length]; int counter = 0; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java index c300e5d6de664..948d4c83c15ea 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ConfigurationParserUtils.java @@ -23,8 +23,11 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; +import org.apache.flink.runtime.memory.MemoryManager; +import org.apache.flink.util.MathUtils; import static org.apache.flink.configuration.MemorySize.MemoryUnit.MEGA_BYTES; +import static org.apache.flink.util.MathUtils.checkedDownCast; /** * Utility class to extract related parameters from {@link Configuration} and to @@ -131,4 +134,30 @@ public static void checkConfigParameter(boolean condition, Object parameter, Str name + " : " + parameter + " - " + errorMessage); } } + + /** + * Parses the configuration to get the page size and validates the value. + * + * @param configuration configuration object + * @return size of memory segment + */ + public static int getPageSize(Configuration configuration) { + final int pageSize = checkedDownCast(MemorySize.parse( + configuration.getString(TaskManagerOptions.MEMORY_SEGMENT_SIZE)).getBytes()); + + // check page size of for minimum size + checkConfigParameter( + pageSize >= MemoryManager.MIN_PAGE_SIZE, + pageSize, + TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(), + "Minimum memory segment size is " + MemoryManager.MIN_PAGE_SIZE); + // check page size for power of two + checkConfigParameter( + MathUtils.isPowerOf2(pageSize), + pageSize, + TaskManagerOptions.MEMORY_SEGMENT_SIZE.key(), + "Memory segment size must be a power of 2."); + + return pageSize; + } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java index c6e50fa5f9c24..c49f8e6240707 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/clusterframework/ContaineredTaskManagerParametersTest.java @@ -21,7 +21,7 @@ import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ResourceManagerOptions; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -52,7 +52,7 @@ public void testOffHeapMemoryWithDefaultConfiguration() { ConfigConstants.DEFAULT_YARN_HEAP_CUTOFF); long cutoff = Math.max((long) (CONTAINER_MEMORY * memoryCutoffRatio), minCutoff); - final long networkBufMB = NetworkEnvironmentConfiguration.calculateNetworkBufferMemory( + final long networkBufMB = NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory( (CONTAINER_MEMORY - cutoff) << 20, // megabytes to bytes conf) >> 20; // bytes to megabytes assertEquals(networkBufMB + cutoff, params.taskManagerDirectMemoryLimitMB()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java similarity index 68% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java index 1950638d18096..9238ddad9d8c6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentBuilder.java @@ -24,12 +24,12 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; /** - * Builder for the {@link NetworkEnvironment}. + * Builder for the {@link NettyShuffleEnvironment}. */ -public class NetworkEnvironmentBuilder { +public class NettyShuffleEnvironmentBuilder { private int numNetworkBuffers = 1024; @@ -57,70 +57,69 @@ public class NetworkEnvironmentBuilder { private IOManager ioManager = new IOManagerAsync(); - public NetworkEnvironmentBuilder setTaskManagerLocation(ResourceID taskManagerLocation) { + public NettyShuffleEnvironmentBuilder setTaskManagerLocation(ResourceID taskManagerLocation) { this.taskManagerLocation = taskManagerLocation; return this; } - public NetworkEnvironmentBuilder setNumNetworkBuffers(int numNetworkBuffers) { + public NettyShuffleEnvironmentBuilder setNumNetworkBuffers(int numNetworkBuffers) { this.numNetworkBuffers = numNetworkBuffers; return this; } - public NetworkEnvironmentBuilder setNetworkBufferSize(int networkBufferSize) { + public NettyShuffleEnvironmentBuilder setNetworkBufferSize(int networkBufferSize) { this.networkBufferSize = networkBufferSize; return this; } - public NetworkEnvironmentBuilder setPartitionRequestInitialBackoff(int partitionRequestInitialBackoff) { + public NettyShuffleEnvironmentBuilder setPartitionRequestInitialBackoff(int partitionRequestInitialBackoff) { this.partitionRequestInitialBackoff = partitionRequestInitialBackoff; return this; } - public NetworkEnvironmentBuilder setPartitionRequestMaxBackoff(int partitionRequestMaxBackoff) { + public NettyShuffleEnvironmentBuilder setPartitionRequestMaxBackoff(int partitionRequestMaxBackoff) { this.partitionRequestMaxBackoff = partitionRequestMaxBackoff; return this; } - public NetworkEnvironmentBuilder setNetworkBuffersPerChannel(int networkBuffersPerChannel) { + public NettyShuffleEnvironmentBuilder setNetworkBuffersPerChannel(int networkBuffersPerChannel) { this.networkBuffersPerChannel = networkBuffersPerChannel; return this; } - public NetworkEnvironmentBuilder setFloatingNetworkBuffersPerGate(int floatingNetworkBuffersPerGate) { + public NettyShuffleEnvironmentBuilder setFloatingNetworkBuffersPerGate(int floatingNetworkBuffersPerGate) { this.floatingNetworkBuffersPerGate = floatingNetworkBuffersPerGate; return this; } - public NetworkEnvironmentBuilder setIsCreditBased(boolean isCreditBased) { + public NettyShuffleEnvironmentBuilder setIsCreditBased(boolean isCreditBased) { this.isCreditBased = isCreditBased; return this; } - public NetworkEnvironmentBuilder setNettyConfig(NettyConfig nettyConfig) { + public NettyShuffleEnvironmentBuilder setNettyConfig(NettyConfig nettyConfig) { this.nettyConfig = nettyConfig; return this; } - public NetworkEnvironmentBuilder setTaskEventDispatcher(TaskEventDispatcher taskEventDispatcher) { + public NettyShuffleEnvironmentBuilder setTaskEventDispatcher(TaskEventDispatcher taskEventDispatcher) { this.taskEventDispatcher = taskEventDispatcher; return this; } - public NetworkEnvironmentBuilder setMetricGroup(MetricGroup metricGroup) { + public NettyShuffleEnvironmentBuilder setMetricGroup(MetricGroup metricGroup) { this.metricGroup = metricGroup; return this; } - public NetworkEnvironmentBuilder setIOManager(IOManager ioManager) { + public NettyShuffleEnvironmentBuilder setIOManager(IOManager ioManager) { this.ioManager = ioManager; return this; } - public NetworkEnvironment build() { - return NetworkEnvironment.create( - taskManagerLocation, - new NetworkEnvironmentConfiguration( + public NettyShuffleEnvironment build() { + return NettyShuffleEnvironment.create( + new NettyShuffleEnvironmentConfiguration( numNetworkBuffers, networkBufferSize, partitionRequestInitialBackoff, @@ -130,6 +129,7 @@ public NetworkEnvironment build() { isCreditBased, isNetworkDetailedMetrics, nettyConfig), + taskManagerLocation, taskEventDispatcher, metricGroup, ioManager); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java similarity index 94% rename from flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java rename to flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java index 933f38556d2ec..d4f0727b346c1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NetworkEnvironmentTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/NettyShuffleEnvironmentTest.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.core.memory.MemorySegmentProvider; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.partition.ResultPartition; @@ -28,6 +28,7 @@ import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate; import org.apache.flink.runtime.io.network.partition.consumer.SingleInputGateBuilder; import org.apache.flink.runtime.taskmanager.Task; +import org.apache.flink.util.TestLogger; import org.junit.Rule; import org.junit.Test; @@ -47,10 +48,10 @@ import static org.powermock.api.mockito.PowerMockito.spy; /** - * Various tests for the {@link NetworkEnvironment} class. + * Various tests for the {@link NettyShuffleEnvironment} class. */ @RunWith(Parameterized.class) -public class NetworkEnvironmentTest { +public class NettyShuffleEnvironmentTest extends TestLogger { @Parameterized.Parameter public boolean enableCreditBasedFlowControl; @@ -69,7 +70,7 @@ public static List parameters() { */ @Test public void testRegisterTaskUsesBoundedBuffers() throws Exception { - final NetworkEnvironment network = new NetworkEnvironmentBuilder() + final NettyShuffleEnvironment network = new NettyShuffleEnvironmentBuilder() .setIsCreditBased(enableCreditBasedFlowControl) .build(); @@ -124,7 +125,7 @@ public void testRegisterTaskUsesBoundedBuffers() throws Exception { for (SingleInputGate ig : inputGates) { ig.close(); } - network.shutdown(); + network.close(); } /** @@ -141,7 +142,7 @@ public void testRegisterTaskWithLimitedBuffers() throws Exception { bufferCount = 20; } else { // incoming: 2 exclusive buffers per channel - bufferCount = 10 + 10 * NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue(); + bufferCount = 10 + 10 * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue(); } testRegisterTaskWithLimitedBuffers(bufferCount); @@ -160,7 +161,7 @@ public void testRegisterTaskWithInsufficientBuffers() throws Exception { bufferCount = 19; } else { // incoming: 2 exclusive buffers per channel - bufferCount = 10 + 10 * NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue() - 1; + bufferCount = 10 + 10 * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue() - 1; } expectedException.expect(IOException.class); @@ -169,7 +170,7 @@ public void testRegisterTaskWithInsufficientBuffers() throws Exception { } private void testRegisterTaskWithLimitedBuffers(int bufferPoolSize) throws Exception { - final NetworkEnvironment network = new NetworkEnvironmentBuilder() + final NettyShuffleEnvironment network = new NettyShuffleEnvironmentBuilder() .setNumNetworkBuffers(bufferPoolSize) .setIsCreditBased(enableCreditBasedFlowControl) .build(); @@ -245,7 +246,7 @@ private void testRegisterTaskWithLimitedBuffers(int bufferPoolSize) throws Excep for (SingleInputGate ig : inputGates) { ig.close(); } - network.shutdown(); + network.close(); } /** @@ -262,7 +263,7 @@ private void testRegisterTaskWithLimitedBuffers(int bufferPoolSize) throws Excep * @return input gate with some fake settings */ private SingleInputGate createSingleInputGate( - NetworkEnvironment network, ResultPartitionType partitionType, int numberOfChannels) { + NettyShuffleEnvironment network, ResultPartitionType partitionType, int numberOfChannels) { return spy(new SingleInputGateBuilder() .setNumberOfChannels(numberOfChannels) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java index 094426dfbe906..e8b079c84a1be 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/netty/NettyConnectionManagerTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.netty; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.ResultPartitionManager; import org.apache.flink.util.NetUtils; @@ -111,9 +111,9 @@ public void testManualConfiguration() throws Exception { // Expected number of threads Configuration flinkConfig = new Configuration(); - flinkConfig.setInteger(NetworkEnvironmentOptions.NUM_ARENAS, numberOfArenas); - flinkConfig.setInteger(NetworkEnvironmentOptions.NUM_THREADS_CLIENT, 3); - flinkConfig.setInteger(NetworkEnvironmentOptions.NUM_THREADS_SERVER, 4); + flinkConfig.setInteger(NettyShuffleEnvironmentOptions.NUM_ARENAS, numberOfArenas); + flinkConfig.setInteger(NettyShuffleEnvironmentOptions.NUM_THREADS_CLIENT, 3); + flinkConfig.setInteger(NettyShuffleEnvironmentOptions.NUM_THREADS_SERVER, 4); NettyConfig config = new NettyConfig( InetAddress.getLocalHost(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java index a6efa79a636a1..12ab07754f09a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartialConsumePipelinedResultTest.java @@ -20,7 +20,7 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.Buffer; @@ -62,7 +62,7 @@ public class PartialConsumePipelinedResultTest extends TestLogger { private static Configuration getFlinkConfiguration() { final Configuration config = new Configuration(); config.setString(AkkaOptions.ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT()); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, NUMBER_OF_NETWORK_BUFFERS); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, NUMBER_OF_NETWORK_BUFFERS); return config; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java index df07db73af761..ace58fa6b8fbd 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/PartitionTestUtils.java @@ -18,7 +18,7 @@ package org.apache.flink.runtime.io.network.partition; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; /** * This class should consolidate all mocking logic for ResultPartitions. @@ -36,11 +36,11 @@ public static ResultPartition createPartition(ResultPartitionType type) { } public static ResultPartition createPartition( - NetworkEnvironment environment, + NettyShuffleEnvironment environment, ResultPartitionType partitionType, int numChannels) { return new ResultPartitionBuilder() - .setupBufferPoolFactoryFromNetworkEnvironment(environment) + .setupBufferPoolFactoryFromNettyShuffleEnvironment(environment) .setResultPartitionType(partitionType) .setNumberOfSubpartitions(numChannels) .build(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java index f34a5981d4e06..45b25c0645872 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionBuilder.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.BufferPoolOwner; import org.apache.flink.runtime.io.network.buffer.NetworkBufferPool; @@ -85,7 +85,7 @@ public ResultPartitionBuilder setIOManager(IOManager ioManager) { return this; } - public ResultPartitionBuilder setupBufferPoolFactoryFromNetworkEnvironment(NetworkEnvironment environment) { + public ResultPartitionBuilder setupBufferPoolFactoryFromNettyShuffleEnvironment(NettyShuffleEnvironment environment) { return setNetworkBuffersPerChannel(environment.getConfiguration().networkBuffersPerChannel()) .setFloatingNetworkBuffersPerGate(environment.getConfiguration().floatingNetworkBuffersPerGate()) .setNetworkBufferPool(environment.getNetworkBufferPool()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java index 36a98fffea344..75459880a60a8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/ResultPartitionTest.java @@ -20,8 +20,8 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; import org.apache.flink.runtime.io.network.buffer.BufferBuilder; import org.apache.flink.runtime.io.network.buffer.BufferBuilderTestUtils; @@ -231,7 +231,7 @@ public void testReleaseMemoryOnPipelinedPartition() throws Exception { */ private void testReleaseMemory(final ResultPartitionType resultPartitionType) throws Exception { final int numAllBuffers = 10; - final NetworkEnvironment network = new NetworkEnvironmentBuilder() + final NettyShuffleEnvironment network = new NettyShuffleEnvironmentBuilder() .setNumNetworkBuffers(numAllBuffers).build(); final ResultPartition resultPartition = createPartition(network, resultPartitionType, 1); try { @@ -258,7 +258,7 @@ private void testReleaseMemory(final ResultPartitionType resultPartitionType) th } } finally { resultPartition.release(); - network.shutdown(); + network.close(); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java index 2e6fcbc6d8f4f..d8e9cc23ce747 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputChannelBuilder.java @@ -22,7 +22,7 @@ import org.apache.flink.runtime.io.network.ConnectionID; import org.apache.flink.runtime.io.network.ConnectionManager; import org.apache.flink.runtime.io.network.LocalConnectionManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.TaskEventPublisher; import org.apache.flink.runtime.io.network.metrics.InputChannelMetrics; @@ -99,7 +99,7 @@ public InputChannelBuilder setMemorySegmentProvider(MemorySegmentProvider memory return this; } - InputChannelBuilder setupFromNetworkEnvironment(NetworkEnvironment network) { + InputChannelBuilder setupFromNettyShuffleEnvironment(NettyShuffleEnvironment network) { this.partitionManager = network.getResultPartitionManager(); this.connectionManager = network.getConnectionManager(); this.initialBackoff = network.getConfiguration().partitionRequestInitialBackoff(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java index 9f5a4a2f3d122..b32ba89fd20b3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/InputGateTestBase.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.io.network.partition.consumer; import org.apache.flink.runtime.io.AsyncDataInput; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.junit.runner.RunWith; @@ -98,7 +98,7 @@ protected SingleInputGate createInputGate(int numberOfInputChannels) { } protected SingleInputGate createInputGate( - NetworkEnvironment environment, int numberOfInputChannels, ResultPartitionType partitionType) { + NettyShuffleEnvironment environment, int numberOfInputChannels, ResultPartitionType partitionType) { SingleInputGateBuilder builder = new SingleInputGateBuilder() .setNumberOfChannels(numberOfInputChannels) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java index 8c04d5f60b85d..b23af06f2d457 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateBuilder.java @@ -18,13 +18,13 @@ package org.apache.flink.runtime.io.network.partition.consumer; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider; import org.apache.flink.runtime.io.network.partition.PartitionProducerStateProvider.ResponseHandle; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.function.SupplierWithException; import java.io.IOException; @@ -75,8 +75,8 @@ public SingleInputGateBuilder setIsCreditBased(boolean isCreditBased) { return this; } - public SingleInputGateBuilder setupBufferPoolFactory(NetworkEnvironment environment) { - NetworkEnvironmentConfiguration config = environment.getConfiguration(); + public SingleInputGateBuilder setupBufferPoolFactory(NettyShuffleEnvironment environment) { + NettyShuffleEnvironmentConfiguration config = environment.getConfiguration(); this.bufferPoolFactory = SingleInputGateFactory.createBufferPoolFactory( environment.getNetworkBufferPool(), config.isCreditBased(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java index 11c6a51bc3cae..ebc9888165df0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/io/network/partition/consumer/SingleInputGateTest.java @@ -24,8 +24,8 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.event.TaskEvent; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.buffer.BufferPool; import org.apache.flink.runtime.io.network.buffer.FreeingBufferRecycler; @@ -346,7 +346,7 @@ public void testRequestBackoffConfiguration() throws Exception { int initialBackoff = 137; int maxBackoff = 1001; - final NetworkEnvironment netEnv = new NetworkEnvironmentBuilder() + final NettyShuffleEnvironment netEnv = new NettyShuffleEnvironmentBuilder() .setPartitionRequestInitialBackoff(initialBackoff) .setPartitionRequestMaxBackoff(maxBackoff) .setIsCreditBased(enableCreditBasedFlowControl) @@ -401,7 +401,7 @@ public void testRequestBackoffConfiguration() throws Exception { } } finally { gate.close(); - netEnv.shutdown(); + netEnv.close(); } } @@ -410,7 +410,7 @@ public void testRequestBackoffConfiguration() throws Exception { */ @Test public void testRequestBuffersWithRemoteInputChannel() throws Exception { - final NetworkEnvironment network = createNetworkEnvironment(); + final NettyShuffleEnvironment network = createNettyShuffleEnvironment(); final SingleInputGate inputGate = createInputGate(network, 1, ResultPartitionType.PIPELINED_BOUNDED); int buffersPerChannel = 2; int extraNetworkBuffersPerGate = 8; @@ -418,7 +418,7 @@ public void testRequestBuffersWithRemoteInputChannel() throws Exception { try { RemoteInputChannel remote = InputChannelBuilder.newBuilder() - .setupFromNetworkEnvironment(network) + .setupFromNettyShuffleEnvironment(network) .buildRemoteAndSetToGate(inputGate); inputGate.setup(); @@ -436,7 +436,7 @@ public void testRequestBuffersWithRemoteInputChannel() throws Exception { } } finally { inputGate.close(); - network.shutdown(); + network.close(); } } @@ -446,7 +446,7 @@ public void testRequestBuffersWithRemoteInputChannel() throws Exception { */ @Test public void testRequestBuffersWithUnknownInputChannel() throws Exception { - final NetworkEnvironment network = createNetworkEnvironment(); + final NettyShuffleEnvironment network = createNettyShuffleEnvironment(); final SingleInputGate inputGate = createInputGate(network, 1, ResultPartitionType.PIPELINED_BOUNDED); int buffersPerChannel = 2; int extraNetworkBuffersPerGate = 8; @@ -487,7 +487,7 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { } } finally { inputGate.close(); - network.shutdown(); + network.close(); } } @@ -497,7 +497,7 @@ public void testRequestBuffersWithUnknownInputChannel() throws Exception { */ @Test public void testUpdateUnknownInputChannel() throws Exception { - final NetworkEnvironment network = createNetworkEnvironment(); + final NettyShuffleEnvironment network = createNettyShuffleEnvironment(); final SingleInputGate inputGate = createInputGate(network, 2, ResultPartitionType.PIPELINED); try { @@ -537,7 +537,7 @@ public void testUpdateUnknownInputChannel() throws Exception { is(instanceOf((LocalInputChannel.class)))); } finally { inputGate.close(); - network.shutdown(); + network.close(); } } @@ -564,8 +564,8 @@ public void testPartitionNotFoundExceptionWhileGetNextBuffer() throws Exception } @Test - public void testInputGateRemovalFromNetworkEnvironment() throws Exception { - NetworkEnvironment network = createNetworkEnvironment(); + public void testInputGateRemovalFromNettyShuffleEnvironment() throws Exception { + NettyShuffleEnvironment network = createNettyShuffleEnvironment(); try { int numberOfGates = 10; @@ -580,14 +580,14 @@ public void testInputGateRemovalFromNetworkEnvironment() throws Exception { assertThat(network.getInputGate(id).isPresent(), is(false)); } } finally { - network.shutdown(); + network.close(); } } // --------------------------------------------------------------------------------------------- private static Map createInputGateWithLocalChannels( - NetworkEnvironment network, + NettyShuffleEnvironment network, int numberOfGates, @SuppressWarnings("SameParameterValue") int numberOfLocalChannels) { ShuffleDescriptor[] channelDescs = new NettyShuffleDescriptor[numberOfLocalChannels]; @@ -614,7 +614,7 @@ private static Map createInputGateWithLocalChannel Arrays.asList(gateDescs), new UnregisteredMetricsGroup(), new UnregisteredMetricsGroup(), - new UnregisteredMetricsGroup()); + new UnregisteredMetricsGroup()).toArray(new SingleInputGate[] {}); Map inputGatesById = new HashMap<>(); for (int i = 0; i < numberOfGates; i++) { inputGatesById.put(new InputGateID(ids[i], consumerID), gates[i]); @@ -624,19 +624,19 @@ private static Map createInputGateWithLocalChannel } private void addUnknownInputChannel( - NetworkEnvironment network, + NettyShuffleEnvironment network, SingleInputGate inputGate, ResultPartitionID partitionId, int channelIndex) { InputChannelBuilder.newBuilder() .setChannelIndex(channelIndex) .setPartitionId(partitionId) - .setupFromNetworkEnvironment(network) + .setupFromNettyShuffleEnvironment(network) .buildUnknownAndSetToGate(inputGate); } - private NetworkEnvironment createNetworkEnvironment() { - return new NetworkEnvironmentBuilder() + private NettyShuffleEnvironment createNettyShuffleEnvironment() { + return new NettyShuffleEnvironmentBuilder() .setIsCreditBased(enableCreditBasedFlowControl) .build(); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java index c4c74310ccfa5..d9eac20378525 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManagerTest.java @@ -205,7 +205,12 @@ private void checkRootDirsClean(File[] rootDirs) { private TaskManagerServicesConfiguration createTaskManagerServiceConfiguration( Configuration config) throws IOException { return TaskManagerServicesConfiguration.fromConfiguration( - config, MEM_SIZE_PARAM, InetAddress.getLocalHost(), true); + config, + ResourceID.generate(), + InetAddress.getLocalHost(), + MEM_SIZE_PARAM, + MEM_SIZE_PARAM, + true); } private TaskManagerServices createTaskManagerServices( @@ -213,9 +218,6 @@ private TaskManagerServices createTaskManagerServices( return TaskManagerServices.fromConfiguration( config, UnregisteredMetricGroups.createUnregisteredTaskManagerMetricGroup(), - ResourceID.generate(), - Executors.directExecutor(), - MEM_SIZE_PARAM, - MEM_SIZE_PARAM); + Executors.directExecutor()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java new file mode 100644 index 0000000000000..1aa481739e12d --- /dev/null +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NettyShuffleEnvironmentConfigurationTest.java @@ -0,0 +1,334 @@ +/* + * 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.taskexecutor; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; +import org.apache.flink.util.TestLogger; + +import org.junit.Test; + +import java.net.InetAddress; +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Unit test for {@link NettyShuffleEnvironmentConfiguration}. + */ +public class NettyShuffleEnvironmentConfigurationTest extends TestLogger { + + private static final long MEM_SIZE_PARAM = 128L * 1024 * 1024; + + /** + * Verifies that {@link NettyShuffleEnvironmentConfiguration#fromConfiguration(Configuration, long, boolean, InetAddress)} + * returns the correct result for new configurations via + * {@link NettyShuffleEnvironmentOptions#NETWORK_REQUEST_BACKOFF_INITIAL}, + * {@link NettyShuffleEnvironmentOptions#NETWORK_REQUEST_BACKOFF_MAX}, + * {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_PER_CHANNEL} and + * {@link NettyShuffleEnvironmentOptions#NETWORK_EXTRA_BUFFERS_PER_GATE} + */ + @Test + public void testNetworkRequestBackoffAndBuffers() { + + // set some non-default values + final Configuration config = new Configuration(); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL, 10); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_EXTRA_BUFFERS_PER_GATE, 100); + + final NettyShuffleEnvironmentConfiguration networkConfig = NettyShuffleEnvironmentConfiguration.fromConfiguration( + config, + MEM_SIZE_PARAM, + true, + InetAddress.getLoopbackAddress()); + + assertEquals(networkConfig.partitionRequestInitialBackoff(), 100); + assertEquals(networkConfig.partitionRequestMaxBackoff(), 200); + assertEquals(networkConfig.networkBuffersPerChannel(), 10); + assertEquals(networkConfig.floatingNetworkBuffersPerGate(), 100); + } + + /** + * Test for {@link NettyShuffleEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using old + * configurations via {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS}. + */ + @SuppressWarnings("deprecation") + @Test + public void calculateNetworkBufOld() { + Configuration config = new Configuration(); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); + + // note: actual network buffer memory size is independent of the totalJavaMemorySize + assertEquals(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(10L << 20, config)); + assertEquals(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(64L << 20, config)); + + // test integer overflow in the memory size + int numBuffers = (int) ((2L << 32) / MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes()); // 2^33 + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, numBuffers); + assertEquals(2L << 32, NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(2L << 33, config)); + } + + /** + * Test for {@link NettyShuffleEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using new + * configurations via {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION}, + * {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} and + * {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}. + */ + @Test + public void calculateNetworkBufNew() throws Exception { + Configuration config = new Configuration(); + + // (1) defaults + final Float defaultFrac = NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.defaultValue(); + final Long defaultMin = MemorySize.parse(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes(); + final Long defaultMax = MemorySize.parse(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.defaultValue()).getBytes(); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), defaultMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config)); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), defaultMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config)); + + calculateNetworkBufNew(config); + } + + /** + * Helper to test {@link NettyShuffleEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} with the + * new configuration parameters. + * + * @param config configuration object + */ + private static void calculateNetworkBufNew(final Configuration config) { + // (2) fixed size memory + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(1L << 20)); // 1MB + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(1L << 20)); // 1MB + + + // note: actual network buffer memory size is independent of the totalJavaMemorySize + assertEquals(1 << 20, NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(10L << 20, config)); + assertEquals(1 << 20, NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(64L << 20, config)); + assertEquals(1 << 20, NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(1L << 30, config)); + + // (3) random fraction, min, and max values + Random ran = new Random(); + for (int i = 0; i < 1_000; ++i){ + float frac = Math.max(ran.nextFloat(), Float.MIN_VALUE); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, frac); + + long min = Math.max(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), ran.nextLong()); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(min)); + + + long max = Math.max(min, ran.nextLong()); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(max)); + + long javaMem = Math.max(max + 1, ran.nextLong()); + + final long networkBufMem = NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory(javaMem, config); + + if (networkBufMem < min) { + fail("Lower bound not met with configuration: " + config.toString()); + } + + if (networkBufMem > max) { + fail("Upper bound not met with configuration: " + config.toString()); + } + + if (networkBufMem > min && networkBufMem < max) { + if ((javaMem * frac) != networkBufMem) { + fail("Wrong network buffer memory size with configuration: " + config.toString() + + ". Expected value: " + (javaMem * frac) + " actual value: " + networkBufMem + '.'); + } + } + } + } + + /** + * Test for {@link NettyShuffleEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using mixed + * old/new configurations. + */ + @SuppressWarnings("deprecation") + @Test + public void calculateNetworkBufMixed() throws Exception { + Configuration config = new Configuration(); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); + + final Float defaultFrac = NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.defaultValue(); + final Long defaultMin = MemorySize.parse(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes(); + final Long defaultMax = MemorySize.parse(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.defaultValue()).getBytes(); + + + // old + 1 new parameter = new: + Configuration config1 = config.clone(); + config1.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + assertEquals(enforceBounds((long) (0.1f * (10L << 20)), defaultMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config1)); + assertEquals(enforceBounds((long) (0.1f * (10L << 30)), defaultMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); + + config1 = config.clone(); + long newMin = MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(); // smallest value possible + config1.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(newMin)); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), newMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 20), config1)); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), newMin, defaultMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); + + config1 = config.clone(); + long newMax = Math.max(64L << 20 + 1, MemorySize.parse(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes()); + config1.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(newMax)); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), defaultMin, newMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config1)); + assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), defaultMin, newMax), + NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config1)); + + // old + any new parameter = new: + calculateNetworkBufNew(config); + } + + /** + * Returns the value or the lower/upper bound in case the value is less/greater than the lower/upper bound, respectively. + * + * @param value value to inspect + * @param lower lower bound + * @param upper upper bound + * + * @return min(upper, max(lower, value)) + */ + private static long enforceBounds(final long value, final long lower, final long upper) { + return Math.min(upper, Math.max(lower, value)); + } + + /** + * Test for {@link TaskManagerServices#calculateHeapSizeMB(long, Configuration)} with some + * manually calculated scenarios. + */ + @Test + public void calculateHeapSizeMB() throws Exception { + Configuration config = new Configuration(); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(64L << 20)); // 64MB + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(1L << 30)); // 1GB + + config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, false); + assertEquals(900, TaskManagerServices.calculateHeapSizeMB(1000, config)); + + config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, false); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.2f); + assertEquals(800, TaskManagerServices.calculateHeapSizeMB(1000, config)); + + config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, true); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "10m"); // 10MB + assertEquals(890, TaskManagerServices.calculateHeapSizeMB(1000, config)); + + config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); // use fraction of given memory + config.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, 0.1f); // 10% + assertEquals(810, TaskManagerServices.calculateHeapSizeMB(1000, config)); + } + + /** + * Verifies that {@link NettyShuffleEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} + * returns the correct result for old configurations via + * {@link NettyShuffleEnvironmentOptions#NETWORK_NUM_BUFFERS}. + */ + @SuppressWarnings("deprecation") + @Test + public void hasNewNetworkBufConfOld() throws Exception { + Configuration config = new Configuration(); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); + + assertFalse(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + } + + /** + * Verifies that {@link NettyShuffleEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} + * returns the correct result for new configurations via + * {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION}, + * {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} and {@link + * NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}. + */ + @Test + public void hasNewNetworkBufConfNew() throws Exception { + Configuration config = new Configuration(); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + // fully defined: + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "2048"); + + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + // partly defined: + config = new Configuration(); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + config = new Configuration(); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + config.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + config = new Configuration(); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + } + + /** + * Verifies that {@link NettyShuffleEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} + * returns the correct result for mixed old/new configurations. + */ + @SuppressWarnings("deprecation") + @Test + public void hasNewNetworkBufConfMixed() throws Exception { + Configuration config = new Configuration(); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); + assertFalse(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config)); + + // old + 1 new parameter = new: + Configuration config1 = config.clone(); + config1.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config1)); + + config1 = config.clone(); + config1.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config1)); + + config1 = config.clone(); + config1.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); + assertTrue(NettyShuffleEnvironmentConfiguration.hasNewNetworkConfig(config1)); + } +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java index e6736fe5029e5..39b9d7407ced1 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkBufferCalculationTest.java @@ -21,8 +21,8 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; -import org.apache.flink.configuration.NetworkEnvironmentOptions; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -35,8 +35,8 @@ public class NetworkBufferCalculationTest extends TestLogger { /** - * Test for {@link NetworkEnvironmentConfiguration#calculateNewNetworkBufferMemory(Configuration, long)} - * using the same (manual) test cases as in {@link NetworkEnvironmentConfigurationTest#calculateHeapSizeMB()}. + * Test for {@link NettyShuffleEnvironmentConfiguration#calculateNewNetworkBufferMemory(Configuration, long)} + * using the same (manual) test cases as in {@link NettyShuffleEnvironmentConfigurationTest#calculateHeapSizeMB()}. */ @Test public void calculateNetworkBufFromHeapSize() { @@ -47,23 +47,23 @@ public void calculateNetworkBufFromHeapSize() { TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), 0.1f, 60L << 20, 1L << 30, MemoryType.HEAP); assertEquals((100L << 20) + 1 /* one too many due to floating point imprecision */, - NetworkEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 900L << 20)); // 900MB + NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 900L << 20)); // 900MB config = getConfig( Long.valueOf(TaskManagerOptions.MANAGED_MEMORY_SIZE.defaultValue()), TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), 0.2f, 60L << 20, 1L << 30, MemoryType.HEAP); assertEquals((200L << 20) + 3 /* slightly too many due to floating point imprecision */, - NetworkEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 800L << 20)); // 800MB + NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 800L << 20)); // 800MB config = getConfig(10, TaskManagerOptions.MANAGED_MEMORY_FRACTION.defaultValue(), 0.1f, 60L << 20, 1L << 30, MemoryType.OFF_HEAP); assertEquals((100L << 20) + 1 /* one too many due to floating point imprecision */, - NetworkEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 890L << 20)); // 890MB + NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 890L << 20)); // 890MB config = getConfig(0, 0.1f, 0.1f, 60L << 20, 1L << 30, MemoryType.OFF_HEAP); assertEquals((100L << 20) + 1 /* one too many due to floating point imprecision */, - NetworkEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 810L << 20)); // 810MB + NettyShuffleEnvironmentConfiguration.calculateNewNetworkBufferMemory(config, 810L << 20)); // 810MB } /** @@ -71,9 +71,9 @@ public void calculateNetworkBufFromHeapSize() { * * @param managedMemory see {@link TaskManagerOptions#MANAGED_MEMORY_SIZE} * @param managedMemoryFraction see {@link TaskManagerOptions#MANAGED_MEMORY_FRACTION} - * @param networkBufFraction see {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION} - * @param networkBufMin see {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} - * @param networkBufMax see {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX} + * @param networkBufFraction see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION} + * @param networkBufMin see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} + * @param networkBufMax see {@link NettyShuffleEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX} * @param memoryType on-heap or off-heap * * @return configuration object @@ -90,9 +90,9 @@ private static Configuration getConfig( configuration.setLong(TaskManagerOptions.MANAGED_MEMORY_SIZE.key(), managedMemory); configuration.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION.key(), managedMemoryFraction); - configuration.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), networkBufFraction); - configuration.setLong(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), networkBufMin); - configuration.setLong(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), networkBufMax); + configuration.setFloat(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.key(), networkBufFraction); + configuration.setLong(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.key(), networkBufMin); + configuration.setLong(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.key(), networkBufMax); configuration.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP.key(), memoryType == MemoryType.OFF_HEAP); return configuration; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkEnvironmentConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkEnvironmentConfigurationTest.java deleted file mode 100644 index 47f4734ebfbcf..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/NetworkEnvironmentConfigurationTest.java +++ /dev/null @@ -1,301 +0,0 @@ -/* - * 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.taskexecutor; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.NetworkEnvironmentOptions; -import org.apache.flink.configuration.TaskManagerOptions; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.util.Random; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * Unit test for {@link TaskManagerServices}. - */ -public class NetworkEnvironmentConfigurationTest extends TestLogger { - - /** - * Test for {@link NetworkEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using old - * configurations via {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS}. - */ - @SuppressWarnings("deprecation") - @Test - public void calculateNetworkBufOld() { - Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); - - // note: actual network buffer memory size is independent of the totalJavaMemorySize - assertEquals(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(10L << 20, config)); - assertEquals(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(64L << 20, config)); - - // test integer overflow in the memory size - int numBuffers = (int) ((2L << 32) / MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes()); // 2^33 - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, numBuffers); - assertEquals(2L << 32, NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(2L << 33, config)); - } - - /** - * Test for {@link NetworkEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using new - * configurations via {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION}, - * {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} and - * {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}. - */ - @Test - public void calculateNetworkBufNew() throws Exception { - Configuration config = new Configuration(); - - // (1) defaults - final Float defaultFrac = NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.defaultValue(); - final Long defaultMin = MemorySize.parse(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes(); - final Long defaultMax = MemorySize.parse(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.defaultValue()).getBytes(); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), defaultMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config)); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), defaultMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config)); - - calculateNetworkBufNew(config); - } - - /** - * Helper to test {@link NetworkEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} with the - * new configuration parameters. - * - * @param config configuration object - */ - private static void calculateNetworkBufNew(final Configuration config) { - // (2) fixed size memory - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(1L << 20)); // 1MB - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(1L << 20)); // 1MB - - - // note: actual network buffer memory size is independent of the totalJavaMemorySize - assertEquals(1 << 20, NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(10L << 20, config)); - assertEquals(1 << 20, NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(64L << 20, config)); - assertEquals(1 << 20, NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(1L << 30, config)); - - // (3) random fraction, min, and max values - Random ran = new Random(); - for (int i = 0; i < 1_000; ++i){ - float frac = Math.max(ran.nextFloat(), Float.MIN_VALUE); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, frac); - - long min = Math.max(MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(), ran.nextLong()); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(min)); - - - long max = Math.max(min, ran.nextLong()); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(max)); - - long javaMem = Math.max(max + 1, ran.nextLong()); - - final long networkBufMem = NetworkEnvironmentConfiguration.calculateNetworkBufferMemory(javaMem, config); - - if (networkBufMem < min) { - fail("Lower bound not met with configuration: " + config.toString()); - } - - if (networkBufMem > max) { - fail("Upper bound not met with configuration: " + config.toString()); - } - - if (networkBufMem > min && networkBufMem < max) { - if ((javaMem * frac) != networkBufMem) { - fail("Wrong network buffer memory size with configuration: " + config.toString() + - ". Expected value: " + (javaMem * frac) + " actual value: " + networkBufMem + '.'); - } - } - } - } - - /** - * Test for {@link NetworkEnvironmentConfiguration#calculateNetworkBufferMemory(long, Configuration)} using mixed - * old/new configurations. - */ - @SuppressWarnings("deprecation") - @Test - public void calculateNetworkBufMixed() throws Exception { - Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); - - final Float defaultFrac = NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION.defaultValue(); - final Long defaultMin = MemorySize.parse(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes(); - final Long defaultMax = MemorySize.parse(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX.defaultValue()).getBytes(); - - - // old + 1 new parameter = new: - Configuration config1 = config.clone(); - config1.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - assertEquals(enforceBounds((long) (0.1f * (10L << 20)), defaultMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config1)); - assertEquals(enforceBounds((long) (0.1f * (10L << 30)), defaultMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); - - config1 = config.clone(); - long newMin = MemorySize.parse(TaskManagerOptions.MEMORY_SEGMENT_SIZE.defaultValue()).getBytes(); // smallest value possible - config1.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(newMin)); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), newMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 20), config1)); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), newMin, defaultMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); - - config1 = config.clone(); - long newMax = Math.max(64L << 20 + 1, MemorySize.parse(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN.defaultValue()).getBytes()); - config1.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(newMax)); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 20)), defaultMin, newMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((64L << 20 + 1), config1)); - assertEquals(enforceBounds((long) (defaultFrac * (10L << 30)), defaultMin, newMax), - NetworkEnvironmentConfiguration.calculateNetworkBufferMemory((10L << 30), config1)); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config1)); - - // old + any new parameter = new: - calculateNetworkBufNew(config); - } - - /** - * Returns the value or the lower/upper bound in case the value is less/greater than the lower/upper bound, respectively. - * - * @param value value to inspect - * @param lower lower bound - * @param upper upper bound - * - * @return min(upper, max(lower, value)) - */ - private static long enforceBounds(final long value, final long lower, final long upper) { - return Math.min(upper, Math.max(lower, value)); - } - - /** - * Test for {@link TaskManagerServices#calculateHeapSizeMB(long, Configuration)} with some - * manually calculated scenarios. - */ - @Test - public void calculateHeapSizeMB() throws Exception { - Configuration config = new Configuration(); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(64L << 20)); // 64MB - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(1L << 30)); // 1GB - - config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, false); - assertEquals(900, TaskManagerServices.calculateHeapSizeMB(1000, config)); - - config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, false); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.2f); - assertEquals(800, TaskManagerServices.calculateHeapSizeMB(1000, config)); - - config.setBoolean(TaskManagerOptions.MEMORY_OFF_HEAP, true); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "10m"); // 10MB - assertEquals(890, TaskManagerServices.calculateHeapSizeMB(1000, config)); - - config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "0"); // use fraction of given memory - config.setFloat(TaskManagerOptions.MANAGED_MEMORY_FRACTION, 0.1f); // 10% - assertEquals(810, TaskManagerServices.calculateHeapSizeMB(1000, config)); - } - - /** - * Verifies that {@link NetworkEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} - * returns the correct result for old configurations via - * {@link NetworkEnvironmentOptions#NETWORK_NUM_BUFFERS}. - */ - @SuppressWarnings("deprecation") - @Test - public void hasNewNetworkBufConfOld() throws Exception { - Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); - - assertFalse(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - } - - /** - * Verifies that {@link NetworkEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} - * returns the correct result for new configurations via - * {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_FRACTION}, - * {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MIN} and {@link - * NetworkEnvironmentOptions#NETWORK_BUFFERS_MEMORY_MAX}. - */ - @Test - public void hasNewNetworkBufConfNew() throws Exception { - Configuration config = new Configuration(); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - // fully defined: - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "2048"); - - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - // partly defined: - config = new Configuration(); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - config = new Configuration(); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - config.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - config = new Configuration(); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - } - - /** - * Verifies that {@link NetworkEnvironmentConfiguration#hasNewNetworkConfig(Configuration)} - * returns the correct result for mixed old/new configurations. - */ - @SuppressWarnings("deprecation") - @Test - public void hasNewNetworkBufConfMixed() throws Exception { - Configuration config = new Configuration(); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 1); - assertFalse(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config)); - - // old + 1 new parameter = new: - Configuration config1 = config.clone(); - config1.setFloat(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_FRACTION, 0.1f); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config1)); - - config1 = config.clone(); - config1.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config1)); - - config1 = config.clone(); - config1.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, "1024"); - assertTrue(NetworkEnvironmentConfiguration.hasNewNetworkConfig(config1)); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java index 8a21da9eca806..5511d1bd45b46 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorSubmissionTest.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.blob.PermanentBlobKey; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -37,7 +37,7 @@ import org.apache.flink.runtime.executiongraph.JobInformation; import org.apache.flink.runtime.executiongraph.PartitionInfo; import org.apache.flink.runtime.executiongraph.TaskInformation; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.partition.PartitionNotFoundException; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.IntermediateDataSetID; @@ -276,7 +276,7 @@ public void testRunJobWithForwardChannel() throws Exception { .addTaskManagerActionListener(eid2, ExecutionState.FINISHED, task2FinishedFuture) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) - .useRealNonMockNetworkEnvironment() + .useRealNonMockShuffleEnvironment() .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -342,7 +342,7 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { .addTaskManagerActionListener(eid2, ExecutionState.CANCELED, task2CanceledFuture) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) - .useRealNonMockNetworkEnvironment() + .useRealNonMockShuffleEnvironment() .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -372,9 +372,9 @@ public void testCancellingDependentAndStateUpdateFails() throws Exception { public void testRemotePartitionNotFound() throws Exception { final int dataPort = NetUtils.getAvailablePort(); Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.DATA_PORT, dataPort); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); + config.setInteger(NettyShuffleEnvironmentOptions.DATA_PORT, dataPort); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); // Remote location (on the same TM though) for the partition NettyShuffleDescriptor sdd = @@ -392,7 +392,7 @@ public void testRemotePartitionNotFound() throws Exception { .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) .setConfiguration(config) .setLocalCommunication(false) - .useRealNonMockNetworkEnvironment() + .useRealNonMockShuffleEnvironment() .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -417,11 +417,11 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { final CompletableFuture taskRunningFuture = new CompletableFuture<>(); final CompletableFuture taskFailedFuture = new CompletableFuture<>(); - final NetworkEnvironment networkEnvironment = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); + final ShuffleEnvironment shuffleEnvironment = mock(ShuffleEnvironment.class, Mockito.RETURNS_MOCKS); try (TaskSubmissionTestEnvironment env = new TaskSubmissionTestEnvironment.Builder(jobId) - .setNetworkEnvironment(networkEnvironment) + .setShuffleEnvironment(shuffleEnvironment) .setSlotSize(1) .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) @@ -437,7 +437,7 @@ public void testUpdateTaskInputPartitionsFailure() throws Exception { NettyShuffleDescriptor shuffleDescriptor = createRemoteWithIdAndLocation(new IntermediateResultPartitionID(), producerLocation); final PartitionInfo partitionUpdate = new PartitionInfo(new IntermediateDataSetID(), shuffleDescriptor); - doThrow(new IOException()).when(networkEnvironment).updatePartitionInfo(eid, partitionUpdate); + doThrow(new IOException()).when(shuffleEnvironment).updatePartitionInfo(eid, partitionUpdate); final CompletableFuture updateFuture = tmGateway.updatePartitions( eid, @@ -464,8 +464,8 @@ public void testLocalPartitionNotFound() throws Exception { ExecutionAttemptID eid = tdd.getExecutionAttemptId(); Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); final CompletableFuture taskRunningFuture = new CompletableFuture<>(); final CompletableFuture taskFailedFuture = new CompletableFuture<>(); @@ -477,7 +477,7 @@ public void testLocalPartitionNotFound() throws Exception { .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .addTaskManagerActionListener(eid, ExecutionState.FAILED, taskFailedFuture) .setConfiguration(config) - .useRealNonMockNetworkEnvironment() + .useRealNonMockShuffleEnvironment() .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); @@ -536,7 +536,7 @@ public void testFailingScheduleOrUpdateConsumers() throws Exception { .addTaskManagerActionListener(eid, ExecutionState.RUNNING, taskRunningFuture) .setJobMasterId(jobMasterId) .setJobMasterGateway(testingJobMasterGateway) - .useRealNonMockNetworkEnvironment() + .useRealNonMockShuffleEnvironment() .build()) { TaskExecutorGateway tmGateway = env.getTaskExecutorGateway(); TaskSlotTable taskSlotTable = env.getTaskSlotTable(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 4d0b4d5a02962..ff369e6cfc6c5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -25,7 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple4; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.memory.MemoryType; import org.apache.flink.core.testutils.OneShotLatch; @@ -57,8 +57,8 @@ import org.apache.flink.runtime.instance.InstanceID; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -197,7 +197,7 @@ public class TaskExecutorTest extends TestLogger { private SettableLeaderRetrievalService jobManagerLeaderRetriever; - private NetworkEnvironment networkEnvironment; + private NettyShuffleEnvironment nettyShuffleEnvironment; @Before public void setup() throws IOException { @@ -223,7 +223,7 @@ public void setup() throws IOException { haServices.setResourceManagerLeaderRetriever(resourceManagerLeaderRetriever); haServices.setJobMasterLeaderRetriever(jobId, jobManagerLeaderRetriever); - networkEnvironment = new NetworkEnvironmentBuilder().build(); + nettyShuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); } @After @@ -243,8 +243,8 @@ public void teardown() throws Exception { dummyBlobCacheService = null; } - if (networkEnvironment != null) { - networkEnvironment.shutdown(); + if (nettyShuffleEnvironment != null) { + nettyShuffleEnvironment.close(); } testingFatalErrorHandler.rethrowError(); @@ -270,7 +270,7 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { MemoryType.HEAP, false); - networkEnvironment.start(); + nettyShuffleEnvironment.start(); final KvStateService kvStateService = new KvStateService(new KvStateRegistry(), null, null); kvStateService.start(); @@ -279,7 +279,7 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { .setTaskManagerLocation(taskManagerLocation) .setMemoryManager(memoryManager) .setIoManager(ioManager) - .setNetworkEnvironment(networkEnvironment) + .setShuffleEnvironment(nettyShuffleEnvironment) .setKvStateService(kvStateService) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) @@ -295,7 +295,7 @@ public void testShouldShutDownTaskManagerServicesInPostStop() throws Exception { } assertThat(memoryManager.isShutdown(), is(true)); - assertThat(networkEnvironment.isShutdown(), is(true)); + assertThat(nettyShuffleEnvironment.isClosed(), is(true)); assertThat(ioManager.isProperlyShutDown(), is(true)); assertThat(kvStateService.isShutdown(), is(true)); } @@ -709,7 +709,7 @@ public void updateTaskExecutionState(TaskExecutionState taskExecutionState) { final TaskExecutorLocalStateStoresManager localStateStoresManager = createTaskExecutorLocalStateStoresManager(); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setNetworkEnvironment(networkEnvironment) + .setShuffleEnvironment(nettyShuffleEnvironment) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setTaskStateManager(localStateStoresManager) @@ -967,7 +967,7 @@ public void testSubmitTaskBeforeAcceptSlot() throws Exception { final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() .setTaskManagerLocation(taskManagerLocation) - .setNetworkEnvironment(networkEnvironment) + .setShuffleEnvironment(nettyShuffleEnvironment) .setTaskSlotTable(taskSlotTable) .setJobLeaderService(jobLeaderService) .setJobManagerTable(jobManagerTable) @@ -1627,9 +1627,9 @@ public void testDisconnectFromJobMasterWhenNewLeader() throws Exception { public void testLogNotFoundHandling() throws Throwable { final int dataPort = NetUtils.getAvailablePort(); Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.DATA_PORT, dataPort); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); + config.setInteger(NettyShuffleEnvironmentOptions.DATA_PORT, dataPort); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); config.setString(ConfigConstants.TASK_MANAGER_LOG_PATH_KEY, "/i/dont/exist"); try (TaskSubmissionTestEnvironment env = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java index f48949d11fdad..58ef6a8f0898a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerRunnerStartupTest.java @@ -22,7 +22,7 @@ import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.IllegalConfigurationException; import org.apache.flink.configuration.MemorySize; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -163,7 +163,7 @@ public void testStartupWhenNetworkStackFailsToInitialize() throws Exception { try { final Configuration cfg = new Configuration(); - cfg.setInteger(NetworkEnvironmentOptions.DATA_PORT, blocker.getLocalPort()); + cfg.setInteger(NettyShuffleEnvironmentOptions.DATA_PORT, blocker.getLocalPort()); startTaskManager( cfg, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java index 0238edf07ecfe..c763db65a8519 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesBuilder.java @@ -21,7 +21,7 @@ import org.apache.flink.core.memory.MemoryType; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.query.KvStateRegistry; @@ -42,7 +42,7 @@ public class TaskManagerServicesBuilder { private TaskManagerLocation taskManagerLocation; private MemoryManager memoryManager; private IOManager ioManager; - private NetworkEnvironment networkEnvironment; + private ShuffleEnvironment shuffleEnvironment; private KvStateService kvStateService; private BroadcastVariableManager broadcastVariableManager; private TaskSlotTable taskSlotTable; @@ -60,7 +60,7 @@ public TaskManagerServicesBuilder() { MemoryType.HEAP, false); ioManager = mock(IOManager.class); - networkEnvironment = mock(NetworkEnvironment.class); + shuffleEnvironment = mock(ShuffleEnvironment.class); kvStateService = new KvStateService(new KvStateRegistry(), null, null); broadcastVariableManager = new BroadcastVariableManager(); taskEventDispatcher = new TaskEventDispatcher(); @@ -85,8 +85,8 @@ public TaskManagerServicesBuilder setIoManager(IOManager ioManager) { return this; } - public TaskManagerServicesBuilder setNetworkEnvironment(NetworkEnvironment networkEnvironment) { - this.networkEnvironment = networkEnvironment; + public TaskManagerServicesBuilder setShuffleEnvironment(ShuffleEnvironment shuffleEnvironment) { + this.shuffleEnvironment = shuffleEnvironment; return this; } @@ -125,7 +125,7 @@ public TaskManagerServices build() { taskManagerLocation, memoryManager, ioManager, - networkEnvironment, + shuffleEnvironment, kvStateService, broadcastVariableManager, taskSlotTable, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java deleted file mode 100644 index d528612a22ab1..0000000000000 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskManagerServicesConfigurationTest.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * 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.taskexecutor; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; - -import java.net.InetAddress; - -import static org.junit.Assert.assertEquals; - -/** - * Unit test for {@link TaskManagerServicesConfiguration}. - */ -public class TaskManagerServicesConfigurationTest extends TestLogger { - - private static final long MEM_SIZE_PARAM = 128L * 1024 * 1024; - - /** - * Verifies that {@link TaskManagerServicesConfiguration#fromConfiguration(Configuration, long, InetAddress, boolean)} - * returns the correct result for new configurations via - * {@link NetworkEnvironmentOptions#NETWORK_REQUEST_BACKOFF_INITIAL}, - * {@link NetworkEnvironmentOptions#NETWORK_REQUEST_BACKOFF_MAX}, - * {@link NetworkEnvironmentOptions#NETWORK_BUFFERS_PER_CHANNEL} and - * {@link NetworkEnvironmentOptions#NETWORK_EXTRA_BUFFERS_PER_GATE} - */ - @Test - public void testNetworkRequestBackoffAndBuffers() throws Exception { - - // set some non-default values - final Configuration config = new Configuration(); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL, 100); - config.setInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX, 200); - config.setInteger(NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL, 10); - config.setInteger(NetworkEnvironmentOptions.NETWORK_EXTRA_BUFFERS_PER_GATE, 100); - - TaskManagerServicesConfiguration tmConfig = - TaskManagerServicesConfiguration.fromConfiguration(config, MEM_SIZE_PARAM, InetAddress.getLoopbackAddress(), true); - - assertEquals(tmConfig.getNetworkConfig().partitionRequestInitialBackoff(), 100); - assertEquals(tmConfig.getNetworkConfig().partitionRequestMaxBackoff(), 200); - assertEquals(tmConfig.getNetworkConfig().networkBuffersPerChannel(), 10); - assertEquals(tmConfig.getNetworkConfig().floatingNetworkBuffersPerGate(), 100); - } -} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java index e6e60f0ff556e..a802c71c0aa43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskSubmissionTestEnvironment.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.runtime.blob.BlobCacheService; import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.clusterframework.types.AllocationID; @@ -34,8 +34,8 @@ import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; import org.apache.flink.runtime.heartbeat.HeartbeatServices; import org.apache.flink.runtime.highavailability.TestingHighAvailabilityServices; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.netty.NettyConfig; import org.apache.flink.runtime.jobmaster.JobMasterGateway; import org.apache.flink.runtime.jobmaster.JobMasterId; @@ -51,7 +51,6 @@ import org.apache.flink.runtime.taskexecutor.slot.TaskSlotTable; import org.apache.flink.runtime.taskexecutor.slot.TimerService; import org.apache.flink.runtime.taskmanager.CheckpointResponder; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.NoOpTaskManagerActions; import org.apache.flink.runtime.taskmanager.Task; import org.apache.flink.runtime.taskmanager.TaskManagerActions; @@ -105,7 +104,7 @@ private TaskSubmissionTestEnvironment( Configuration configuration, List>> taskManagerActionListeners, TestingRpcService testingRpcService, - NetworkEnvironment networkEnvironment) throws Exception { + ShuffleEnvironment shuffleEnvironment) throws Exception { this.haServices = new TestingHighAvailabilityServices(); this.haServices.setResourceManagerLeaderRetriever(new SettableLeaderRetrievalService()); @@ -155,7 +154,7 @@ private TaskSubmissionTestEnvironment( Executors.directExecutor()); final TaskManagerServices taskManagerServices = new TaskManagerServicesBuilder() - .setNetworkEnvironment(networkEnvironment) + .setShuffleEnvironment(shuffleEnvironment) .setTaskSlotTable(taskSlotTable) .setJobManagerTable(jobManagerTable) .setTaskStateManager(localStateStoresManager) @@ -230,33 +229,33 @@ private static JobManagerConnection createJobManagerConnection(JobID jobId, JobM partitionProducerStateChecker); } - private static NetworkEnvironment createNetworkEnvironment( + private static ShuffleEnvironment createShuffleEnvironment( ResourceID taskManagerLocation, boolean localCommunication, Configuration configuration, RpcService testingRpcService, - boolean mockNetworkEnvironment) throws Exception { - final NetworkEnvironment networkEnvironment; - if (mockNetworkEnvironment) { - networkEnvironment = mock(NetworkEnvironment.class, Mockito.RETURNS_MOCKS); + boolean mockShuffleEnvironment) throws Exception { + final ShuffleEnvironment shuffleEnvironment; + if (mockShuffleEnvironment) { + shuffleEnvironment = mock(ShuffleEnvironment.class, Mockito.RETURNS_MOCKS); } else { final InetSocketAddress socketAddress = new InetSocketAddress( - InetAddress.getByName(testingRpcService.getAddress()), configuration.getInteger(NetworkEnvironmentOptions.DATA_PORT)); + InetAddress.getByName(testingRpcService.getAddress()), configuration.getInteger(NettyShuffleEnvironmentOptions.DATA_PORT)); final NettyConfig nettyConfig = new NettyConfig(socketAddress.getAddress(), socketAddress.getPort(), - NetworkEnvironmentConfiguration.getPageSize(configuration), ConfigurationParserUtils.getSlot(configuration), configuration); + ConfigurationParserUtils.getPageSize(configuration), ConfigurationParserUtils.getSlot(configuration), configuration); - networkEnvironment = new NetworkEnvironmentBuilder() + shuffleEnvironment = new NettyShuffleEnvironmentBuilder() .setTaskManagerLocation(taskManagerLocation) - .setPartitionRequestInitialBackoff(configuration.getInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL)) - .setPartitionRequestMaxBackoff(configuration.getInteger(NetworkEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX)) + .setPartitionRequestInitialBackoff(configuration.getInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_INITIAL)) + .setPartitionRequestMaxBackoff(configuration.getInteger(NettyShuffleEnvironmentOptions.NETWORK_REQUEST_BACKOFF_MAX)) .setNettyConfig(localCommunication ? null : nettyConfig) .build(); - networkEnvironment.start(); + shuffleEnvironment.start(); } - return networkEnvironment; + return shuffleEnvironment; } @Override @@ -275,14 +274,14 @@ public void close() throws Exception { public static final class Builder { private JobID jobId; - private boolean mockNetworkEnvironment = true; + private boolean mockShuffleEnvironment = true; private int slotSize; private JobMasterId jobMasterId = JobMasterId.generate(); private TestingJobMasterGateway jobMasterGateway; private boolean localCommunication = true; private Configuration configuration = new Configuration(); @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private Optional optionalNetworkEnvironment = Optional.empty(); + private Optional> optionalShuffleEnvironment = Optional.empty(); private ResourceID resourceID = ResourceID.generate(); private List>> taskManagerActionListeners = new ArrayList<>(); @@ -291,15 +290,15 @@ public Builder(JobID jobId) { this.jobId = jobId; } - public Builder useRealNonMockNetworkEnvironment() { - this.optionalNetworkEnvironment = Optional.empty(); - this.mockNetworkEnvironment = false; + public Builder useRealNonMockShuffleEnvironment() { + this.optionalShuffleEnvironment = Optional.empty(); + this.mockShuffleEnvironment = false; return this; } - public Builder setNetworkEnvironment(NetworkEnvironment optionalNetworkEnvironment) { - this.mockNetworkEnvironment = false; - this.optionalNetworkEnvironment = Optional.of(optionalNetworkEnvironment); + public Builder setShuffleEnvironment(ShuffleEnvironment optionalShuffleEnvironment) { + this.mockShuffleEnvironment = false; + this.optionalShuffleEnvironment = Optional.of(optionalShuffleEnvironment); return this; } @@ -340,14 +339,13 @@ public Builder setResourceID(ResourceID resourceID) { public TaskSubmissionTestEnvironment build() throws Exception { final TestingRpcService testingRpcService = new TestingRpcService(); - final NetworkEnvironment network = optionalNetworkEnvironment.orElseGet(() -> { + final ShuffleEnvironment network = optionalShuffleEnvironment.orElseGet(() -> { try { - return createNetworkEnvironment( - resourceID, + return createShuffleEnvironment(resourceID, localCommunication, configuration, testingRpcService, - mockNetworkEnvironment); + mockShuffleEnvironment); } catch (Exception e) { throw new FlinkRuntimeException("Failed to build TaskSubmissionTestEnvironment", e); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java index 01cd18477dc33..27e1f7e72ad43 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskAsyncCallTest.java @@ -40,8 +40,8 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; @@ -106,7 +106,7 @@ public class TaskAsyncCallTest extends TestLogger { private static final List classLoaders = Collections.synchronizedList(new ArrayList<>()); - private NetworkEnvironment networkEnvironment; + private ShuffleEnvironment shuffleEnvironment; @Before public void createQueuesAndActors() { @@ -117,15 +117,15 @@ public void createQueuesAndActors() { notifyCheckpointCompleteLatch = new OneShotLatch(); stopLatch = new OneShotLatch(); - networkEnvironment = new NetworkEnvironmentBuilder().build(); + shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); classLoaders.clear(); } @After - public void teardown() { - if (networkEnvironment != null) { - networkEnvironment.shutdown(); + public void teardown() throws Exception { + if (shuffleEnvironment != null) { + shuffleEnvironment.close(); } } @@ -255,7 +255,7 @@ private Task createTask(Class invokableClass) throw 0, mock(MemoryManager.class), mock(IOManager.class), - networkEnvironment, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java index 33d89e541db06..76969760d9ab8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskCancelAsyncProducerConsumerITCase.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.concurrent.FutureUtils; import org.apache.flink.runtime.execution.Environment; @@ -74,7 +74,7 @@ public class TaskCancelAsyncProducerConsumerITCase extends TestLogger { private static Configuration getFlinkConfiguration() { Configuration config = new Configuration(); config.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, "4096"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 9); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 9); return config; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java index f23fc9d737376..9a1b874d32053 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskmanager/TaskTest.java @@ -32,7 +32,6 @@ import org.apache.flink.runtime.blob.VoidBlobStore; import org.apache.flink.runtime.broadcast.BroadcastVariableManager; import org.apache.flink.runtime.clusterframework.types.AllocationID; -import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.Executors; import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.deployment.ResultPartitionDeploymentDescriptor; @@ -47,8 +46,8 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; @@ -125,7 +124,7 @@ public class TaskTest extends TestLogger { private static OneShotLatch awaitLatch; private static OneShotLatch triggerLatch; - private NetworkEnvironment networkEnvironment; + private ShuffleEnvironment shuffleEnvironment; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -135,13 +134,13 @@ public void setup() { awaitLatch = new OneShotLatch(); triggerLatch = new OneShotLatch(); - networkEnvironment = new NetworkEnvironmentBuilder().build(); + shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); } @After - public void teardown() { - if (networkEnvironment != null) { - networkEnvironment.shutdown(); + public void teardown() throws Exception { + if (shuffleEnvironment != null) { + shuffleEnvironment.close(); } } @@ -308,7 +307,7 @@ private void testExecutionFailsInNetworkRegistration( final PartitionProducerStateChecker partitionProducerStateChecker = mock(PartitionProducerStateChecker.class); final QueuedNoOpTaskManagerActions taskManagerActions = new QueuedNoOpTaskManagerActions(); - final Task task = new TaskBuilder(networkEnvironment) + final Task task = new TaskBuilder(shuffleEnvironment) .setTaskManagerActions(taskManagerActions) .setConsumableNotifier(consumableNotifier) .setPartitionProducerStateChecker(partitionProducerStateChecker) @@ -317,7 +316,7 @@ private void testExecutionFailsInNetworkRegistration( .build(); // shut down the network to make the following task registration failure - networkEnvironment.shutdown(); + shuffleEnvironment.close(); // should fail task.run(); @@ -974,7 +973,7 @@ private void setState(Task task, ExecutionState state) { } private TaskBuilder createTaskBuilder() { - return new TaskBuilder(networkEnvironment); + return new TaskBuilder(shuffleEnvironment); } private static final class TaskBuilder { @@ -983,7 +982,7 @@ private static final class TaskBuilder { private LibraryCacheManager libraryCacheManager; private ResultPartitionConsumableNotifier consumableNotifier; private PartitionProducerStateChecker partitionProducerStateChecker; - private final NetworkEnvironment networkEnvironment; + private final ShuffleEnvironment shuffleEnvironment; private KvStateService kvStateService; private Executor executor; private Configuration taskManagerConfig; @@ -1012,8 +1011,8 @@ private static final class TaskBuilder { requiredJarFileBlobKeys = Collections.emptyList(); } - private TaskBuilder(NetworkEnvironment networkEnvironment) { - this.networkEnvironment = Preconditions.checkNotNull(networkEnvironment); + private TaskBuilder(ShuffleEnvironment shuffleEnvironment) { + this.shuffleEnvironment = Preconditions.checkNotNull(shuffleEnvironment); } TaskBuilder setInvokable(Class invokable) { @@ -1117,7 +1116,7 @@ private Task build() throws Exception { 0, mock(MemoryManager.class), mock(IOManager.class), - networkEnvironment, + shuffleEnvironment, kvStateService, mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java index 51930a0e756e8..6f39813146fa8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/JvmExitOnFatalErrorTest.java @@ -43,8 +43,8 @@ import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.taskexecutor.PartitionProducerStateChecker; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; @@ -168,7 +168,7 @@ public static void main(String[] args) throws Exception { final MemoryManager memoryManager = new MemoryManager(1024 * 1024, 1); final IOManager ioManager = new IOManagerAsync(); - final NetworkEnvironment networkEnvironment = new NetworkEnvironmentBuilder().build(); + final ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); final TaskManagerRuntimeInfo tmInfo = TaskManagerConfiguration.fromConfiguration(taskManagerConfig); @@ -206,7 +206,7 @@ public static void main(String[] args) throws Exception { 0, // targetSlotNumber memoryManager, ioManager, - networkEnvironment, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), new BroadcastVariableManager(), new TaskEventDispatcher(), diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java index da401a7fdeebd..b77c7d038536c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/InputProcessorUtil.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.IllegalConfigurationException; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.network.partition.consumer.InputGate; @@ -53,7 +53,7 @@ public static CheckpointBarrierHandler createCheckpointBarrierHandler( + " must be positive or -1 (infinite)"); } - if (taskManagerConfig.getBoolean(NetworkEnvironmentOptions.NETWORK_CREDIT_MODEL)) { + if (taskManagerConfig.getBoolean(NettyShuffleEnvironmentOptions.NETWORK_CREDIT_MODEL)) { barrierHandler = new BarrierBuffer( inputGate, new CachedBufferBlocker(inputGate.getPageSize()), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java index 770af88d9eb58..0c399f7fea3b4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkBenchmarkEnvironment.java @@ -26,8 +26,8 @@ import org.apache.flink.runtime.deployment.InputGateDeploymentDescriptor; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironment; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.api.writer.RecordWriter; import org.apache.flink.runtime.io.network.api.writer.RecordWriterBuilder; @@ -47,7 +47,6 @@ import org.apache.flink.runtime.shuffle.ShuffleDescriptor; import org.apache.flink.runtime.taskmanager.ConsumableNotifyingResultPartitionWriterDecorator; import org.apache.flink.runtime.taskmanager.InputGateWithMetrics; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; import org.apache.flink.runtime.taskmanager.NoOpTaskActions; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.ConfigurationParserUtils; @@ -80,8 +79,8 @@ public class StreamNetworkBenchmarkEnvironment { protected final JobID jobId = new JobID(); protected final IntermediateDataSetID dataSetID = new IntermediateDataSetID(); - protected NetworkEnvironment senderEnv; - protected NetworkEnvironment receiverEnv; + protected NettyShuffleEnvironment senderEnv; + protected NettyShuffleEnvironment receiverEnv; protected IOManager ioManager; protected int channels; @@ -145,13 +144,13 @@ public void setUp( ioManager = new IOManagerAsync(); - senderEnv = createNettyNetworkEnvironment(senderBufferPoolSize, config); + senderEnv = createShuffleEnvironment(senderBufferPoolSize, config); this.dataPort = senderEnv.start(); if (localMode && senderBufferPoolSize == receiverBufferPoolSize) { receiverEnv = senderEnv; } else { - receiverEnv = createNettyNetworkEnvironment(receiverBufferPoolSize, config); + receiverEnv = createShuffleEnvironment(receiverBufferPoolSize, config); receiverEnv.start(); } @@ -167,8 +166,8 @@ public void setUp( } public void tearDown() { - suppressExceptions(senderEnv::shutdown); - suppressExceptions(receiverEnv::shutdown); + suppressExceptions(senderEnv::close); + suppressExceptions(receiverEnv::close); suppressExceptions(ioManager::shutdown); } @@ -197,17 +196,17 @@ private void generatePartitionIds() throws Exception { } } - private NetworkEnvironment createNettyNetworkEnvironment( + private NettyShuffleEnvironment createShuffleEnvironment( @SuppressWarnings("SameParameterValue") int bufferPoolSize, Configuration config) throws Exception { final NettyConfig nettyConfig = new NettyConfig( LOCAL_ADDRESS, 0, - NetworkEnvironmentConfiguration.getPageSize(config), + ConfigurationParserUtils.getPageSize(config), // please note that the number of slots directly influences the number of netty threads! ConfigurationParserUtils.getSlot(config), config); - return new NetworkEnvironmentBuilder() + return new NettyShuffleEnvironmentBuilder() .setNumNetworkBuffers(bufferPoolSize) .setNettyConfig(nettyConfig) .build(); @@ -216,7 +215,7 @@ private NetworkEnvironment createNettyNetworkEnvironment( protected ResultPartitionWriter createResultPartition( JobID jobId, ResultPartitionID partitionId, - NetworkEnvironment environment, + NettyShuffleEnvironment environment, int channels) throws Exception { ResultPartitionWriter resultPartitionWriter = new ResultPartitionBuilder() @@ -225,7 +224,7 @@ protected ResultPartitionWriter createResultPartition( .setNumberOfSubpartitions(channels) .setResultPartitionManager(environment.getResultPartitionManager()) .setIOManager(ioManager) - .setupBufferPoolFactoryFromNetworkEnvironment(environment) + .setupBufferPoolFactoryFromNettyShuffleEnvironment(environment) .build(); ResultPartitionWriter consumableNotifyingPartitionWriter = new ConsumableNotifyingResultPartitionWriterDecorator( diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java index a15aca445266d..b311060718313 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/StreamNetworkThroughputBenchmarkTest.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.runtime.io.benchmark; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.junit.Rule; import org.junit.Test; @@ -82,7 +82,7 @@ public void remoteModeInsufficientBuffersSender() throws Exception { expectedException.expect(IOException.class); expectedException.expectMessage("Insufficient number of network buffers"); - env.setUp(writers, channels, 100, false, writers * channels - 1, writers * channels * NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue()); + env.setUp(writers, channels, 100, false, writers * channels - 1, writers * channels * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue()); } @Test @@ -94,7 +94,7 @@ public void remoteModeInsufficientBuffersReceiver() throws Exception { expectedException.expect(IOException.class); expectedException.expectMessage("Insufficient number of network buffers"); - env.setUp(writers, channels, 100, false, writers * channels, writers * channels * NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue() - 1); + env.setUp(writers, channels, 100, false, writers * channels, writers * channels * NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue() - 1); } @Test @@ -104,7 +104,7 @@ public void remoteModeMinimumBuffers() throws Exception { int channels = 2; env.setUp(writers, channels, 100, false, writers * channels, writers * channels * - NetworkEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue()); + NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_PER_CHANNEL.defaultValue()); env.executeBenchmark(10_000); env.tearDown(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java index ce5a7e168094d..62ab1d4fa8639 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/InterruptSensitiveRestoreTest.java @@ -43,8 +43,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -53,6 +52,7 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.DefaultOperatorStateBackend; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -181,7 +181,7 @@ private static Task createTask( StreamStateHandle state, int mode) throws IOException { - NetworkEnvironment networkEnvironment = new NetworkEnvironmentBuilder().build(); + ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); Collection keyedStateFromBackend = Collections.emptyList(); Collection keyedStateFromStream = Collections.emptyList(); @@ -270,7 +270,7 @@ private static Task createTask( 0, mock(MemoryManager.class), mock(IOManager.class), - networkEnvironment, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java index 4cffdbd700372..e92ba9c23aa72 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTerminationTest.java @@ -43,8 +43,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -54,6 +53,7 @@ import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; import org.apache.flink.runtime.query.TaskKvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.CheckpointStorage; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -150,7 +150,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E final TaskManagerRuntimeInfo taskManagerRuntimeInfo = new TestingTaskManagerRuntimeInfo(); - final NetworkEnvironment networkEnv = new NetworkEnvironmentBuilder().build(); + final ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); BlobCacheService blobService = new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); @@ -167,7 +167,7 @@ public void testConcurrentAsyncCheckpointCannotFailFinishedStreamTask() throws E 0, new MemoryManager(32L * 1024L, 1), new IOManagerAsync(), - networkEnv, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 343957457ad94..f1fcfca3ba310 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -49,8 +49,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; @@ -65,6 +64,7 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointStorage; @@ -902,7 +902,7 @@ public static Task createTask( PartitionProducerStateChecker partitionProducerStateChecker = mock(PartitionProducerStateChecker.class); Executor executor = mock(Executor.class); - NetworkEnvironment network = new NetworkEnvironmentBuilder().build(); + ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); JobInformation jobInformation = new JobInformation( new JobID(), @@ -932,7 +932,7 @@ public static Task createTask( 0, mock(MemoryManager.class), mock(IOManager.class), - network, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java index 0416e5c84d918..ab3bcf8e8a3a4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SynchronousCheckpointITCase.java @@ -42,8 +42,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.io.network.partition.ResultPartitionConsumableNotifier; @@ -54,6 +53,7 @@ import org.apache.flink.runtime.metrics.groups.TaskMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.taskexecutor.KvStateService; @@ -263,7 +263,7 @@ private Task createTask(Class invokableClass) throw ResultPartitionConsumableNotifier consumableNotifier = new NoOpResultPartitionConsumableNotifier(); PartitionProducerStateChecker partitionProducerStateChecker = mock(PartitionProducerStateChecker.class); Executor executor = mock(Executor.class); - NetworkEnvironment networkEnvironment = new NetworkEnvironmentBuilder().build(); + ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); TaskMetricGroup taskMetricGroup = UnregisteredMetricGroups.createUnregisteredTaskMetricGroup(); @@ -295,7 +295,7 @@ private Task createTask(Class invokableClass) throw 0, mock(MemoryManager.class), mock(IOManager.class), - networkEnvironment, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java index a6268cea8413a..c02ff99bab64b 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TaskCheckpointingBehaviourTest.java @@ -45,8 +45,7 @@ import org.apache.flink.runtime.executiongraph.TaskInformation; import org.apache.flink.runtime.filecache.FileCache; import org.apache.flink.runtime.io.disk.iomanager.IOManager; -import org.apache.flink.runtime.io.network.NetworkEnvironment; -import org.apache.flink.runtime.io.network.NetworkEnvironmentBuilder; +import org.apache.flink.runtime.io.network.NettyShuffleEnvironmentBuilder; import org.apache.flink.runtime.io.network.TaskEventDispatcher; import org.apache.flink.runtime.io.network.partition.NoOpResultPartitionConsumableNotifier; import org.apache.flink.runtime.jobgraph.JobVertexID; @@ -55,6 +54,7 @@ import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.query.KvStateRegistry; +import org.apache.flink.runtime.shuffle.ShuffleEnvironment; import org.apache.flink.runtime.state.AbstractSnapshotStrategy; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.CheckpointStreamFactory; @@ -224,7 +224,7 @@ private static Task createTask( TestStreamTask.class.getName(), taskConfig); - NetworkEnvironment network = new NetworkEnvironmentBuilder().build(); + ShuffleEnvironment shuffleEnvironment = new NettyShuffleEnvironmentBuilder().build(); BlobCacheService blobService = new BlobCacheService(mock(PermanentBlobCache.class), mock(TransientBlobCache.class)); @@ -241,7 +241,7 @@ private static Task createTask( 0, mock(MemoryManager.class), mock(IOManager.class), - network, + shuffleEnvironment, new KvStateService(new KvStateRegistry(), null, null), mock(BroadcastVariableManager.class), new TaskEventDispatcher(), diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java index 08a5e1a4da2e5..ab98e926f7253 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/CancelingTestBase.java @@ -24,7 +24,7 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.optimizer.DataStatistics; import org.apache.flink.optimizer.Optimizer; @@ -80,7 +80,7 @@ private static Configuration getConfiguration() { config.setBoolean(CoreOptions.FILESYTEM_DEFAULT_OVERRIDE, true); config.setString(AkkaOptions.ASK_TIMEOUT, TestingUtils.DEFAULT_AKKA_ASK_TIMEOUT()); config.setString(TaskManagerOptions.MEMORY_SEGMENT_SIZE, "4096"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 2048); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 2048); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index 99c5a757dc0e1..818eb13e4a7cd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -28,7 +28,7 @@ import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.contrib.streaming.state.RocksDBOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; @@ -211,7 +211,7 @@ protected Configuration createClusterConfig() throws IOException { Configuration config = new Configuration(); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "48m"); // the default network buffers size (10% of heap max =~ 150MB) seems to much for this test case - config.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(80L << 20)); // 80 MB + config.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(80L << 20)); // 80 MB config.setString(AkkaOptions.FRAMESIZE, String.valueOf(MAX_MEM_STATE_SIZE) + "b"); if (zkServer != null) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java index 95aab44d57858..38902c7619da9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java +++ b/flink-tests/src/test/java/org/apache/flink/test/manual/StreamingScalabilityAndLatency.java @@ -21,7 +21,7 @@ import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; @@ -51,7 +51,7 @@ public static void main(String[] args) throws Exception { try { Configuration config = new Configuration(); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "80m"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 20000); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 20000); config.setInteger("taskmanager.net.server.numThreads", 1); config.setInteger("taskmanager.net.client.numThreads", 1); diff --git a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java index 642f6015395c7..7cef448116d7e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/misc/SuccessAfterNetworkBuffersFailureITCase.java @@ -26,7 +26,7 @@ import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.examples.java.clustering.KMeans; import org.apache.flink.examples.java.clustering.util.KMeansData; @@ -63,7 +63,7 @@ public class SuccessAfterNetworkBuffersFailureITCase extends TestLogger { private static Configuration getConfiguration() { Configuration config = new Configuration(); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "80m"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 800); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 800); return config; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java index a8f6ffca07aa2..1677a88a3ad0f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/AbstractTaskManagerProcessFailureRecoveryTest.java @@ -24,7 +24,7 @@ import org.apache.flink.configuration.HeartbeatManagerOptions; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.types.ResourceID; @@ -103,7 +103,7 @@ public void testTaskManagerProcessFailure() throws Exception { config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); try (final StandaloneSessionClusterEntrypoint clusterEntrypoint = new StandaloneSessionClusterEntrypoint(config)) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index 2ff3f593d0ee8..39ee5d9df204a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -29,7 +29,7 @@ import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.clusterframework.types.ResourceID; import org.apache.flink.runtime.concurrent.FutureUtils; @@ -251,7 +251,7 @@ public void testDispatcherProcessFailure() throws Exception { zooKeeper.getConnectString(), zookeeperStoragePath.getPath()); // Task manager configuration config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); final RpcService rpcService = AkkaRpcServiceUtils.createRpcService("localhost", 0, config); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java index e4dd1f114e98b..933d63077f3a3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureCancelingITCase.java @@ -32,7 +32,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.runtime.client.JobStatusMessage; @@ -115,7 +115,7 @@ public void testCancelingOnProcessFailure() throws Exception { config.setString(HighAvailabilityOptions.HA_STORAGE_PATH, temporaryFolder.newFolder().getAbsolutePath()); config.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, 2); config.setString(TaskManagerOptions.MANAGED_MEMORY_SIZE, "4m"); - config.setInteger(NetworkEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); + config.setInteger(NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, 100); config.setInteger(RestOptions.PORT, 0); final RpcService rpcService = AkkaRpcServiceUtils.createRpcService("localhost", 0, config); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java index 05fd914e54f40..1657d8b22678b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/NettyEpollITCase.java @@ -20,7 +20,7 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -73,7 +73,7 @@ public Integer getKey(Integer value) throws Exception { private MiniClusterWithClientResource trySetUpCluster() throws Exception { try { Configuration config = new Configuration(); - config.setString(NetworkEnvironmentOptions.TRANSPORT_TYPE, "epoll"); + config.setString(NettyShuffleEnvironmentOptions.TRANSPORT_TYPE, "epoll"); MiniClusterWithClientResource cluster = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() .setConfiguration(config) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index 140b8c8fe65ad..0c93748829a0d 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -35,7 +35,7 @@ import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerInfo; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersHeaders; import org.apache.flink.runtime.rest.messages.taskmanager.TaskManagersInfo; -import org.apache.flink.runtime.taskmanager.NetworkEnvironmentConfiguration; +import org.apache.flink.runtime.taskmanager.NettyShuffleEnvironmentConfiguration; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.test.testdata.WordCountData; import org.apache.flink.util.ExceptionUtils; @@ -206,7 +206,7 @@ public void perJobYarnClusterOffHeap() throws Exception { // set memory constraints (otherwise this is the same test as perJobYarnCluster() above) final long taskManagerMemoryMB = 1024; //noinspection NumericOverflow if the calculation of the total Java memory size overflows, default configuration parameters are wrong in the first place, so we can ignore this inspection - final long networkBuffersMB = NetworkEnvironmentConfiguration.calculateNetworkBufferMemory( + final long networkBuffersMB = NettyShuffleEnvironmentConfiguration.calculateNetworkBufferMemory( (taskManagerMemoryMB - ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN.defaultValue()) << 20, new Configuration()) >> 20; final long offHeapMemory = taskManagerMemoryMB diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java index 28653b6bd247c..86b0052974f9b 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YarnConfigurationITCase.java @@ -25,7 +25,7 @@ import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.NetworkEnvironmentOptions; +import org.apache.flink.configuration.NettyShuffleEnvironmentOptions; import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; import org.apache.flink.runtime.jobgraph.JobGraph; @@ -88,8 +88,8 @@ public void testFlinkContainerMemory() throws Exception { // disable heap cutoff min configuration.setInteger(ResourceManagerOptions.CONTAINERIZED_HEAP_CUTOFF_MIN, 0); - configuration.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(1L << 20)); - configuration.setString(NetworkEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(4L << 20)); + configuration.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MIN, String.valueOf(1L << 20)); + configuration.setString(NettyShuffleEnvironmentOptions.NETWORK_BUFFERS_MEMORY_MAX, String.valueOf(4L << 20)); final YarnConfiguration yarnConfiguration = getYarnConfiguration(); final YarnClusterDescriptor clusterDescriptor = new YarnClusterDescriptor(