Github user zohar-mizrahi commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3838#discussion_r117793843
  
    --- Diff: 
flink-libraries/flink-streaming-python/src/main/java/org/apache/flink/streaming/python/api/environment/PythonStreamExecutionEnvironment.java
 ---
    @@ -0,0 +1,442 @@
    +/*
    + * 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.streaming.python.api.environment;
    +
    +import org.apache.flink.annotation.Public;
    +import org.apache.flink.api.common.JobExecutionResult;
    +import org.apache.flink.api.common.cache.DistributedCache;
    +import org.apache.flink.api.java.tuple.Tuple2;
    +import org.apache.flink.api.java.typeutils.TypeExtractor;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.core.fs.FileSystem;
    +import org.apache.flink.core.fs.Path;
    +import org.apache.flink.runtime.filecache.FileCache;
    +import org.apache.flink.streaming.api.CheckpointingMode;
    +import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
    +import 
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
    +import org.apache.flink.streaming.api.functions.source.SourceFunction;
    +import org.apache.flink.streaming.python.api.datastream.PythonDataStream;
    +import 
org.apache.flink.streaming.python.api.functions.PythonGeneratorFunction;
    +import 
org.apache.flink.streaming.python.api.functions.PythonIteratorFunction;
    +import org.apache.flink.streaming.python.api.functions.UtilityFunctions;
    +import 
org.apache.flink.streaming.python.util.serialization.PyObjectSerializer;
    +import org.python.core.PyObject;
    +import org.python.core.PyString;
    +import org.python.core.PyInteger;
    +import org.python.core.PyLong;
    +import org.python.core.PyUnicode;
    +import org.python.core.PyTuple;
    +import org.python.core.PyObjectDerived;
    +import org.python.core.PyInstance;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.nio.file.Paths;
    +import java.util.Collection;
    +import java.util.Iterator;
    +import java.util.Random;
    +
    +
    +/**
    + * A thin wrapper layer over {@link StreamExecutionEnvironment}.
    + *
    + * <p>The PythonStreamExecutionEnvironment is the context in which a 
streaming program is executed.
    + * </p>
    + *
    + * <p>The environment provides methods to control the job execution (such 
as setting the parallelism
    + * or the fault tolerance/checkpointing parameters) and to interact with 
the outside world
    + * (data access).</p>
    + */
    +@Public
    +public class PythonStreamExecutionEnvironment {
    +   private final StreamExecutionEnvironment env;
    +   private static final Logger LOG = 
LoggerFactory.getLogger(PythonStreamExecutionEnvironment.class);
    +
    +   /**
    +    * A thin wrapper layer over {@link 
StreamExecutionEnvironment#getExecutionEnvironment()}. In addition it takes
    +    * care for required Jython serializers registration.
    +    *
    +    * @return The python execution environment of the context in which the 
program is
    +    * executed.
    +    */
    +   public static PythonStreamExecutionEnvironment 
get_execution_environment() {
    +           return new PythonStreamExecutionEnvironment();
    +   }
    +
    +   /**
    +    * Creates a {@link LocalStreamEnvironment}. The local execution 
environment
    +    * will run the program in a multi-threaded fashion in the same JVM as 
the
    +    * environment was created in. The default parallelism of the local
    +    * environment is the number of hardware contexts (CPU cores / threads),
    +    * unless it was specified differently by {@link #setParallelism(int)}.
    +    *
    +    * @param configuration
    +    *              Pass a custom configuration into the cluster
    +    * @return A local execution environment with the specified parallelism.
    +    */
    +   public static PythonStreamExecutionEnvironment 
create_local_execution_environment(Configuration config) {
    --- End diff --
    
    It is somehow refers to your first comment concerning the Python coding 
style versus maintainability. So, it came up to a point, where we better decide 
what would the best approach.
    As far as I see it there are 3 possible options (maybe you'll come up with 
others):
    1) Strict Python - wrap every single java class to provide uniform 
interface that comply with the Python coding style guide - this would require a 
lot of effort, even if we find a way to automatically generate this layer, 
because it'll always require some adaptations.
    2) Java interface - leave the Java interface as is. A python script would 
be written in Java coding style. IMHO, its not a good option.
    3) Mixed Python / Java - in this option, make sure the major APIs, and thus 
the script body, comply with the Python coding style. In certain circumstances, 
it is allowed to deviate from the Python coding style and use the Java objects 
directly. IMHO, it is a fair reasonable option, which has balance between 
coding style and maintenance. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to