Skip to content
Closed
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
Original file line number Diff line number Diff line change
Expand Up @@ -17,13 +17,13 @@

package org.apache.flink.cdc.cli;

import org.apache.flink.cdc.cli.parser.PipelineDefinitionParser;
import org.apache.flink.cdc.cli.parser.YamlPipelineDefinitionParser;
import org.apache.flink.cdc.cli.utils.FlinkEnvironmentUtils;
import org.apache.flink.cdc.common.annotation.VisibleForTesting;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.composer.PipelineComposer;
import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.cdc.composer.parser.PipelineDefinitionParser;
import org.apache.flink.cdc.composer.parser.YamlPipelineDefinitionParser;
import org.apache.flink.cdc.composer.definition.PipelineDef;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
package org.apache.flink.cdc.cli;

import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.composer.PipelineComposer;
import org.apache.flink.cdc.composer.PipelineExecution;
import org.apache.flink.cdc.composer.definition.PipelineDef;
import org.apache.flink.cdc.composer.flink.FlinkPipelineComposer;
import org.apache.flink.cdc.composer.parser.PipelineDefinitionParser;
import org.apache.flink.cdc.composer.parser.YamlPipelineDefinitionParser;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.nio.file.Path;
import java.nio.file.Paths;

public class PipelineEmbeddedExecutor {

private static final Logger LOG = LoggerFactory.getLogger(PipelineEmbeddedExecutor.class);

private final Path pipelineDefPath;

public PipelineEmbeddedExecutor(Path pipelineDefPath) {
this.pipelineDefPath = pipelineDefPath;
}

public static void main(String[] args) throws Exception {
final ParameterTool params = ParameterTool.fromArgs(args);
PipelineEmbeddedExecutor executor = new PipelineEmbeddedExecutor(Paths.get(params.get("pipeline-file")));
PipelineExecution.ExecutionInfo executionInfo = executor.run();
logExecutionInfo(executionInfo);
}

public PipelineExecution.ExecutionInfo run() throws Exception {
// Parse pipeline definition file
PipelineDefinitionParser pipelineDefinitionParser = new YamlPipelineDefinitionParser();
PipelineDef pipelineDef =
pipelineDefinitionParser.parse(pipelineDefPath, new Configuration());

// Create composer
PipelineComposer composer = FlinkPipelineComposer.ofApplicationCluster();

// Compose pipeline
PipelineExecution execution = composer.compose(pipelineDef);

// Execute the pipeline
return execution.execute();
}

private static void logExecutionInfo(PipelineExecution.ExecutionInfo info) {
LOG.info("Pipeline has been submitted to cluster.");
LOG.info("Job ID: {}\n", info.getId());
LOG.info("Job Description: {}\n", info.getDescription());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,11 @@ public static FlinkPipelineComposer ofMiniCluster() {
StreamExecutionEnvironment.getExecutionEnvironment(), true);
}

public static FlinkPipelineComposer ofApplicationCluster() {
return new FlinkPipelineComposer(
StreamExecutionEnvironment.getExecutionEnvironment(), false);
}

private FlinkPipelineComposer(StreamExecutionEnvironment env, boolean isBlocking) {
this.env = env;
this.isBlocking = isBlocking;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.flink.cdc.cli.parser;
package org.apache.flink.cdc.composer.parser;

import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.composer.definition.PipelineDef;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.flink.cdc.cli.parser;
package org.apache.flink.cdc.composer.parser;

import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.utils.StringUtils;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
* limitations under the License.
*/

package org.apache.flink.cdc.cli.parser;
package org.apache.flink.cdc.composer;

import org.apache.flink.cdc.common.configuration.Configuration;
import org.apache.flink.cdc.common.pipeline.PipelineOptions;
Expand All @@ -25,6 +25,7 @@
import org.apache.flink.cdc.composer.definition.SourceDef;
import org.apache.flink.cdc.composer.definition.TransformDef;

import org.apache.flink.cdc.composer.parser.YamlPipelineDefinitionParser;
import org.apache.flink.shaded.guava31.com.google.common.collect.ImmutableMap;
import org.apache.flink.shaded.guava31.com.google.common.io.Resources;

Expand All @@ -40,7 +41,7 @@
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy;

/** Unit test for {@link org.apache.flink.cdc.cli.parser.YamlPipelineDefinitionParser}. */
/** Unit test for {@link org.apache.flink.cdc.composer.parser.YamlPipelineDefinitionParser}. */
class YamlPipelineDefinitionParserTest {

@Test
Expand Down