forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-16560] Forward Configuration in PackagedProgramUtils#getPipeli…
…neFromProgram Before, when using PackagedProgramUtils (for example in the standalone cluster entrypoint or the web ui) the Flink Configuration would not be applied to the execution environment. This also adds a test that verifies that we forward configuration.
- Loading branch information
Showing
9 changed files
with
144 additions
and
11 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
123 changes: 123 additions & 0 deletions
123
flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,123 @@ | ||
/* | ||
* 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.client.program; | ||
|
||
import org.apache.flink.api.common.ExecutionConfig; | ||
import org.apache.flink.api.common.Plan; | ||
import org.apache.flink.api.dag.Pipeline; | ||
import org.apache.flink.api.java.ExecutionEnvironment; | ||
import org.apache.flink.configuration.Configuration; | ||
import org.apache.flink.configuration.PipelineOptions; | ||
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; | ||
import org.apache.flink.streaming.api.graph.StreamGraph; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.hamcrest.Matchers.is; | ||
import static org.junit.Assert.assertThat; | ||
|
||
/** | ||
* Tests {@link PackagedProgramUtils}. | ||
*/ | ||
public class PackagedProgramUtilsTest { | ||
|
||
/** | ||
* This tests whether configuration forwarding from a {@link Configuration} to the environment | ||
* works. | ||
*/ | ||
@Test | ||
public void testDataSetConfigurationForwarding() throws Exception { | ||
assertPrecondition(ExecutionEnvironment.getExecutionEnvironment().getConfig()); | ||
|
||
PackagedProgram packagedProgram = PackagedProgram.newBuilder() | ||
.setEntryPointClassName(DataSetTestProgram.class.getName()) | ||
.build(); | ||
|
||
Configuration config = createConfigurationWithOption(); | ||
|
||
Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram( | ||
packagedProgram, | ||
config, | ||
1 /* parallelism */, | ||
false /* suppress output */); | ||
|
||
ExecutionConfig executionConfig = ((Plan) pipeline).getExecutionConfig(); | ||
|
||
assertExpectedOption(executionConfig); | ||
} | ||
|
||
/** | ||
* This tests whether configuration forwarding from a {@link Configuration} to the environment | ||
* works. | ||
*/ | ||
@Test | ||
public void testDataStreamConfigurationForwarding() throws Exception { | ||
assertPrecondition(ExecutionEnvironment.getExecutionEnvironment().getConfig()); | ||
|
||
PackagedProgram packagedProgram = PackagedProgram.newBuilder() | ||
.setEntryPointClassName(DataStreamTestProgram.class.getName()) | ||
.build(); | ||
|
||
Configuration config = createConfigurationWithOption(); | ||
|
||
Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram( | ||
packagedProgram, | ||
config, | ||
1 /* parallelism */, | ||
false /* suppress output */); | ||
|
||
ExecutionConfig executionConfig = ((StreamGraph) pipeline).getExecutionConfig(); | ||
|
||
assertExpectedOption(executionConfig); | ||
} | ||
|
||
private static void assertPrecondition(ExecutionConfig executionConfig) { | ||
// we want to test forwarding with this config, ensure that the default is what we expect. | ||
assertThat(executionConfig.isAutoTypeRegistrationDisabled(), is(false)); | ||
} | ||
|
||
private static void assertExpectedOption(ExecutionConfig executionConfig) { | ||
// we want to test forwarding with this config, ensure that the default is what we expect. | ||
assertThat(executionConfig.isAutoTypeRegistrationDisabled(), is(true)); | ||
} | ||
|
||
private static Configuration createConfigurationWithOption() { | ||
Configuration config = new Configuration(); | ||
config.set(PipelineOptions.AUTO_TYPE_REGISTRATION, false); | ||
return config; | ||
} | ||
|
||
/** Test Program for the DataSet API. */ | ||
public static class DataSetTestProgram { | ||
public static void main(String[] args) throws Exception { | ||
ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); | ||
env.fromElements("hello").print(); | ||
env.execute(); | ||
} | ||
} | ||
|
||
/** Test Program for the DataStream API. */ | ||
public static class DataStreamTestProgram { | ||
public static void main(String[] args) throws Exception { | ||
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); | ||
env.fromElements("hello").print(); | ||
env.execute(); | ||
} | ||
} | ||
} |