[ 
https://issues.apache.org/jira/browse/FLINK-7065?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16101814#comment-16101814
 ] 

ASF GitHub Bot commented on FLINK-7065:
---------------------------------------

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4273#discussion_r129599728
  
    --- Diff: 
flink-core/src/main/java/org/apache/flink/api/common/Executor.java ---
    @@ -0,0 +1,89 @@
    +/*
    + * 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.api.common;
    +
    +import org.apache.flink.annotation.Internal;
    +
    +/**
    + * A Executor execute a Flink program's job.
    + *
    + * <p>The specific implementation (such as the 
org.apache.flink.client.LocalExecutor
    + * and org.apache.flink.client.RemoteExecutor) determines where and how to 
run the dataflow.
    + * The concrete implementations of the executors are loaded dynamically, 
because they depend on
    + * the full set of all runtime classes.</p>
    + */
    +@Internal
    +public interface Executor {
    --- End diff --
    
    Can we rename it? It clashes with `java.util.concurrent.Executor`. Maybe 
something like `ProgramExecutor` or so.


> separate the flink-streaming-java module from flink-clients 
> ------------------------------------------------------------
>
>                 Key: FLINK-7065
>                 URL: https://issues.apache.org/jira/browse/FLINK-7065
>             Project: Flink
>          Issue Type: Improvement
>          Components: DataStream API
>            Reporter: Xu Pingyong
>            Assignee: Xu Pingyong
>
> Motivation:
>      It is not good that "flink-streaming-java" module depends on 
> "flink-clients". Flink-clients should see something in "flink-streaming-java".
> Related Change:
>       1. LocalStreamEnvironment and RemoteStreamEnvironment can also execute 
> a job by the executors(LocalExecutor and RemoteExecutor).  Introduce 
> StreamGraphExecutor which executors a streamGraph as PlanExecutor executors 
> the plan.  StreamGraphExecutor and PlanExecutor all extend Executor.
>       2. Introduce  StreamExecutionEnvironmentFactory which works similarly 
> to ContextEnvironmentFactory in flink-clients.
>           When a object of ContextEnvironmentFactory, 
> OptimizerPlanEnvironmentFactory or PreviewPlanEnvironmentFactory is set into 
> ExecutionEnvironment(by calling initializeContextEnvironment), the relevant 
> StreamEnvFactory is alsot set into StreamExecutionEnvironment. It is similar 
> when calling unsetContext.
>      



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to