reswqa commented on code in PR #24422:
URL: https://github.com/apache/flink/pull/24422#discussion_r1527940518


##########
flink-process-function-parent/flink-process-function/src/main/java/org/apache/flink/process/impl/ExecutionEnvironmentImpl.java:
##########
@@ -0,0 +1,341 @@
+/*
+ * 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.process.impl;
+
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.functions.InvalidTypesException;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.connector.v2.FromDataSource;
+import org.apache.flink.api.connector.v2.Source;
+import org.apache.flink.api.connector.v2.WrappedSource;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.api.java.typeutils.MissingTypeInfo;
+import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.DeploymentOptions;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.configuration.ReadableConfig;
+import 
org.apache.flink.connector.datagen.functions.FromElementsGeneratorFunction;
+import org.apache.flink.connector.datagen.source.DataGeneratorSource;
+import org.apache.flink.core.execution.DefaultExecutorServiceLoader;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.core.execution.PipelineExecutor;
+import org.apache.flink.core.execution.PipelineExecutorFactory;
+import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
+import org.apache.flink.process.api.ExecutionEnvironment;
+import org.apache.flink.process.api.stream.NonKeyedPartitionStream;
+import org.apache.flink.process.impl.stream.NonKeyedPartitionStreamImpl;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
+import org.apache.flink.streaming.api.transformations.SourceTransformation;
+import 
org.apache.flink.streaming.runtime.translators.ProcessFunctionSinkTransformationTranslator;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkException;
+import org.apache.flink.util.Preconditions;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+
+import static 
org.apache.flink.streaming.api.graph.StreamGraphGenerator.DEFAULT_TIME_CHARACTERISTIC;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/** The implementation of {@link ExecutionEnvironment}. */
+public class ExecutionEnvironmentImpl implements ExecutionEnvironment {
+    private final List<Transformation<?>> transformations = new ArrayList<>();
+
+    private final ExecutionConfig executionConfig;
+
+    /** Settings that control the checkpointing behavior. */
+    private final CheckpointConfig checkpointCfg;
+
+    private final Configuration configuration;
+
+    private final ClassLoader userClassloader;
+
+    private final PipelineExecutorServiceLoader executorServiceLoader;
+
+    static {
+        try {
+            // All transformation translator must be put to a map in 
StreamGraphGenerator, but
+            // streaming-java is not depend on process-function module, using 
reflect to handle
+            // this.
+            
ProcessFunctionSinkTransformationTranslator.registerSinkTransformationTranslator();
+        } catch (Exception e) {
+            throw new RuntimeException(
+                    "Can not register process function transformation 
translator.");
+        }
+    }
+
+    /**
+     * The environment of the context (local by default, cluster if invoked 
through command line).
+     */
+    private static ExecutionEnvironmentFactory contextEnvironmentFactory = 
null;
+
+    public static ExecutionEnvironment newInstance() {
+        if (contextEnvironmentFactory != null) {
+            return contextEnvironmentFactory.createExecutionEnvironment(new 
Configuration());
+        } else {
+            final Configuration configuration = new Configuration();
+            configuration.set(DeploymentOptions.TARGET, "local");
+            configuration.set(DeploymentOptions.ATTACHED, true);
+            return new ExecutionEnvironmentImpl(
+                    new DefaultExecutorServiceLoader(), configuration, null);
+        }
+    }
+
+    ExecutionEnvironmentImpl(
+            PipelineExecutorServiceLoader executorServiceLoader,
+            Configuration configuration,
+            ClassLoader classLoader) {
+        this.executorServiceLoader = checkNotNull(executorServiceLoader);
+        this.configuration = configuration;
+        this.executionConfig = new ExecutionConfig(this.configuration);
+        this.checkpointCfg = new CheckpointConfig(this.configuration);
+        this.userClassloader = classLoader == null ? 
getClass().getClassLoader() : classLoader;
+        configure(configuration, userClassloader);
+    }
+
+    @Override
+    public void execute(String jobName) throws Exception {
+        StreamGraph streamGraph = getStreamGraph();
+        if (jobName != null) {
+            streamGraph.setJobName(jobName);
+        }
+
+        execute(streamGraph);
+    }
+
+    @Override
+    public ExecutionEnvironment setExecutionMode(RuntimeExecutionMode 
runtimeMode) {
+        checkNotNull(runtimeMode);
+        configuration.set(ExecutionOptions.RUNTIME_MODE, runtimeMode);
+        return this;
+    }
+
+    protected static void 
initializeContextEnvironment(ExecutionEnvironmentFactory ctx) {
+        contextEnvironmentFactory = ctx;
+    }
+
+    protected static void resetContextEnvironment() {
+        contextEnvironmentFactory = null;
+    }
+
+    @Override
+    public <OUT> NonKeyedPartitionStream<OUT> fromSource(Source<OUT> source, 
String sourceName) {
+        if (source instanceof WrappedSource) {
+            org.apache.flink.api.connector.source.Source<OUT, ?, ?> 
innerSource =
+                    ((WrappedSource<OUT>) source).getWrappedSource();
+            final TypeInformation<OUT> resolvedTypeInfo =
+                    getSourceTypeInfo(innerSource, sourceName);
+
+            SourceTransformation<OUT, ?, ?> sourceTransformation =
+                    new SourceTransformation<>(
+                            sourceName,
+                            innerSource,
+                            WatermarkStrategy.noWatermarks(),
+                            resolvedTypeInfo,
+                            getParallelism(),
+                            false);
+            return new NonKeyedPartitionStreamImpl<>(this, 
sourceTransformation);
+        } else if (source instanceof FromDataSource) {
+            Collection<OUT> data = ((FromDataSource<OUT>) source).getData();
+            TypeInformation<OUT> outType = extractTypeInfoFromCollection(data);
+
+            FromElementsGeneratorFunction<OUT> generatorFunction =
+                    new FromElementsGeneratorFunction<>(outType, 
executionConfig, data);
+
+            DataGeneratorSource<OUT> generatorSource =
+                    new DataGeneratorSource<>(generatorFunction, data.size(), 
outType);
+
+            return fromSource(new WrappedSource<>(generatorSource), 
"Collection Source");
+        } else {
+            throw new UnsupportedOperationException(
+                    "Unsupported type of sink, you could use SourceUtils to 
wrap a FLIP-27 based source.");
+        }
+    }
+
+    public Configuration getConfiguration() {
+        return this.configuration;
+    }
+
+    public ExecutionConfig getExecutionConfig() {
+        return executionConfig;
+    }
+
+    public int getParallelism() {
+        return executionConfig.getParallelism();
+    }
+
+    public List<Transformation<?>> getTransformations() {
+        return transformations;
+    }
+
+    public void setParallelism(int parallelism) {

Review Comment:
   Moved.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to