Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-36322] Fix compile errors based on 2.0-preview #97

Merged
merged 2 commits into from
Oct 10, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 0 additions & 22 deletions pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -585,28 +585,6 @@ under the License.
</extension>
</extensions>
<plugins>
<plugin>
<groupId>net.alchim31.maven</groupId>
<artifactId>scala-maven-plugin</artifactId>
<version>4.4.0</version>
<executions>
<execution>
<id>scala-compile-first</id>
<phase>process-resources</phase>
<goals>
<goal>add-source</goal>
<goal>compile</goal>
</goals>
</execution>
<execution>
<id>scala-test-compile</id>
<phase>process-test-resources</phase>
<goals>
<goal>testCompile</goal>
</goals>
</execution>
</executions>
</plugin>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-compiler-plugin</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.async.ResultFuture;
import org.apache.flink.streaming.api.functions.async.RichAsyncFunction;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.OperationsPerInvocation;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ private static RestClusterClient<StandaloneClusterId> createClient(
int port, Configuration clientConfiguration) throws Exception {
final Configuration clientConfig = new Configuration();
clientConfig.addAll(clientConfiguration);
clientConfig.setInteger(RestOptions.PORT, port);
clientConfig.set(RestOptions.PORT, port);
return new RestClusterClient<>(clientConfig, StandaloneClusterId.getInstance());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -60,22 +60,6 @@ public void compressedFilePartition(CompressedFileEnvironmentContext context) th
executeBenchmark(context.env);
}

@Benchmark
public void uncompressedMmapPartition(UncompressedMmapEnvironmentContext context)
throws Exception {
executeBenchmark(context.env);
}

@Benchmark
public void compressedSortPartition(CompressedSortEnvironmentContext context) throws Exception {
executeBenchmark(context.env);
}

@Benchmark
public void uncompressedSortPartition(UncompressedSortEnvironmentContext context) throws Exception {
executeBenchmark(context.env);
}

private void executeBenchmark(StreamExecutionEnvironment env) throws Exception {
StreamGraph streamGraph =
StreamGraphUtils.buildGraphForBatchJob(env, RECORDS_PER_INVOCATION);
Expand All @@ -102,23 +86,15 @@ public void setUp() throws Exception {
}

protected Configuration createConfiguration(
boolean compressionEnabled, String subpartitionType, boolean isSortShuffle) {
boolean compressionEnabled) {
Configuration configuration = super.createConfiguration();

if (isSortShuffle) {
configuration.setInteger(
NettyShuffleEnvironmentOptions.NETWORK_SORT_SHUFFLE_MIN_PARALLELISM, 1);
} else {
configuration.setInteger(
NettyShuffleEnvironmentOptions.NETWORK_SORT_SHUFFLE_MIN_PARALLELISM,
Integer.MAX_VALUE);
}
configuration.setBoolean(
NettyShuffleEnvironmentOptions.BATCH_SHUFFLE_COMPRESSION_ENABLED,
compressionEnabled);
configuration.setString(
NettyShuffleEnvironmentOptions.NETWORK_BLOCKING_SHUFFLE_TYPE, subpartitionType);
configuration.setString(
configuration.set(
NettyShuffleEnvironmentOptions.SHUFFLE_COMPRESSION_CODEC,
compressionEnabled ?
NettyShuffleEnvironmentOptions.CompressionCodec.LZ4
: NettyShuffleEnvironmentOptions.CompressionCodec.NONE);
configuration.set(
CoreOptions.TMP_DIRS,
FileUtils.getCurrentWorkingDirectory().toAbsolutePath().toString());
return configuration;
Expand All @@ -129,39 +105,15 @@ public static class UncompressedFileEnvironmentContext
extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(false, "file", false);
return createConfiguration(false);
}
}

public static class CompressedFileEnvironmentContext
extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(true, "file", false);
}
}

public static class UncompressedMmapEnvironmentContext
extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(false, "mmap", false);
}
}

public static class CompressedSortEnvironmentContext
extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(true, "file", true);
}
}

public static class UncompressedSortEnvironmentContext
extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(false, "file", true);
return createConfiguration(true);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,14 +48,6 @@ public static void main(String[] args) throws RunnerException {
new Runner(options).run();
}

@Benchmark
public void remoteFilePartition(RemoteFileEnvironmentContext context) throws Exception {
StreamGraph streamGraph =
StreamGraphUtils.buildGraphForBatchJob(context.env, RECORDS_PER_INVOCATION);
context.miniCluster.executeJobBlocking(
StreamingJobGraphGenerator.createJobGraph(streamGraph));
}

@Benchmark
public void remoteSortPartition(RemoteSortEnvironmentContext context) throws Exception {
StreamGraph streamGraph =
Expand All @@ -75,20 +67,10 @@ public void setUp() throws Exception {
env.setBufferTimeout(-1);
}

protected Configuration createConfiguration(boolean isSortShuffle) {
protected Configuration createConfiguration() {
Configuration configuration = super.createConfiguration();

if (isSortShuffle) {
configuration.setInteger(
NettyShuffleEnvironmentOptions.NETWORK_SORT_SHUFFLE_MIN_PARALLELISM, 1);
} else {
configuration.setInteger(
NettyShuffleEnvironmentOptions.NETWORK_SORT_SHUFFLE_MIN_PARALLELISM,
Integer.MAX_VALUE);
}
configuration.setString(
NettyShuffleEnvironmentOptions.NETWORK_BLOCKING_SHUFFLE_TYPE, "file");
configuration.setString(
configuration.set(
CoreOptions.TMP_DIRS,
FileUtils.getCurrentWorkingDirectory().toAbsolutePath().toString());
return configuration;
Expand All @@ -100,17 +82,10 @@ protected int getNumberOfVertices() {
}
}

public static class RemoteFileEnvironmentContext extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(false);
}
}

public static class RemoteSortEnvironmentContext extends BlockingPartitionEnvironmentContext {
@Override
protected Configuration createConfiguration() {
return createConfiguration(true);
return super.createConfiguration();
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@
package org.apache.flink.benchmark;

import org.apache.flink.api.common.JobID;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.MemorySize;
import org.apache.flink.configuration.TaskManagerOptions;
import org.apache.flink.core.execution.JobClient;
import org.apache.flink.runtime.jobgraph.JobVertexID;
import org.apache.flink.runtime.testutils.CommonTestUtils;
import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;

import java.time.Duration;
Expand Down Expand Up @@ -93,31 +93,31 @@ protected Configuration createConfiguration() {
public enum CheckpointMode {
UNALIGNED(
config -> {
config.set(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, true);
config.set(CheckpointingOptions.ENABLE_UNALIGNED, true);
config.set(
TaskManagerOptions.MEMORY_SEGMENT_SIZE,
CheckpointEnvironmentContext.START_MEMORY_SEGMENT_SIZE);
config.set(
ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT,
CheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT,
Duration.ofMillis(0));
config.set(TaskManagerOptions.BUFFER_DEBLOAT_ENABLED, false);
return config;
}),
UNALIGNED_1(
config -> {
config.set(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, true);
config.set(CheckpointingOptions.ENABLE_UNALIGNED, true);
config.set(
TaskManagerOptions.MEMORY_SEGMENT_SIZE,
CheckpointEnvironmentContext.START_MEMORY_SEGMENT_SIZE);
config.set(
ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT,
CheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT,
Duration.ofMillis(1));
config.set(TaskManagerOptions.BUFFER_DEBLOAT_ENABLED, false);
return config;
}),
ALIGNED(
config -> {
config.set(ExecutionCheckpointingOptions.ENABLE_UNALIGNED, false);
config.set(CheckpointingOptions.ENABLE_UNALIGNED, false);
config.set(
TaskManagerOptions.MEMORY_SEGMENT_SIZE,
CheckpointEnvironmentContext.START_MEMORY_SEGMENT_SIZE);
Expand Down
2 changes: 1 addition & 1 deletion src/main/java/org/apache/flink/benchmark/CollectSink.java
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.flink.benchmark;

import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction;

import java.util.ArrayList;
import java.util.List;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,19 +18,18 @@
package org.apache.flink.benchmark;

import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.FileInputSplit;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.testutils.OneShotLatch;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
import org.apache.flink.streaming.api.functions.sink.legacy.SinkFunction;
import org.apache.flink.streaming.api.functions.source.ContinuousFileReaderOperatorFactory;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.functions.source.TimestampedFileInputSplit;

import joptsimple.internal.Strings;
import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.OperationsPerInvocation;
import org.openjdk.jmh.runner.Runner;
Expand Down Expand Up @@ -76,7 +75,6 @@ public static void main(String[] args) throws RunnerException {
public void readFileSplit(FlinkEnvironmentContext context) throws Exception {
TARGET_COUNT_REACHED_LATCH.reset();
StreamExecutionEnvironment env = context.env;
env.setRestartStrategy(new RestartStrategies.NoRestartStrategyConfiguration());
env.enableCheckpointing(100)
.setParallelism(1)
.addSource(new MockSourceFunction())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,15 +18,14 @@

package org.apache.flink.benchmark;

import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.DeploymentOptions;
import org.apache.flink.configuration.NettyShuffleEnvironmentOptions;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.configuration.RestartStrategyOptions;
import org.apache.flink.configuration.StateBackendOptions;
import org.apache.flink.runtime.minicluster.MiniCluster;
import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
import org.apache.flink.runtime.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader;

Expand Down Expand Up @@ -54,6 +53,8 @@ public void setUp() throws Exception {
throw new RuntimeException("setUp was called multiple times!");
}
final Configuration clusterConfig = createConfiguration();
clusterConfig.set(RestartStrategyOptions.RESTART_STRATEGY, "none");
clusterConfig.set(StateBackendOptions.STATE_BACKEND, "hashmap");
miniCluster =
new MiniCluster(
new MiniClusterConfiguration.Builder()
Expand All @@ -78,8 +79,6 @@ public void setUp() throws Exception {
if (objectReuse) {
env.getConfig().enableObjectReuse();
}
env.setRestartStrategy(RestartStrategies.noRestart());
env.setStateBackend(new MemoryStateBackend());
}

@TearDown
Expand All @@ -102,14 +101,15 @@ public void execute() throws Exception {

protected Configuration createConfiguration() {
final Configuration configuration = new Configuration();
configuration.setString(RestOptions.BIND_PORT, "0");
configuration.setInteger(
NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, NUM_NETWORK_BUFFERS);
configuration.set(RestOptions.BIND_PORT, "0");
// no equivalent config available.
//configuration.setInteger(
// NettyShuffleEnvironmentOptions.NETWORK_NUM_BUFFERS, NUM_NETWORK_BUFFERS);
configuration.set(DeploymentOptions.TARGET, MiniClusterPipelineExecutorServiceLoader.NAME);
configuration.set(DeploymentOptions.ATTACHED, true);
// It doesn't make sense to wait for the final checkpoint in benchmarks since it only prolongs
// the test but doesn't give any advantages.
configuration.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, false);
configuration.set(CheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, false);
// TODO: remove this line after FLINK-28243 will be done
configuration.set(REQUIREMENTS_CHECK_DELAY, Duration.ZERO);
return configuration;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.flink.benchmark.functions.MultiplyByTwo;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink;

import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.OperationsPerInvocation;
Expand Down
10 changes: 6 additions & 4 deletions src/main/java/org/apache/flink/benchmark/KeyByBenchmarks.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@

import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.benchmark.functions.BaseSourceWithKeyRange;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;
import org.apache.flink.streaming.api.functions.sink.legacy.DiscardingSink;

import org.apache.flink.streaming.util.keys.KeySelectorUtil;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.OperationsPerInvocation;
import org.openjdk.jmh.runner.Runner;
Expand Down Expand Up @@ -54,7 +56,7 @@ public void tupleKeyBy(FlinkEnvironmentContext context) throws Exception {
env.setParallelism(4);

env.addSource(new IncreasingTupleSource(TUPLE_RECORDS_PER_INVOCATION, 10))
.keyBy(0)
.keyBy(e -> e.f0)
.addSink(new DiscardingSink<>());

env.execute();
Expand All @@ -66,8 +68,8 @@ public void arrayKeyBy(FlinkEnvironmentContext context) throws Exception {
StreamExecutionEnvironment env = context.env;
env.setParallelism(4);

env.addSource(new IncreasingArraySource(ARRAY_RECORDS_PER_INVOCATION, 10))
.keyBy(0)
DataStreamSource<int[]> source = env.addSource(new IncreasingArraySource(ARRAY_RECORDS_PER_INVOCATION, 10));
source.keyBy(KeySelectorUtil.getSelectorForArray(new int[]{0}, source.getType()))
.addSink(new DiscardingSink<>());

env.execute();
Expand Down
Loading
Loading